diff --git a/.gitignore b/.gitignore index e89f6224eea5d6f7eb99bf068724d003628dbd4c..886d61b82e3c9731ac250f8d0ce94ea96f429bb6 100644 --- a/.gitignore +++ b/.gitignore @@ -36,6 +36,8 @@ flink-python/dev/.conda/ flink-python/dev/log/ flink-python/dev/.stage.txt flink-python/.eggs/ +flink-python/**/*.c +flink-python/**/*.so atlassian-ide-plugin.xml out/ /docs/api diff --git a/.travis.yml b/.travis.yml index 8d99bda37d5178ab33c59c573a82859ed4b4ac0a..3b4d212746979c57e9405f22c183f6a0e49ee50d 100644 --- a/.travis.yml +++ b/.travis.yml @@ -59,7 +59,8 @@ before_script: # Install maven 3.2.5 since xenial uses 3.3.9 for which shading is broken before_install: - - source ./tools/travis/setup_maven.sh + - source ./tools/ci/maven-utils.sh + - setup_maven notifications: slack: diff --git a/NOTICE b/NOTICE index 35f4f6e897016b47085aa7b1e73c60294cfa12f9..707f8d92f3a83b5ff8123e90bc175bdecd23dd4b 100644 --- a/NOTICE +++ b/NOTICE @@ -1,5 +1,5 @@ Apache Flink -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/azure-pipelines.yml b/azure-pipelines.yml index d486643a5dbf7160362a2410d07f44658f664fe3..d1966a93f8552ad981ec4a3e46154ac9e5adc183 100644 --- a/azure-pipelines.yml +++ b/azure-pipelines.yml @@ -29,20 +29,32 @@ # of the Azure Pipelines web ui. # +trigger: + branches: + include: + - '*' # must quote since "*" is a YAML reserved character; we want a string resources: containers: # Container with Maven 3.2.5, SSL to have the same environment everywhere. - container: flink-build-container - image: rmetzger/flink-ci:ubuntu-amd64-3528acd + image: rmetzger/flink-ci:ubuntu-amd64-bcef226 + # On AZP provided machines, set this flag to allow writing coredumps in docker + options: --privileged -# See tools/azure-pipelines/jobs-template.yml for a short summary of the caching +# Define variables: +# - See tools/azure-pipelines/jobs-template.yml for a short summary of the caching +# - See https://stackoverflow.com/questions/60742105/how-can-i-access-a-secret-value-from-an-azure-pipelines-expression +# to understand why the secrets are handled like this variables: MAVEN_CACHE_FOLDER: $(Pipeline.Workspace)/.m2/repository MAVEN_OPTS: '-Dmaven.repo.local=$(MAVEN_CACHE_FOLDER)' CACHE_KEY: maven | $(Agent.OS) | **/pom.xml, !**/target/** CACHE_FALLBACK_KEY: maven | $(Agent.OS) CACHE_FLINK_DIR: $(Pipeline.Workspace)/flink_cache + SECRET_S3_BUCKET: $[variables.IT_CASE_S3_BUCKET] + SECRET_S3_ACCESS_KEY: $[variables.IT_CASE_S3_ACCESS_KEY] + SECRET_S3_SECRET_KEY: $[variables.IT_CASE_S3_SECRET_KEY] jobs: @@ -53,7 +65,7 @@ jobs: vmImage: 'ubuntu-latest' e2e_pool_definition: vmImage: 'ubuntu-16.04' - environment: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11" + environment: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11 -Pe2e-hadoop" run_end_to_end: false container: flink-build-container jdk: jdk8 diff --git a/docs/_config.yml b/docs/_config.yml index 7d4dc7b526c8de5842f93171c9df5cbed4b8939b..d7df380c661da52cbe9b5b599eceb9bac1174bb0 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -64,17 +64,17 @@ is_stable: false show_outdated_warning: false previous_docs: - 1.10: http://ci.apache.org/projects/flink/flink-docs-release-1.10 - 1.9: http://ci.apache.org/projects/flink/flink-docs-release-1.9 - 1.8: http://ci.apache.org/projects/flink/flink-docs-release-1.8 - 1.7: http://ci.apache.org/projects/flink/flink-docs-release-1.7 - 1.6: http://ci.apache.org/projects/flink/flink-docs-release-1.6 - 1.5: http://ci.apache.org/projects/flink/flink-docs-release-1.5 - 1.4: http://ci.apache.org/projects/flink/flink-docs-release-1.4 - 1.3: http://ci.apache.org/projects/flink/flink-docs-release-1.3 - 1.2: http://ci.apache.org/projects/flink/flink-docs-release-1.2 - 1.1: http://ci.apache.org/projects/flink/flink-docs-release-1.1 - 1.0: http://ci.apache.org/projects/flink/flink-docs-release-1.0 + '1.10': http://ci.apache.org/projects/flink/flink-docs-release-1.10 + '1.9': http://ci.apache.org/projects/flink/flink-docs-release-1.9 + '1.8': http://ci.apache.org/projects/flink/flink-docs-release-1.8 + '1.7': http://ci.apache.org/projects/flink/flink-docs-release-1.7 + '1.6': http://ci.apache.org/projects/flink/flink-docs-release-1.6 + '1.5': http://ci.apache.org/projects/flink/flink-docs-release-1.5 + '1.4': http://ci.apache.org/projects/flink/flink-docs-release-1.4 + '1.3': http://ci.apache.org/projects/flink/flink-docs-release-1.3 + '1.2': http://ci.apache.org/projects/flink/flink-docs-release-1.2 + '1.1': http://ci.apache.org/projects/flink/flink-docs-release-1.1 + '1.0': http://ci.apache.org/projects/flink/flink-docs-release-1.0 #------------------------------------------------------------------------------ # BUILD CONFIG @@ -85,7 +85,6 @@ previous_docs: exclude: - "build_docs.sh" - - "build_docs.bat" - "check_links.sh" # Used in some documents to initialize arrays. Don't delete. diff --git a/docs/_config_dev_en.yml b/docs/_config_dev_en.yml index a7ca0d0c6d0afe334ee1808a8bb5a58e2e9bf457..eb11fbf39f3192b89559ed5e9d287eaa7506a134 100644 --- a/docs/_config_dev_en.yml +++ b/docs/_config_dev_en.yml @@ -18,7 +18,6 @@ exclude: - "*.zh.md" - "build_docs.sh" - - "build_docs.bat" - "check_links.sh" - "content" - "content_en" diff --git a/docs/_config_dev_zh.yml b/docs/_config_dev_zh.yml index 813a6361734a7431baed4d09038a2a191a44ec74..8b9ddeb88caaf51df11433564eeb3f546318e3c0 100644 --- a/docs/_config_dev_zh.yml +++ b/docs/_config_dev_zh.yml @@ -18,7 +18,6 @@ exclude: - "*.md" - "build_docs.sh" - - "build_docs.bat" - "check_links.sh" - "content" - "content_en" diff --git a/docs/_includes/generated/all_jobmanager_section.html b/docs/_includes/generated/all_jobmanager_section.html index ab01a856e032db1fd9acf753447b4c8b58dea8b0..6ef552c064896b53b48ea89a47c152ec04f49a9d 100644 --- a/docs/_includes/generated/all_jobmanager_section.html +++ b/docs/_includes/generated/all_jobmanager_section.html @@ -26,12 +26,6 @@ String This option specifies how the job computation recovers from task failures. Accepted values are: - -
jobmanager.heap.size
- "1024m" - String - JVM heap size for the JobManager. -
jobmanager.rpc.address
(none) diff --git a/docs/_includes/generated/all_taskmanager_section.html b/docs/_includes/generated/all_taskmanager_section.html index 692c38b6e22f659a0aa5f1081cc3af524696a776..186212aab42686447a76b91ac56225dc854850a6 100644 --- a/docs/_includes/generated/all_taskmanager_section.html +++ b/docs/_includes/generated/all_taskmanager_section.html @@ -30,7 +30,7 @@
taskmanager.data.port
0 Integer - 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. diff --git a/docs/_includes/generated/common_host_port_section.html b/docs/_includes/generated/common_host_port_section.html index 5deb71edc484fda84e711183ebb3d00055815081..fb5dd6e179621abd9e32998485f527670b9a55c6 100644 --- a/docs/_includes/generated/common_host_port_section.html +++ b/docs/_includes/generated/common_host_port_section.html @@ -54,19 +54,19 @@
taskmanager.data.port
0 Integer - 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. diff --git a/docs/_includes/generated/common_memory_section.html b/docs/_includes/generated/common_memory_section.html index 713edff3b253b979073723b895fbb0c0397be307..ce0878c69f22665d07af0fd2e4d8967139994a91 100644 --- a/docs/_includes/generated/common_memory_section.html +++ b/docs/_includes/generated/common_memory_section.html @@ -8,6 +8,54 @@ + +
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. +
taskmanager.memory.flink.size
(none) @@ -28,7 +76,7 @@
taskmanager.memory.jvm-metaspace.size
- 96 mb + 256 mb MemorySize JVM Metaspace Size for the TaskExecutors. diff --git a/docs/_includes/generated/common_miscellaneous_section.html b/docs/_includes/generated/common_miscellaneous_section.html index 0a4e15af8fd2d250feb7c14668eae4b5003e5fc1..547a05b25429e05dfaf638d3d249db18e44c27bc 100644 --- a/docs/_includes/generated/common_miscellaneous_section.html +++ b/docs/_includes/generated/common_miscellaneous_section.html @@ -8,6 +8,12 @@ + +
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) +
fs.default-scheme
(none) diff --git a/docs/_includes/generated/core_configuration.html b/docs/_includes/generated/core_configuration.html index cf68ddf65641eb003ed91b2c82be4659892db7b8..a9e7a7d98011b8c6ce4a384c6161e4c5598d6ff7 100644 --- a/docs/_includes/generated/core_configuration.html +++ b/docs/_includes/generated/core_configuration.html @@ -26,6 +26,12 @@ String 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) +
fs.default-scheme
(none) diff --git a/docs/_includes/generated/deployment_configuration.html b/docs/_includes/generated/deployment_configuration.html index 325dc982012b2e500cbc8676ed8373cb9d1b230f..1f50f6fd3f5aa7b0bef4e3f4117c8eb43ea05a44 100644 --- a/docs/_includes/generated/deployment_configuration.html +++ b/docs/_includes/generated/deployment_configuration.html @@ -14,6 +14,12 @@ Boolean Specifies if the pipeline is submitted in attached or detached mode. + +
execution.job-listeners
+ (none) + List<String> + Custom JobListeners to be registered with the execution environment. The registered listeners cannot have constructors with arguments. +
execution.shutdown-on-attached-exit
false diff --git a/docs/_includes/generated/environment_configuration.html b/docs/_includes/generated/environment_configuration.html index 32b4139359f73151f3350ce5c95b2f45137a34bf..912ecbb8525a8d9e3c640d94658533c1427bf6fb 100644 --- a/docs/_includes/generated/environment_configuration.html +++ b/docs/_includes/generated/environment_configuration.html @@ -20,6 +20,12 @@ String Java options to start the JVM of all Flink processes with. + +
env.java.opts.client
+ (none) + String + Java options to start the JVM of the Flink Client with. +
env.java.opts.historyserver
(none) diff --git a/docs/_includes/generated/execution_checkpointing_configuration.html b/docs/_includes/generated/execution_checkpointing_configuration.html index 59a84f45fd74d29523a191df7d29bedb164991f6..506fa9d08a72f80a28caac283461b9ce39fea859 100644 --- a/docs/_includes/generated/execution_checkpointing_configuration.html +++ b/docs/_includes/generated/execution_checkpointing_configuration.html @@ -56,5 +56,11 @@ Integer 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: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.
table.exec.sort.async-merge-enabled

Batch diff --git a/docs/_includes/generated/execution_configuration.html b/docs/_includes/generated/execution_configuration.html index 037960cdcb399007873b72adfbac41ab6a849128..f893a759f5cffd548a5d4f290473f7daea021cfe 100644 --- a/docs/_includes/generated/execution_configuration.html +++ b/docs/_includes/generated/execution_configuration.html @@ -20,5 +20,17 @@ Boolean Tells if we should use compression for the state snapshot data or not + +
execution.embedded-rpc-retry-period
+ 2 s + Duration + The retry period (in ms) between consecutive attempts to get the job status when executing applications in "Application Mode". + + +
execution.embedded-rpc-timeout
+ 1 h + Duration + The rpc timeout (in ms) when executing applications in "Application Mode". This affects all rpc's available through the Job Client and job submission. + diff --git a/docs/_includes/generated/job_manager_configuration.html b/docs/_includes/generated/job_manager_configuration.html index ef9c5652a64ba075e552a0b04da52289d4f7b35d..52cfa13baccd3f7b42d04731533e7f5f53558218 100644 --- a/docs/_includes/generated/job_manager_configuration.html +++ b/docs/_includes/generated/job_manager_configuration.html @@ -14,6 +14,12 @@ String 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: -
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. +
jobmanager.rpc.port
6123 diff --git a/docs/_includes/generated/kubernetes_config_configuration.html b/docs/_includes/generated/kubernetes_config_configuration.html index 86f8ec6b30d8102c9403301f9e161a2c835bca2c..cd38fc7cabb8a9cbf51e60444426fd22808f6f08 100644 --- a/docs/_includes/generated/kubernetes_config_configuration.html +++ b/docs/_includes/generated/kubernetes_config_configuration.html @@ -12,7 +12,7 @@
kubernetes.cluster-id
(none) String - 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 + diff --git a/docs/_includes/generated/netty_shuffle_environment_configuration.html b/docs/_includes/generated/netty_shuffle_environment_configuration.html index 901a40b3bbb7ea6ef659cff60a2a140c58e6d9e9..18a4028e1f8b9a71e9535de1481ed37f0e390bd4 100644 --- a/docs/_includes/generated/netty_shuffle_environment_configuration.html +++ b/docs/_includes/generated/netty_shuffle_environment_configuration.html @@ -8,11 +8,17 @@ + +
taskmanager.data.bind-port
+ (none) + Integer + The task manager's bind port used for data exchange operations. If not configured, 'taskmanager.data.port' will be used. +
taskmanager.data.port
0 Integer - 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
diff --git a/docs/_includes/generated/python_configuration.html b/docs/_includes/generated/python_configuration.html index 212cec28805c70d330f757b74008426fea7ca7f0..967cb661931ad161ac67d330b64a9739e9ea5908 100644 --- a/docs/_includes/generated/python_configuration.html +++ b/docs/_includes/generated/python_configuration.html @@ -8,9 +8,33 @@ + +
python.archives
+ (none) + String + 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". + diff --git a/docs/_includes/generated/resource_manager_configuration.html b/docs/_includes/generated/resource_manager_configuration.html index c1955f81cf66dba3d96a5b09d98fad37a368c721..cb79c38a7b31f141cc8b76ca3d025f0d23b326c6 100644 --- a/docs/_includes/generated/resource_manager_configuration.html +++ b/docs/_includes/generated/resource_manager_configuration.html @@ -8,18 +8,6 @@ - -
containerized.heap-cutoff-min
- 600 - Integer - Minimum amount of heap memory to remove in Job Master containers, as a safety margin. - - -
containerized.heap-cutoff-ratio
- 0.25 - Float - Percentage of heap space to remove from Job Master containers (YARN / Mesos / Kubernetes), to compensate for other JVM memory usage. -
resourcemanager.job.timeout
"5 minutes" diff --git a/docs/_includes/generated/rest_v1_dispatcher.html b/docs/_includes/generated/rest_v1_dispatcher.html index 113c37e480aae99abbf8f3f2edaa16f7366dde2f..c4b154d90b859ef6e8d192783ba8d89cad2b9851 100644 --- a/docs/_includes/generated/rest_v1_dispatcher.html +++ b/docs/_includes/generated/rest_v1_dispatcher.html @@ -99,6 +99,187 @@ + + + + + + + + + + + + + + + + + + + +
/datasets
Verb: GETResponse code: 200 OK
Returns all cluster data sets.
+ +
+
+            
+{}            
+          
+
+
+ +
+
+            
+{
+  "type" : "object",
+  "id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:dataset:ClusterDataSetListResponseBody",
+  "properties" : {
+    "dataSets" : {
+      "type" : "array",
+      "items" : {
+        "type" : "object",
+        "id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:dataset:ClusterDataSetEntry",
+        "properties" : {
+          "id" : {
+            "type" : "string"
+          },
+          "isComplete" : {
+            "type" : "boolean"
+          }
+        }
+      }
+    }
+  }
+}            
+          
+
+
+ + + + + + + + + + + + + + + + + + + + + + + + + +
/datasets/delete/:triggerid
Verb: GETResponse code: 200 OK
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.
  • +
+
+ +
+
+            
+{}            
+          
+
+
+ +
+
+            
+{
+  "type" : "object",
+  "id" : "urn:jsonschema:org:apache:flink:runtime:rest:handler:async:AsynchronousOperationResult",
+  "properties" : {
+    "operation" : {
+      "type" : "object",
+      "id" : "urn:jsonschema:org:apache:flink:runtime:rest:handler:async:AsynchronousOperationInfo",
+      "properties" : {
+        "failure-cause" : {
+          "type" : "any"
+        }
+      }
+    },
+    "status" : {
+      "type" : "object",
+      "id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:queue:QueueStatus",
+      "properties" : {
+        "id" : {
+          "type" : "string",
+          "required" : true,
+          "enum" : [ "IN_PROGRESS", "COMPLETED" ]
+        }
+      }
+    }
+  }
+}            
+          
+
+
+ + + + + + + + + + + + + + + + + + + + + + + + + +
/datasets/:datasetid
Verb: DELETEResponse code: 202 Accepted
Triggers the deletion of a cluster data set. This async operation would return a 'triggerid' for further query identifier.
Path parameters
+
    +
  • datasetid - 32-character hexadecimal string value that identifies a cluster data set.
  • +
+
+ +
+
+            
+{}            
+          
+
+
+ +
+
+            
+{
+  "type" : "object",
+  "id" : "urn:jsonschema:org:apache:flink:runtime:rest:handler:async:TriggerResponse",
+  "properties" : {
+    "request-id" : {
+      "type" : "any"
+    }
+  }
+}            
+          
+
+
@@ -599,6 +780,62 @@ Using 'curl' you can upload a jar via 'curl -X POST -H "Expect:" -F "jarfile=@pa
+ + + + + + + + + + + + + + + + + + + +
/jobmanager/logs
Verb: GETResponse code: 200 OK
Returns the list of log files on the JobManager.
+ +
+
+            
+{}            
+          
+
+
+ +
+
+            
+{
+  "type" : "object",
+  "id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:LogListInfo",
+  "properties" : {
+    "logs" : {
+      "type" : "array",
+      "items" : {
+        "type" : "object",
+        "id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:LogInfo",
+        "properties" : {
+          "name" : {
+            "type" : "string"
+          },
+          "size" : {
+            "type" : "integer"
+          }
+        }
+      }
+    }
+  }
+}            
+          
+
+
@@ -2256,7 +2493,13 @@ Using 'curl' you can upload a jar via 'curl -X POST -H "Expect:" -F "jarfile=@pa "id" : "urn:jsonschema:org:apache:flink:runtime:rest:handler:async:AsynchronousOperationResult", "properties" : { "operation" : { - "type" : "any" + "type" : "object", + "id" : "urn:jsonschema:org:apache:flink:runtime:rest:handler:async:AsynchronousOperationInfo", + "properties" : { + "failure-cause" : { + "type" : "any" + } + } }, "status" : { "type" : "object", @@ -2387,7 +2630,16 @@ Using 'curl' you can upload a jar via 'curl -X POST -H "Expect:" -F "jarfile=@pa "id" : "urn:jsonschema:org:apache:flink:runtime:rest:handler:async:AsynchronousOperationResult", "properties" : { "operation" : { - "type" : "any" + "type" : "object", + "id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:job:savepoints:SavepointInfo", + "properties" : { + "failure-cause" : { + "type" : "any" + }, + "location" : { + "type" : "string" + } + } }, "status" : { "type" : "object", @@ -3760,7 +4012,13 @@ Using 'curl' you can upload a jar via 'curl -X POST -H "Expect:" -F "jarfile=@pa "id" : "urn:jsonschema:org:apache:flink:runtime:rest:handler:async:AsynchronousOperationResult", "properties" : { "operation" : { - "type" : "any" + "type" : "object", + "id" : "urn:jsonschema:org:apache:flink:runtime:rest:handler:async:AsynchronousOperationInfo", + "properties" : { + "failure-cause" : { + "type" : "any" + } + } }, "status" : { "type" : "object", @@ -4070,6 +4328,72 @@ Using 'curl' you can upload a jar via 'curl -X POST -H "Expect:" -F "jarfile=@pa
+ + + + + + + + + + + + + + + + + + + + + + + + + +
/taskmanagers/:taskmanagerid/logs
Verb: GETResponse code: 200 OK
Returns the list of log files on a TaskManager.
Path parameters
+
    +
  • taskmanagerid - 32-character hexadecimal string that identifies a task manager.
  • +
+
+ +
+
+            
+{}            
+          
+
+
+ +
+
+            
+{
+  "type" : "object",
+  "id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:LogListInfo",
+  "properties" : {
+    "logs" : {
+      "type" : "array",
+      "items" : {
+        "type" : "object",
+        "id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:LogInfo",
+        "properties" : {
+          "name" : {
+            "type" : "string"
+          },
+          "size" : {
+            "type" : "integer"
+          }
+        }
+      }
+    }
+  }
+}            
+          
+
+
diff --git a/docs/_includes/generated/rocks_db_configuration.html b/docs/_includes/generated/rocks_db_configuration.html index 3ce04c75d2d98155be024936bfd30978f17d87ad..1648ab3725e99816008746e1c31969cb6bc35987 100644 --- a/docs/_includes/generated/rocks_db_configuration.html +++ b/docs/_includes/generated/rocks_db_configuration.html @@ -58,8 +58,8 @@ - - + + diff --git a/docs/_includes/generated/state_backend_rocksdb_section.html b/docs/_includes/generated/state_backend_rocksdb_section.html index 974c5c12abed3cf7765432929a13f384f208655a..a5c9e78e95625c5c5b2bdf049c8e2001929e8071 100644 --- a/docs/_includes/generated/state_backend_rocksdb_section.html +++ b/docs/_includes/generated/state_backend_rocksdb_section.html @@ -34,8 +34,8 @@ - - + + 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 @@ +
state.backend.rocksdb.timer-service.factory
"ROCKSDB"StringROCKSDB

Enum

Possible values: [HEAP, 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 .
state.backend.rocksdb.timer-service.factory
"ROCKSDB"StringROCKSDB

Enum

Possible values: [HEAP, 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 .
+ + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
table.dynamic-table-options.enabled

Batch Streaming
falseBooleanEnable 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

Batch Streaming
"default"StringThe 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
diff --git a/docs/_includes/generated/task_manager_configuration.html b/docs/_includes/generated/task_manager_configuration.html index db02deee1fe7fe98a5f0c41c3b542e525b232203..3980ed166d69a26bdbe266df4c9ba148cf848b12 100644 --- a/docs/_includes/generated/task_manager_configuration.html +++ b/docs/_includes/generated/task_manager_configuration.html @@ -26,6 +26,12 @@ Long 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. diff --git a/docs/_includes/generated/task_manager_memory_configuration.html b/docs/_includes/generated/task_manager_memory_configuration.html index a7777e475369984da8f04909c3de8e7d947538ff..3a25b14bbdbdd755da616e83c1beccaeabfdf3a3 100644 --- a/docs/_includes/generated/task_manager_memory_configuration.html +++ b/docs/_includes/generated/task_manager_memory_configuration.html @@ -28,7 +28,7 @@
taskmanager.memory.jvm-metaspace.size
- 96 mb + 256 mb MemorySize JVM Metaspace Size for the TaskExecutors. diff --git a/docs/_includes/generated/web_configuration.html b/docs/_includes/generated/web_configuration.html index f377f51a05fda3256fcb6bdbdebef5e708a6cd61..4b5a98d45af0572ee51cd07a3351cebd21e0d8f3 100644 --- a/docs/_includes/generated/web_configuration.html +++ b/docs/_includes/generated/web_configuration.html @@ -70,7 +70,7 @@
web.timeout
- 10000 + 600000 Long Timeout for asynchronous operations by the web monitor in milliseconds. diff --git a/docs/_includes/generated/yarn_config_configuration.html b/docs/_includes/generated/yarn_config_configuration.html index 3c32b856a0ee83bd5ae6bacaf90808da64a104d6..bdc83bba22b550bf9bb23954eb7e5f5481162ba0 100644 --- a/docs/_includes/generated/yarn_config_configuration.html +++ b/docs/_includes/generated/yarn_config_configuration.html @@ -110,6 +110,18 @@ String 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. Programming levels of abstraction - - 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. Programming levels of abstraction - - 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. + +State and Partitioning + +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. + +
+ Checkpoint barriers in data streams +
+ +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. + +
+ Aligning data streams at operators with multiple inputs +
+ +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 + +
+ Illustration of the Checkpointing Mechanism +
+ +#### 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. + +checkpoints and snapshots + +{% 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. - -A DataStream program, and its dataflow. - -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. - -A parallel dataflow - -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. + +Event Time and Processing Time + +{% 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. + +A data stream with events (in order) and watermarks + +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. + +A data stream with events (out of order) and watermarks + +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. + +Parallel data streams and operators with events and watermarks + +## 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). + +Time- and Count Windows + +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: - -
-
-{% highlight java %} -DataStream> input = // [...] -KeyedStream,Tuple> keyed = input.keyBy(0) -{% endhighlight %} -
-
-{% highlight scala %} -val input: DataStream[(Int, String, Long)] = // [...] -val keyed = input.keyBy(0) -{% endhighlight %} -
-
- -The tuples are grouped on the first field (the one of -Integer type). - -
-
-{% highlight java %} -DataStream> input = // [...] -KeyedStream,Tuple> keyed = input.keyBy(0,1) -{% endhighlight %} -
-
-{% highlight scala %} -val input: DataSet[(Int, String, Long)] = // [...] -val grouped = input.groupBy(0,1) -{% endhighlight %} -
-
- -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 Kafkaexactly onceUse the appropriate Kafka connector for your version精确一次根据你的版本用恰当的 Kafka 连接器
AWS Kinesis Streamsexactly once精确一次
RabbitMQat most once (v 0.10) / exactly once (v 1.0) 至多一次 (v 0.10) / 精确一次 (v 1.0)
Twitter Streaming APIat most once至多一次
Google PubSubat least once至少一次
Collectionsexactly once精确一次
Filesexactly once精确一次
Socketsat 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: - - + + - + - - + + - - + + - + - + - + - + - + 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:
{% highlight java %} @@ -143,22 +135,22 @@ env.addSource(pubsubSource)
-### 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. - -A data stream with events (in order) and watermarks - -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. - -A data stream with events (out of order) and watermarks - -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. - -Parallel data streams and operators with events and watermarks - -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`类,它决定是否接受一个事件只取决于事件自身的属性。
@@ -335,8 +332,7 @@ start.where(event => event.getName.startsWith("foo"))
-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() {
{% highlight scala %} -start.subtype(classOf[SubEvent]).where(subEvent => ... /* some condition */) +start.subtype(classOf[SubEvent]).where(subEvent => ... /* 一些判断条件 */) {% endhighlight %}
-**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()`方法。
@@ -365,12 +362,12 @@ start.subtype(classOf[SubEvent]).where(subEvent => ... /* some condition */) pattern.where(new SimpleCondition() { @Override public boolean filter(Event value) { - return ... // some condition + return ... // 一些判断条件 } }).or(new SimpleCondition() { @Override public boolean filter(Event value) { - return ... // or condition + return ... // 一些判断条件 } }); {% endhighlight %} @@ -378,45 +375,44 @@ pattern.where(new SimpleCondition() {
{% highlight scala %} -pattern.where(event => ... /* some condition */).or(event => ... /* or condition */) +pattern.where(event => ... /* 一些判断条件 */).or(event => ... /* 一些判断条件 */) {% endhighlight %}
-**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}`由于停止条件没有被输出。
HDFS BucketingSinkexactly onceImplementation depends on Hadoop version精确一次实现方法取决于 Hadoop 的版本
Elasticsearchat least once至少一次
Kafka producerat least once / exactly onceexactly once with transactional producers (v 0.11+)至少一次 / 精确一次当使用事务生产者时,保证精确一次 (v 0.11+)
Cassandra sinkat least once / exactly onceexactly once only for idempotent updates至少一次 / 精确一次只有当更新是幂等时,保证精确一次
AWS Kinesis Streamsat least once至少一次
File sinksexactly once精确一次
Socket sinksat least once至少一次
Standard outputat least once至少一次
Redis sinkat least once至少一次
- - + +
Pattern OperationDescription模式操作描述
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:

+

为当前模式定义一个条件。为了匹配这个模式,一个事件必须满足某些条件。 + 多个连续的where()语句取与组成判断条件:

{% highlight java %} pattern.where(new IterativeCondition() { @Override public boolean filter(Event value, Context ctx) throws Exception { - return ... // some condition + return ... // 一些判断条件 } }); {% endhighlight %} @@ -425,18 +421,17 @@ pattern.where(new IterativeCondition() {
or(condition) -

Adds a new condition which is ORed with an existing one. An event can match the pattern only if it - passes at least one of the conditions:

+

增加一个新的判断,和当前的判断取或。一个事件只要满足至少一个判断条件就匹配到模式:

{% highlight java %} pattern.where(new IterativeCondition() { @Override public boolean filter(Event value, Context ctx) throws Exception { - return ... // some condition + return ... // 一些判断条件 } }).or(new IterativeCondition() { @Override public boolean filter(Event value, Context ctx) throws Exception { - return ... // alternative condition + return ... // 替代条件 } }); {% endhighlight %} @@ -445,15 +440,14 @@ pattern.where(new IterativeCondition() {
until(condition) -

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.

+

为循环模式指定一个停止条件。意思是满足了给定的条件的事件出现后,就不会再有事件被接受进入模式了。

+

只适用于和oneOrMore()同时使用。

+

NOTE: 在基于事件的条件中,它可用于清理对应模式的状态。

{% highlight java %} pattern.oneOrMore().until(new IterativeCondition() { @Override public boolean filter(Event value, Context ctx) throws Exception { - return ... // alternative condition + return ... // 替代条件 } }); {% endhighlight %} @@ -462,8 +456,7 @@ pattern.oneOrMore().until(new IterativeCondition() {
subtype(subClass) -

Defines a subtype condition for the current pattern. An event can only match the pattern if it is - of this subtype:

+

为当前模式定义一个子类型条件。一个事件只有是这个子类型的时候才能匹配到模式:

{% highlight java %} pattern.subtype(SubEvent.class); {% endhighlight %} @@ -472,10 +465,10 @@ pattern.subtype(SubEvent.class);
oneOrMore() -

Specifies that this pattern expects at least one occurrence of a matching event.

-

By default a relaxed internal contiguity (between subsequent events) is used. For more info on - internal contiguity see consecutive.

-

NOTE: It is advised to use either until() or within() to enable state clearing

+

指定模式期望匹配到的事件至少出现一次。.

+

默认(在子事件间)使用松散的内部连续性。 + 关于内部连续性的更多信息可以参考连续性

+

NOTE: 推荐使用until()或者within()来清理状态。

{% highlight java %} pattern.oneOrMore(); {% endhighlight %} @@ -484,10 +477,9 @@ pattern.oneOrMore();
timesOrMore(#times) -

Specifies that this pattern expects at least #times occurrences - of a matching event.

-

By default a relaxed internal contiguity (between subsequent events) is used. For more info on - internal contiguity see consecutive.

+

指定模式期望匹配到的事件至少出现#times次。.

+

默认(在子事件间)使用松散的内部连续性。 + 关于内部连续性的更多信息可以参考连续性

{% highlight java %} pattern.timesOrMore(2); {% endhighlight %} @@ -496,9 +488,9 @@ pattern.timesOrMore(2);
times(#ofTimes) -

Specifies that this pattern expects an exact number of occurrences of a matching event.

-

By default a relaxed internal contiguity (between subsequent events) is used. For more info on - internal contiguity see consecutive.

+

指定模式期望匹配到的事件正好出现的次数。

+

默认(在子事件间)使用松散的内部连续性。 + 关于内部连续性的更多信息可以参考连续性

{% highlight java %} pattern.times(2); {% endhighlight %} @@ -507,10 +499,9 @@ pattern.times(2);
times(#fromTimes, #toTimes) -

Specifies that this pattern expects occurrences between #fromTimes - and #toTimes of a matching event.

-

By default a relaxed internal contiguity (between subsequent events) is used. For more info on - internal contiguity see consecutive.

+

指定模式期望匹配到的事件出现次数在#fromTimes#toTimes之间。

+

默认(在子事件间)使用松散的内部连续性。 + 关于内部连续性的更多信息可以参考连续性

{% highlight java %} pattern.times(2, 4); {% endhighlight %} @@ -519,8 +510,7 @@ pattern.times(2, 4);
optional() -

Specifies that this pattern is optional, i.e. it may not occur at all. This is applicable to all - aforementioned quantifiers.

+

指定这个模式是可选的,也就是说,它可能根本不出现。这对所有之前提到的量词都适用。

{% highlight java %} pattern.oneOrMore().optional(); {% endhighlight %} @@ -529,8 +519,7 @@ pattern.oneOrMore().optional();
greedy() -

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 java %} pattern.oneOrMore().greedy(); {% endhighlight %} @@ -544,8 +533,8 @@ pattern.oneOrMore().greedy(); - - + + @@ -553,41 +542,38 @@ pattern.oneOrMore().greedy(); - - +
Pattern OperationDescription模式操作描述
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:

+

为当前模式定义一个条件。为了匹配这个模式,一个事件必须满足某些条件。 + 多个连续的where()语句取与组成判断条件:

{% highlight scala %} -pattern.where(event => ... /* some condition */) +pattern.where(event => ... /* 一些判断条件 */) {% endhighlight %}
or(condition) -

Adds a new condition which is ORed with an existing one. An event can match the pattern only if it - passes at least one of the conditions:

+

增加一个新的判断,和当前的判断取或。一个事件只要满足至少一个判断条件就匹配到模式:

{% highlight scala %} -pattern.where(event => ... /* some condition */) - .or(event => ... /* alternative condition */) +pattern.where(event => ... /* 一些判断条件 */) + .or(event => ... /* 替代条件 */) {% endhighlight %}
until(condition) -

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.

+

为循环模式指定一个停止条件。意思是满足了给定的条件的事件出现后,就不会再有事件被接受进入模式了。

+

只适用于和oneOrMore()同时使用。

+

提示: 在基于事件的条件中,它可用于清理对应模式的状态。

{% highlight scala %} -pattern.oneOrMore().until(event => ... /* some condition */) +pattern.oneOrMore().until(event => ... /* 替代条件 */) {% endhighlight %}
subtype(subClass) -

Defines a subtype condition for the current pattern. An event can only match the pattern if it is - of this subtype:

+

为当前模式定义一个子类型条件。一个事件只有是这个子类型的时候才能匹配到模式:

{% highlight scala %} pattern.subtype(classOf[SubEvent]) {% endhighlight %} @@ -596,10 +582,10 @@ pattern.subtype(classOf[SubEvent])
oneOrMore() -

Specifies that this pattern expects at least one occurrence of a matching event.

-

By default a relaxed internal contiguity (between subsequent events) is used. For more info on - internal contiguity see consecutive.

-

NOTE: It is advised to use either until() or within() to enable state clearing

+

指定模式期望匹配到的事件至少出现一次。.

+

默认(在子事件间)使用松散的内部连续性。 + 关于内部连续性的更多信息可以参考连续性

+

提示: 推荐使用until()或者within()来清理状态。

{% highlight scala %} pattern.oneOrMore() {% endhighlight %} @@ -608,21 +594,20 @@ pattern.oneOrMore()
timesOrMore(#times) -

Specifies that this pattern expects at least #times occurrences - of a matching event.

-

By default a relaxed internal contiguity (between subsequent events) is used. For more info on - internal contiguity see consecutive.

+

指定模式期望匹配到的事件至少出现#times次。.

+

默认(在子事件间)使用松散的内部连续性。 + 关于内部连续性的更多信息可以参考连续性

{% highlight scala %} pattern.timesOrMore(2) {% endhighlight %}
times(#ofTimes) -

Specifies that this pattern expects an exact number of occurrences of a matching event.

-

By default a relaxed internal contiguity (between subsequent events) is used. - For more info on internal contiguity see consecutive.

+
times(#ofTimes) +

指定模式期望匹配到的事件正好出现的次数。

+

默认(在子事件间)使用松散的内部连续性。 + 关于内部连续性的更多信息可以参考连续性

{% highlight scala %} pattern.times(2) {% endhighlight %} @@ -631,10 +616,9 @@ pattern.times(2)
times(#fromTimes, #toTimes) -

Specifies that this pattern expects occurrences between #fromTimes - and #toTimes of a matching event.

-

By default a relaxed internal contiguity (between subsequent events) is used. For more info on - internal contiguity see consecutive.

+

指定模式期望匹配到的事件出现次数在#fromTimes#toTimes之间。

+

默认(在子事件间)使用松散的内部连续性。 + 关于内部连续性的更多信息可以参考连续性

{% highlight scala %} pattern.times(2, 4) {% endhighlight %} @@ -643,8 +627,7 @@ pattern.times(2, 4)
optional() -

Specifies that this pattern is optional, i.e. it may not occur at all. This is applicable to all - aforementioned quantifiers.

+

指定这个模式是可选的,也就是说,它可能根本不出现。这对所有之前提到的量词都适用。

{% highlight scala %} pattern.oneOrMore().optional() {% endhighlight %} @@ -653,8 +636,7 @@ pattern.oneOrMore().optional()
greedy() -

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: +模式序列由一个初始模式作为开头,如下所示:
@@ -686,48 +667,46 @@ val start : Pattern[Event, _] = Pattern.begin("start")
-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 java %} -// strict contiguity +// 严格连续 Pattern strict = start.next("middle").where(...); -// relaxed contiguity +// 松散连续 Pattern relaxed = start.followedBy("middle").where(...); -// non-deterministic relaxed contiguity +// 不确定的松散连续 Pattern nonDetermin = start.followedByAny("middle").where(...); -// NOT pattern with strict contiguity +// 严格连续的NOT模式 Pattern strictNot = start.notNext("not").where(...); -// NOT pattern with relaxed contiguity +// 松散连续的NOT模式 Pattern relaxedNot = start.notFollowedBy("not").where(...); {% endhighlight %} @@ -736,41 +715,39 @@ Pattern relaxedNot = start.notFollowedBy("not").where(...);
{% 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 注意 %} 一个模式序列只能有一个时间限制。如果限制了多个时间在不同的单个模式上,会使用最小的那个时间限制。
@@ -786,42 +763,41 @@ next.within(Time.seconds(10))
-#### 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()`方法。
- - + + @@ -890,19 +866,19 @@ Pattern.begin("start").where(new SimpleCondition() {
Pattern OperationDescription模式操作描述
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.

+

oneOrMore()times()一起使用, 在匹配的事件之间施加严格的连续性, + 也就是说,任何不匹配的事件都会终止匹配(和next()一样)。

+

如果不使用它,那么就是松散连续(和followedBy()一样)。

-

E.g. a pattern like:

+

例如,一个如下的模式:

{% highlight java %} Pattern.begin("start").where(new SimpleCondition() { @Override @@ -842,20 +818,20 @@ Pattern.begin("start").where(new SimpleCondition() { } }); {% endhighlight %} -

Will generate the following matches for an input sequence: C D A1 A2 A3 D A4 B

+

输入:C D A1 A2 A3 D A4 B,会产生下面的输出:

-

with consecutive applied: {C A1 B}, {C A1 A2 B}, {C A1 A2 A3 B}

-

without consecutive applied: {C A1 B}, {C A1 A2 B}, {C A1 A2 A3 B}, {C A1 A2 A3 A4 B}

+

如果施加严格连续性: {C A1 B},{C A1 A2 B},{C A1 A2 A3 B}

+

不施加严格连续性: {C A1 B},{C A1 A2 B},{C A1 A2 A3 B},{C A1 A2 A3 A4 B}

allowCombinations() -

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.

+

oneOrMore()times()一起使用, + 在匹配的事件中间施加不确定松散连续性(和followedByAny()一样)。

+

如果不使用,就是松散连续(和followedBy()一样)。

-

E.g. a pattern like:

+

例如,一个如下的模式:

{% highlight java %} Pattern.begin("start").where(new SimpleCondition() { @Override @@ -876,10 +852,10 @@ Pattern.begin("start").where(new SimpleCondition() { } }); {% endhighlight %} -

Will generate the following matches for an input sequence: C D A1 A2 A3 D A4 B

+

输入:C D A1 A2 A3 D A4 B,会产生如下的输出:

-

with combinations enabled: {C A1 B}, {C A1 A2 B}, {C A1 A3 B}, {C A1 A4 B}, {C A1 A2 A3 B}, {C A1 A2 A4 B}, {C A1 A3 A4 B}, {C A1 A2 A3 A4 B}

-

without combinations enabled: {C A1 B}, {C A1 A2 B}, {C A1 A2 A3 B}, {C A1 A2 A3 A4 B}

+

如果使用不确定松散连续: {C A1 B},{C A1 A2 B},{C A1 A3 B},{C A1 A4 B},{C A1 A2 A3 B},{C A1 A2 A4 B},{C A1 A3 A4 B},{C A1 A2 A3 A4 B}

+

如果不使用:{C A1 B},{C A1 A2 B},{C A1 A2 A3 B},{C A1 A2 A3 A4 B}

- - + + @@ -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()`。
@@ -957,17 +932,17 @@ Pattern start = Pattern.begin( Pattern.begin("start").where(...).followedBy("start_middle").where(...) ); -// strict contiguity +// 严格连续 Pattern strict = start.next( Pattern.begin("next_start").where(...).followedBy("next_middle").where(...) ).times(3); -// relaxed contiguity +// 松散连续 Pattern relaxed = start.followedBy( Pattern.begin("followedby_start").where(...).followedBy("followedby_middle").where(...) ).oneOrMore(); -// non-deterministic relaxed contiguity +// 不确定松散连续 Pattern nonDetermin = start.followedByAny( Pattern.begin("followedbyany_start").where(...).followedBy("followedbyany_middle").where(...) ).optional(); @@ -982,17 +957,17 @@ val start: Pattern[Event, _] = Pattern.begin( Pattern.begin[Event]("start").where(...).followedBy("start_middle").where(...) ) -// strict contiguity +// 严格连续 val strict: Pattern[Event, _] = start.next( Pattern.begin[Event]("next_start").where(...).followedBy("next_middle").where(...) ).times(3) -// relaxed contiguity +// 松散连续 val relaxed: Pattern[Event, _] = start.followedBy( Pattern.begin[Event]("followedby_start").where(...).followedBy("followedby_middle").where(...) ).oneOrMore() -// non-deterministic relaxed contiguity +// 不确定松散连续 val nonDetermin: Pattern[Event, _] = start.followedByAny( Pattern.begin[Event]("followedbyany_start").where(...).followedBy("followedbyany_middle").where(...) ).optional() @@ -1008,15 +983,15 @@ val nonDetermin: Pattern[Event, _] = start.followedByAny(
Pattern OperationDescription模式操作描述
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.

+

oneOrMore()times()一起使用, 在匹配的事件之间施加严格的连续性, + 也就是说,任何不匹配的事件都会终止匹配(和next()一样)。

+

如果不使用它,那么就是松散连续(和followedBy()一样)。

-

E.g. a pattern like:

+

例如,一个如下的模式:

{% highlight scala %} Pattern.begin("start").where(_.getName().equals("c")) .followedBy("middle").where(_.getName().equals("a")) @@ -910,20 +886,20 @@ Pattern.begin("start").where(_.getName().equals("c")) .followedBy("end1").where(_.getName().equals("b")) {% endhighlight %} -

Will generate the following matches for an input sequence: C D A1 A2 A3 D A4 B

+

输入:C D A1 A2 A3 D A4 B,会产生下面的输出:

-

with consecutive applied: {C A1 B}, {C A1 A2 B}, {C A1 A2 A3 B}

-

without consecutive applied: {C A1 B}, {C A1 A2 B}, {C A1 A2 A3 B}, {C A1 A2 A3 A4 B}

+

如果施加严格连续性: {C A1 B},{C A1 A2 B},{C A1 A2 A3 B}

+

不施加严格连续性: {C A1 B},{C A1 A2 B},{C A1 A2 A3 B},{C A1 A2 A3 A4 B}

allowCombinations() -

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.

+

oneOrMore()times()一起使用, + 在匹配的事件中间施加不确定松散连续性(和followedByAny()一样)。

+

如果不使用,就是松散连续(和followedBy()一样)。

-

E.g. a pattern like:

+

例如,一个如下的模式:

{% highlight scala %} Pattern.begin("start").where(_.getName().equals("c")) .followedBy("middle").where(_.getName().equals("a")) @@ -931,10 +907,10 @@ Pattern.begin("start").where(_.getName().equals("c")) .followedBy("end1").where(_.getName().equals("b")) {% endhighlight %} -

Will generate the following matches for an input sequence: C D A1 A2 A3 D A4 B

+

输入:C D A1 A2 A3 D A4 B,会产生如下的输出:

-

with combinations enabled: {C A1 B}, {C A1 A2 B}, {C A1 A3 B}, {C A1 A4 B}, {C A1 A2 A3 B}, {C A1 A2 A4 B}, {C A1 A3 A4 B}, {C A1 A2 A3 A4 B}

-

without combinations enabled: {C A1 B}, {C A1 A2 B}, {C A1 A2 A3 B}, {C A1 A2 A3 A4 B}

+

如果使用不确定松散连续: {C A1 B},{C A1 A2 B},{C A1 A3 B},{C A1 A4 B},{C A1 A2 A3 B},{C A1 A2 A4 B},{C A1 A3 A4 B},{C A1 A2 A3 A4 B}

+

如果不使用:{C A1 B},{C A1 A2 B},{C A1 A2 A3 B},{C A1 A2 A3 A4 B}

- - + +
Pattern OperationDescription模式操作描述
begin(#name) -

Defines a starting pattern:

+

定义一个开始的模式:

{% highlight java %} Pattern start = Pattern.begin("start"); {% endhighlight %} @@ -1025,7 +1000,7 @@ Pattern start = Pattern.begin("start");
begin(#pattern_sequence) -

Defines a starting pattern:

+

定义一个开始的模式:

{% highlight java %} Pattern start = Pattern.begin( Pattern.begin("start").where(...).followedBy("middle").where(...) @@ -1036,8 +1011,7 @@ Pattern start = Pattern.begin(
next(#name) -

Appends a new pattern. A matching event has to directly succeed the previous matching event - (strict contiguity):

+

增加一个新的模式。匹配的事件必须是直接跟在前面匹配到的事件后面(严格连续):

{% highlight java %} Pattern next = start.next("middle"); {% endhighlight %} @@ -1046,8 +1020,7 @@ Pattern 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 java %} Pattern next = start.next( Pattern.begin("start").where(...).followedBy("middle").where(...) @@ -1058,8 +1031,7 @@ Pattern 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 java %} Pattern followedBy = start.followedBy("middle"); {% endhighlight %} @@ -1068,8 +1040,7 @@ Pattern 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 java %} Pattern followedBy = start.followedBy( Pattern.begin("start").where(...).followedBy("middle").where(...) @@ -1080,9 +1051,8 @@ Pattern 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 java %} Pattern followedByAny = start.followedByAny("middle"); {% endhighlight %} @@ -1091,9 +1061,8 @@ Pattern 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 java %} Pattern followedByAny = start.followedByAny( Pattern.begin("start").where(...).followedBy("middle").where(...) @@ -1104,8 +1073,7 @@ Pattern 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 java %} Pattern notNext = start.notNext("not"); {% endhighlight %} @@ -1114,9 +1082,8 @@ Pattern 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 java %} Pattern notFollowedBy = start.notFollowedBy("not"); {% endhighlight %} @@ -1125,8 +1092,7 @@ Pattern 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 java %} pattern.within(Time.seconds(10)); {% endhighlight %} @@ -1140,15 +1106,15 @@ pattern.within(Time.seconds(10)); - - + + diff --git a/docs/dev/table/sql/queries.zh.md b/docs/dev/table/sql/queries.zh.md index 0d68ae4671ac7dc2ec75951752ba9e220e1245ce..711f15caf3a57dcf9c437f8c6aa3e9b97577c5e5 100644 --- a/docs/dev/table/sql/queries.zh.md +++ b/docs/dev/table/sql/queries.zh.md @@ -406,7 +406,7 @@ SELECT PRETTY_PRINT(user) FROM Orders @@ -545,7 +547,7 @@ FROM Orders INNER JOIN Product ON Orders.productId = Product.id - + diff --git a/flink-runtime-web/web-dashboard/src/app/pages/job-manager/logs/job-manager-logs.component.html b/flink-runtime-web/web-dashboard/src/app/pages/job-manager/logs/job-manager-logs.component.html index 6ba7783345fbf039af4c75c254fc7dcab1464b9b..f2f9461fe2a357bd8c31b5e189f9bfb193b094e9 100644 --- a/flink-runtime-web/web-dashboard/src/app/pages/job-manager/logs/job-manager-logs.component.html +++ b/flink-runtime-web/web-dashboard/src/app/pages/job-manager/logs/job-manager-logs.component.html @@ -17,4 +17,4 @@ --> - + diff --git a/flink-runtime-web/web-dashboard/src/app/pages/job-manager/stdout/job-manager-stdout.component.html b/flink-runtime-web/web-dashboard/src/app/pages/job-manager/stdout/job-manager-stdout.component.html index 4221237c1533fa8f30b787976ae0fcb370132507..8528b2b0df9fee3309cc9273b8c539055dd72a33 100644 --- a/flink-runtime-web/web-dashboard/src/app/pages/job-manager/stdout/job-manager-stdout.component.html +++ b/flink-runtime-web/web-dashboard/src/app/pages/job-manager/stdout/job-manager-stdout.component.html @@ -17,4 +17,4 @@ --> - + diff --git a/flink-runtime-web/web-dashboard/src/app/pages/job/exceptions/job-exceptions.component.html b/flink-runtime-web/web-dashboard/src/app/pages/job/exceptions/job-exceptions.component.html index f3ee3bd56017b5d60faff43e1098c57cb4311ebb..ea58a7f3f0c23eab39c47947285a9fca77742025 100644 --- a/flink-runtime-web/web-dashboard/src/app/pages/job/exceptions/job-exceptions.component.html +++ b/flink-runtime-web/web-dashboard/src/app/pages/job/exceptions/job-exceptions.component.html @@ -53,6 +53,11 @@ + + + diff --git a/flink-runtime-web/web-dashboard/src/app/pages/job/exceptions/job-exceptions.component.ts b/flink-runtime-web/web-dashboard/src/app/pages/job/exceptions/job-exceptions.component.ts index fd955369deeb1fe6720ebd3f897b4ee1573879f5..cb726044069c7fa2c5ac5cb836b162092bca1b56 100644 --- a/flink-runtime-web/web-dashboard/src/app/pages/job/exceptions/job-exceptions.component.ts +++ b/flink-runtime-web/web-dashboard/src/app/pages/job/exceptions/job-exceptions.component.ts @@ -19,7 +19,7 @@ import { formatDate } from '@angular/common'; import { Component, OnInit, ChangeDetectionStrategy, ChangeDetectorRef } from '@angular/core'; import { JobExceptionItemInterface } from 'interfaces'; -import { distinctUntilChanged, flatMap } from 'rxjs/operators'; +import { distinctUntilChanged, flatMap, tap } from 'rxjs/operators'; import { JobService } from 'services'; @Component({ @@ -31,18 +31,24 @@ import { JobService } from 'services'; export class JobExceptionsComponent implements OnInit { rootException = ''; listOfException: JobExceptionItemInterface[] = []; + truncated = false; + isLoading = false; + maxExceptions = 0; trackExceptionBy(_: number, node: JobExceptionItemInterface) { return node.timestamp; } - - constructor(private jobService: JobService, private cdr: ChangeDetectorRef) {} - - ngOnInit() { + loadMore() { + this.isLoading = true; + this.maxExceptions += 10; this.jobService.jobDetail$ .pipe( distinctUntilChanged((pre, next) => pre.jid === next.jid), - flatMap(job => this.jobService.loadExceptions(job.jid)) + flatMap(job => this.jobService.loadExceptions(job.jid, this.maxExceptions)), + tap(() => { + this.isLoading = false; + this.cdr.markForCheck(); + }) ) .subscribe(data => { // @ts-ignore @@ -51,8 +57,14 @@ export class JobExceptionsComponent implements OnInit { } else { this.rootException = 'No Root Exception'; } + this.truncated = data.truncated; this.listOfException = data['all-exceptions']; - this.cdr.markForCheck(); }); } + + constructor(private jobService: JobService, private cdr: ChangeDetectorRef) {} + + ngOnInit() { + this.loadMore(); + } } diff --git a/flink-runtime-web/web-dashboard/src/app/pages/job/overview/accumulators/job-overview-drawer-accumulators.component.html b/flink-runtime-web/web-dashboard/src/app/pages/job/overview/accumulators/job-overview-drawer-accumulators.component.html index c750094b67cb53f7e936776172aa0e70830fef14..fe1585ec823ddc94557c550e75b71e253d511055 100644 --- a/flink-runtime-web/web-dashboard/src/app/pages/job/overview/accumulators/job-overview-drawer-accumulators.component.html +++ b/flink-runtime-web/web-dashboard/src/app/pages/job/overview/accumulators/job-overview-drawer-accumulators.component.html @@ -37,7 +37,7 @@ - + diff --git a/flink-runtime-web/web-dashboard/src/app/pages/task-manager/log-detail/task-manager-log-detail.component.html b/flink-runtime-web/web-dashboard/src/app/pages/task-manager/log-detail/task-manager-log-detail.component.html new file mode 100644 index 0000000000000000000000000000000000000000..b5a46adb399e277cd40f7ccfa7a6c69f64b71b71 --- /dev/null +++ b/flink-runtime-web/web-dashboard/src/app/pages/task-manager/log-detail/task-manager-log-detail.component.html @@ -0,0 +1,36 @@ + + + + diff --git a/flink-runtime-web/web-dashboard/src/app/pages/task-manager/log-detail/task-manager-log-detail.component.less b/flink-runtime-web/web-dashboard/src/app/pages/task-manager/log-detail/task-manager-log-detail.component.less new file mode 100644 index 0000000000000000000000000000000000000000..d9bd69a31ff1bb318c635968343c6a09e998d8bc --- /dev/null +++ b/flink-runtime-web/web-dashboard/src/app/pages/task-manager/log-detail/task-manager-log-detail.component.less @@ -0,0 +1,54 @@ +/* + * 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. + */ + +@import "theme"; + +:host { + display: block; + height: 100%; + &.full-screen { + position: fixed; + top: 0; + bottom: 0; + right: 0; + left: 0; + background: @component-background; + z-index: 99; + flink-monaco-editor { + height: calc(~"100vh - 65px"); + } + } +} + +flink-monaco-editor { + height: calc(~"100vh - 386px"); + border: 1px solid @border-color-split; +} + +.breadcrumb { + background: @component-background; + border: 1px solid @border-color-split; + margin-bottom: 16px; + padding: 12px 24px; + position: relative; +} + +flink-refresh-download { + position: absolute; + right: 12px; + top: 0; + line-height: 47px; +} diff --git a/flink-runtime-web/web-dashboard/src/app/pages/task-manager/log-detail/task-manager-log-detail.component.ts b/flink-runtime-web/web-dashboard/src/app/pages/task-manager/log-detail/task-manager-log-detail.component.ts new file mode 100644 index 0000000000000000000000000000000000000000..f589122c7dddcd7a6d6c47666e216b35c381f678 --- /dev/null +++ b/flink-runtime-web/web-dashboard/src/app/pages/task-manager/log-detail/task-manager-log-detail.component.ts @@ -0,0 +1,89 @@ +/* + * 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. + */ + +import { ChangeDetectionStrategy, ChangeDetectorRef, Component, OnInit, ViewChild } from '@angular/core'; +import { ActivatedRoute } from '@angular/router'; +import { TaskManagerDetailInterface } from 'interfaces'; +import { TaskManagerService } from 'services'; +import { first } from 'rxjs/operators'; +import { MonacoEditorComponent } from 'share/common/monaco-editor/monaco-editor.component'; + +@Component({ + selector: 'flink-task-manager-log-detail', + templateUrl: './task-manager-log-detail.component.html', + changeDetection: ChangeDetectionStrategy.OnPush, + host: { + '[class.full-screen]': 'isFullScreen' + }, + styleUrls: ['./task-manager-log-detail.component.less'] +}) +export class TaskManagerLogDetailComponent implements OnInit { + logs = ''; + logName = ''; + downloadUrl = ''; + isLoading = false; + taskManagerDetail: TaskManagerDetailInterface; + isFullScreen = false; + hasLogName = false; + @ViewChild(MonacoEditorComponent) monacoEditorComponent: MonacoEditorComponent; + + constructor( + private taskManagerService: TaskManagerService, + private cdr: ChangeDetectorRef, + private activatedRoute: ActivatedRoute + ) {} + + reloadLog() { + this.isLoading = true; + this.cdr.markForCheck(); + this.taskManagerService.loadLog(this.taskManagerDetail.id, this.logName, this.hasLogName).subscribe( + data => { + this.logs = data.data; + this.downloadUrl = data.url; + this.isLoading = false; + this.layoutEditor(); + this.cdr.markForCheck(); + }, + () => { + this.isLoading = false; + this.layoutEditor(); + this.cdr.markForCheck(); + } + ); + } + + toggleFullScreen(fullScreen: boolean) { + this.isFullScreen = fullScreen; + this.layoutEditor(); + } + + layoutEditor(): void { + setTimeout(() => this.monacoEditorComponent.layout()); + } + + ngOnInit() { + this.taskManagerService.taskManagerDetail$.pipe(first()).subscribe(data => { + this.taskManagerDetail = data; + this.hasLogName = this.activatedRoute.snapshot.data.hasLogName; + if (this.hasLogName) { + this.logName = this.activatedRoute.snapshot.params.logName; + } else { + this.logName = `taskmanager_${data.id}_log`; + } + this.reloadLog(); + }); + } +} diff --git a/flink-runtime-web/web-dashboard/src/app/pages/task-manager/log-list/task-manager-log-list.component.html b/flink-runtime-web/web-dashboard/src/app/pages/task-manager/log-list/task-manager-log-list.component.html new file mode 100644 index 0000000000000000000000000000000000000000..c40615626936c3dc4805b978756a4da706317d77 --- /dev/null +++ b/flink-runtime-web/web-dashboard/src/app/pages/task-manager/log-list/task-manager-log-list.component.html @@ -0,0 +1,39 @@ + + + + + + + + + + + + + + + + + + diff --git a/flink-runtime-web/web-dashboard/src/app/pages/task-manager/log-list/task-manager-log-list.component.ts b/flink-runtime-web/web-dashboard/src/app/pages/task-manager/log-list/task-manager-log-list.component.ts new file mode 100644 index 0000000000000000000000000000000000000000..f565759500b41366674954e853a4f65d50c676ea --- /dev/null +++ b/flink-runtime-web/web-dashboard/src/app/pages/task-manager/log-list/task-manager-log-list.component.ts @@ -0,0 +1,50 @@ +/* + * 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. + */ + +import { ChangeDetectionStrategy, ChangeDetectorRef, Component, OnInit } from '@angular/core'; +import { first, flatMap } from 'rxjs/operators'; +import { TaskManagerService } from 'services'; + +@Component({ + selector: 'flink-task-manager-log-list', + templateUrl: './task-manager-log-list.component.html', + changeDetection: ChangeDetectionStrategy.OnPush +}) +export class TaskManagerLogListComponent implements OnInit { + listOfLog: { name: string; size: number }[] = []; + isLoading = true; + + constructor(private taskManagerService: TaskManagerService, private cdr: ChangeDetectorRef) {} + + ngOnInit() { + this.taskManagerService.taskManagerDetail$ + .pipe( + first(), + flatMap(data => this.taskManagerService.loadLogList(data.id)) + ) + .subscribe( + data => { + this.listOfLog = data; + this.isLoading = false; + this.cdr.markForCheck(); + }, + () => { + this.isLoading = false; + this.cdr.markForCheck(); + } + ); + } +} diff --git a/flink-runtime-web/web-dashboard/src/app/pages/task-manager/status/task-manager-status.component.ts b/flink-runtime-web/web-dashboard/src/app/pages/task-manager/status/task-manager-status.component.ts index d54e4ec0777f6673a7e3e3bc6b1f94186c2fa2ff..b244445a23fe135b5742ff0706d59555418e7987 100644 --- a/flink-runtime-web/web-dashboard/src/app/pages/task-manager/status/task-manager-status.component.ts +++ b/flink-runtime-web/web-dashboard/src/app/pages/task-manager/status/task-manager-status.component.ts @@ -30,11 +30,7 @@ import { TaskManagerService } from 'services'; }) export class TaskManagerStatusComponent implements OnInit, OnDestroy { @Input() isLoading = true; - listOfNavigation = [ - { path: 'metrics', title: 'Metrics' }, - { path: 'logs', title: 'Logs' }, - { path: 'stdout', title: 'Stdout' } - ]; + listOfNavigation = [{ path: 'metrics', title: 'Metrics' }, { path: 'log-list', title: 'Log' }]; taskManagerDetail: TaskManagerDetailInterface; private destroy$ = new Subject(); diff --git a/flink-runtime-web/web-dashboard/src/app/pages/task-manager/stdout/task-manager-stdout.component.ts b/flink-runtime-web/web-dashboard/src/app/pages/task-manager/stdout/task-manager-stdout.component.ts deleted file mode 100644 index a674f09351d2bf476aef5a14c3846b555ef42376..0000000000000000000000000000000000000000 --- a/flink-runtime-web/web-dashboard/src/app/pages/task-manager/stdout/task-manager-stdout.component.ts +++ /dev/null @@ -1,60 +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. - */ - -import { ChangeDetectionStrategy, ChangeDetectorRef, Component, OnInit, ViewChild } from '@angular/core'; -import { TaskManagerDetailInterface } from 'interfaces'; -import { first } from 'rxjs/operators'; -import { TaskManagerService } from 'services'; -import { MonacoEditorComponent } from 'share/common/monaco-editor/monaco-editor.component'; - -@Component({ - selector: 'flink-task-manager-stdout', - templateUrl: './task-manager-stdout.component.html', - styleUrls: ['./task-manager-stdout.component.less'], - changeDetection: ChangeDetectionStrategy.OnPush -}) -export class TaskManagerStdoutComponent implements OnInit { - @ViewChild(MonacoEditorComponent) monacoEditorComponent: MonacoEditorComponent; - stdout = ''; - taskManagerDetail: TaskManagerDetailInterface; - - reload() { - if (this.taskManagerDetail) { - this.taskManagerService.loadStdout(this.taskManagerDetail.id).subscribe( - data => { - this.monacoEditorComponent.layout(); - this.stdout = data; - this.cdr.markForCheck(); - }, - () => { - this.cdr.markForCheck(); - } - ); - } - } - - constructor(private taskManagerService: TaskManagerService, private cdr: ChangeDetectorRef) {} - - ngOnInit() { - this.taskManagerService.taskManagerDetail$.pipe(first()).subscribe(data => { - this.taskManagerDetail = data; - this.reload(); - this.cdr.markForCheck(); - }); - } -} diff --git a/flink-runtime-web/web-dashboard/src/app/pages/task-manager/task-manager-routing.module.ts b/flink-runtime-web/web-dashboard/src/app/pages/task-manager/task-manager-routing.module.ts index 0f5c7113ee2a3ed1949802789b2500ee255843ed..25e0ae107ce81ded90046c6a749db6e5fbd7cd68 100644 --- a/flink-runtime-web/web-dashboard/src/app/pages/task-manager/task-manager-routing.module.ts +++ b/flink-runtime-web/web-dashboard/src/app/pages/task-manager/task-manager-routing.module.ts @@ -18,11 +18,11 @@ import { NgModule } from '@angular/core'; import { Routes, RouterModule } from '@angular/router'; +import { TaskManagerLogDetailComponent } from './log-detail/task-manager-log-detail.component'; +import { TaskManagerLogListComponent } from './log-list/task-manager-log-list.component'; import { TaskManagerComponent } from './task-manager.component'; import { TaskManagerListComponent } from './list/task-manager-list.component'; -import { TaskManagerLogsComponent } from './logs/task-manager-logs.component'; import { TaskManagerMetricsComponent } from './metrics/task-manager-metrics.component'; -import { TaskManagerStdoutComponent } from './stdout/task-manager-stdout.component'; const routes: Routes = [ { @@ -41,17 +41,26 @@ const routes: Routes = [ } }, { - path: 'logs', - component: TaskManagerLogsComponent, + path: 'log-list', + component: TaskManagerLogListComponent, + data: { + path: 'log-list' + } + }, + { + path: 'log-list/:logName', + component: TaskManagerLogDetailComponent, data: { - path: 'logs' + path: 'log-list', + hasLogName: true } }, { - path: 'stdout', - component: TaskManagerStdoutComponent, + path: 'logs', + component: TaskManagerLogDetailComponent, data: { - path: 'stdout' + path: 'log-list', + hasLogName: false } }, { diff --git a/flink-runtime-web/web-dashboard/src/app/pages/task-manager/task-manager.module.ts b/flink-runtime-web/web-dashboard/src/app/pages/task-manager/task-manager.module.ts index 0a521b7f46c70e4f9505ed5c96e5a89182c4bb51..d2ea2cbe2da1a0af4bf2484880a1f7712d1e231f 100644 --- a/flink-runtime-web/web-dashboard/src/app/pages/task-manager/task-manager.module.ts +++ b/flink-runtime-web/web-dashboard/src/app/pages/task-manager/task-manager.module.ts @@ -19,14 +19,14 @@ import { NgModule } from '@angular/core'; import { CommonModule } from '@angular/common'; import { ShareModule } from 'share/share.module'; +import { TaskManagerLogDetailComponent } from './log-detail/task-manager-log-detail.component'; +import { TaskManagerLogListComponent } from './log-list/task-manager-log-list.component'; import { TaskManagerRoutingModule } from './task-manager-routing.module'; import { TaskManagerListComponent } from './list/task-manager-list.component'; import { TaskManagerMetricsComponent } from './metrics/task-manager-metrics.component'; import { TaskManagerComponent } from './task-manager.component'; import { TaskManagerStatusComponent } from './status/task-manager-status.component'; -import { TaskManagerLogsComponent } from './logs/task-manager-logs.component'; -import { TaskManagerStdoutComponent } from './stdout/task-manager-stdout.component'; @NgModule({ imports: [CommonModule, ShareModule, TaskManagerRoutingModule], @@ -35,8 +35,8 @@ import { TaskManagerStdoutComponent } from './stdout/task-manager-stdout.compone TaskManagerMetricsComponent, TaskManagerComponent, TaskManagerStatusComponent, - TaskManagerLogsComponent, - TaskManagerStdoutComponent + TaskManagerLogListComponent, + TaskManagerLogDetailComponent ] }) export class TaskManagerModule {} diff --git a/flink-runtime-web/web-dashboard/src/app/services/job.service.ts b/flink-runtime-web/web-dashboard/src/app/services/job.service.ts index 4abaa3f04dbc29cebba2519c28007b306fa3a9d6..e1d057e5167d51f1095ba87e3d3fe79600ebbfe9 100644 --- a/flink-runtime-web/web-dashboard/src/app/services/job.service.ts +++ b/flink-runtime-web/web-dashboard/src/app/services/job.service.ts @@ -155,9 +155,12 @@ export class JobService { /** * Get job exception * @param jobId + * @param maxExceptions */ - loadExceptions(jobId: string) { - return this.httpClient.get(`${BASE_URL}/jobs/${jobId}/exceptions`); + loadExceptions(jobId: string, maxExceptions: number) { + return this.httpClient.get( + `${BASE_URL}/jobs/${jobId}/exceptions?maxExceptions=${maxExceptions}` + ); } /** diff --git a/flink-runtime-web/web-dashboard/src/app/services/task-manager.service.ts b/flink-runtime-web/web-dashboard/src/app/services/task-manager.service.ts index f2b3ff96863a36a0f45fa3957fead64d1ee6aa5a..bd790bf07aee3e7c086f8d429e915a760c1df199 100644 --- a/flink-runtime-web/web-dashboard/src/app/services/task-manager.service.ts +++ b/flink-runtime-web/web-dashboard/src/app/services/task-manager.service.ts @@ -21,7 +21,7 @@ import { Injectable } from '@angular/core'; import { EMPTY, of, ReplaySubject } from 'rxjs'; import { catchError, map } from 'rxjs/operators'; import { BASE_URL } from 'config'; -import { TaskManagerListInterface, TaskManagerDetailInterface } from 'interfaces'; +import { TaskManagerListInterface, TaskManagerDetailInterface, TaskManagerLogInterface } from 'interfaces'; @Injectable({ providedIn: 'root' @@ -50,25 +50,38 @@ export class TaskManagerService { } /** - * Load TM logs + * Load TM log list * @param taskManagerId */ - loadLogs(taskManagerId: string) { - return this.httpClient.get(`${BASE_URL}/taskmanagers/${taskManagerId}/log`, { - responseType: 'text', - headers: new HttpHeaders().append('Cache-Control', 'no-cache') - }); + loadLogList(taskManagerId: string) { + return this.httpClient + .get(`${BASE_URL}/taskmanagers/${taskManagerId}/logs`) + .pipe(map(data => data.logs)); } /** - * Load TM stdout + * Load TM log * @param taskManagerId + * @param logName + * @param hasLogName */ - loadStdout(taskManagerId: string) { - return this.httpClient.get(`${BASE_URL}/taskmanagers/${taskManagerId}/stdout`, { - responseType: 'text', - headers: new HttpHeaders().append('Cache-Control', 'no-cache') - }); + loadLog(taskManagerId: string, logName: string, hasLogName: boolean) { + let url = ''; + if (hasLogName) { + url = `${BASE_URL}/taskmanagers/${taskManagerId}/logs/${logName}`; + } else { + url = `${BASE_URL}/taskmanagers/${taskManagerId}/log`; + } + return this.httpClient + .get(url, { responseType: 'text', headers: new HttpHeaders().append('Cache-Control', 'no-cache') }) + .pipe( + map(data => { + return { + data, + url + }; + }) + ); } constructor(private httpClient: HttpClient) {} diff --git a/flink-runtime-web/web-dashboard/src/app/share/common/navigation/navigation.component.ts b/flink-runtime-web/web-dashboard/src/app/share/common/navigation/navigation.component.ts index a5b1d6068e0eae61ea63553f18fa812d546bb1dd..b48e4e01d41962bfe06744b404d2920fff193c67 100644 --- a/flink-runtime-web/web-dashboard/src/app/share/common/navigation/navigation.component.ts +++ b/flink-runtime-web/web-dashboard/src/app/share/common/navigation/navigation.component.ts @@ -17,9 +17,9 @@ */ import { ChangeDetectionStrategy, ChangeDetectorRef, Component, Input, OnDestroy, OnInit } from '@angular/core'; -import { ActivatedRoute, Router } from '@angular/router'; +import { ActivatedRoute, NavigationEnd, Router } from '@angular/router'; import { Subject } from 'rxjs'; -import { map, takeUntil } from 'rxjs/operators'; +import { filter, flatMap, map, startWith, takeUntil } from 'rxjs/operators'; @Component({ selector: 'flink-navigation', @@ -41,17 +41,19 @@ export class NavigationComponent implements OnInit, OnDestroy { constructor(private activatedRoute: ActivatedRoute, private router: Router, private cdr: ChangeDetectorRef) {} ngOnInit() { - if (this.activatedRoute && this.activatedRoute.firstChild) { - this.activatedRoute.firstChild.data - .pipe( - takeUntil(this.destroy$), - map(data => data.path) - ) - .subscribe(data => { - this.navIndex = this.listOfNavigation.map(nav => nav.path).indexOf(data); - this.cdr.markForCheck(); - }); - } + this.router.events + .pipe( + filter(e => e instanceof NavigationEnd), + startWith(true), + filter(() => !!(this.activatedRoute && this.activatedRoute.firstChild)), + flatMap(() => this.activatedRoute!.firstChild!.data), + takeUntil(this.destroy$), + map(data => data.path) + ) + .subscribe(data => { + this.navIndex = this.listOfNavigation.map(nav => nav.path).indexOf(data); + this.cdr.markForCheck(); + }); } ngOnDestroy() { diff --git a/flink-runtime-web/web-dashboard/src/app/share/customize/refresh-download/refresh-download.component.html b/flink-runtime-web/web-dashboard/src/app/share/customize/refresh-download/refresh-download.component.html index 41e94fdf9285cf1238f2e41f4d541385ccfb2fef..8994733cb4a660ce5719429efa45e7b0f5ac13f2 100644 --- a/flink-runtime-web/web-dashboard/src/app/share/customize/refresh-download/refresh-download.component.html +++ b/flink-runtime-web/web-dashboard/src/app/share/customize/refresh-download/refresh-download.component.html @@ -16,7 +16,26 @@ ~ limitations under the License. --> - + + + + + Loading... + + Reload + + + Download + + + + Exit FullScreen + + + FullScreen + + + diff --git a/flink-runtime-web/web-dashboard/src/app/share/customize/refresh-download/refresh-download.component.less b/flink-runtime-web/web-dashboard/src/app/share/customize/refresh-download/refresh-download.component.less index c80e517eb469093493b2fc952735ccb8b6ebf576..b9f7bf224b6feb3cb4a187686a70f6589bdb7e73 100644 --- a/flink-runtime-web/web-dashboard/src/app/share/customize/refresh-download/refresh-download.component.less +++ b/flink-runtime-web/web-dashboard/src/app/share/customize/refresh-download/refresh-download.component.less @@ -21,3 +21,7 @@ top: 8px; right: 32px; } + +.operate-icon { + margin-right: 6px; +} diff --git a/flink-runtime-web/web-dashboard/src/app/share/customize/refresh-download/refresh-download.component.ts b/flink-runtime-web/web-dashboard/src/app/share/customize/refresh-download/refresh-download.component.ts index a1346751068049828299d2e0d696d942ed63fa6a..506ae7e22386647d6a7ea81ab7ba4b56cc8cf925 100644 --- a/flink-runtime-web/web-dashboard/src/app/share/customize/refresh-download/refresh-download.component.ts +++ b/flink-runtime-web/web-dashboard/src/app/share/customize/refresh-download/refresh-download.component.ts @@ -27,5 +27,14 @@ import { Component, ChangeDetectionStrategy, Input, Output, EventEmitter } from export class RefreshDownloadComponent { @Input() downloadName: string; @Input() downloadHref: string; + @Input() isLoading = false; + @Input() compactMode = false; @Output() reload = new EventEmitter(); + @Output() fullScreen = new EventEmitter(); + isFullScreen = false; + + toggleFullScreen() { + this.isFullScreen = !this.isFullScreen; + this.fullScreen.emit(this.isFullScreen); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/akka/StoppingSupervisorWithoutLoggingActorKilledExceptionStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/akka/EscalatingSupervisorStrategy.java similarity index 61% rename from flink-runtime/src/main/java/org/apache/flink/runtime/akka/StoppingSupervisorWithoutLoggingActorKilledExceptionStrategy.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/akka/EscalatingSupervisorStrategy.java index e2d8fd6089fb8cfc5fbec233e906084b18f92df4..dc0832ffc64ec33040d5c5ae1ff92ec2d19b2aea 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/akka/StoppingSupervisorWithoutLoggingActorKilledExceptionStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/akka/EscalatingSupervisorStrategy.java @@ -18,36 +18,23 @@ package org.apache.flink.runtime.akka; -import akka.actor.ActorKilledException; import akka.actor.OneForOneStrategy; import akka.actor.SupervisorStrategy; import akka.actor.SupervisorStrategyConfigurator; import akka.japi.pf.PFBuilder; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** - * Stopping supervisor strategy which logs {@link ActorKilledException} only on debug log level. + * Escalating supervisor strategy. */ -public class StoppingSupervisorWithoutLoggingActorKilledExceptionStrategy implements SupervisorStrategyConfigurator { - - private static final Logger LOG = LoggerFactory.getLogger(StoppingSupervisorWithoutLoggingActorKilledExceptionStrategy.class); +public class EscalatingSupervisorStrategy implements SupervisorStrategyConfigurator { @Override public SupervisorStrategy create() { return new OneForOneStrategy( false, new PFBuilder() - .match( - Exception.class, - (Exception e) -> { - if (e instanceof ActorKilledException) { - LOG.debug("Actor was killed. Stopping it now.", e); - } else { - LOG.error("Actor failed with exception. Stopping it now.", e); - } - return SupervisorStrategy.Stop$.MODULE$; - }) + .matchAny( + (ignored) -> SupervisorStrategy.escalate()) .build()); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/AbstractCheckpointStats.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/AbstractCheckpointStats.java index f49790d361959fd7b97a0086aa0773c6929a2bbd..09a14554a3e588209e40f2837f67695eb442b59f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/AbstractCheckpointStats.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/AbstractCheckpointStats.java @@ -88,15 +88,6 @@ public abstract class AbstractCheckpointStats implements Serializable { */ public abstract long getStateSize(); - /** - * Returns the total buffered bytes during alignment over all subtasks. - * - *

Can return -1 if the runtime did not report this. - * - * @return Total buffered bytes during alignment over all subtasks. - */ - public abstract long getAlignmentBuffered(); - /** * Returns the latest acknowledged subtask stats or null if * none was acknowledged yet. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java index 070e00c0018ede367e0ec48c2696bdb45636a55b..d5d93deae5d71f25660ddcec43691028be453f9b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java @@ -55,6 +55,8 @@ import javax.annotation.Nullable; import java.io.IOException; import java.util.ArrayDeque; +import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.LinkedHashMap; @@ -110,6 +112,9 @@ public class CheckpointCoordinator { /** Tasks who need to be sent a message when a checkpoint is confirmed. */ private final ExecutionVertex[] tasksToCommitTo; + /** The operator coordinators that need to be checkpointed. */ + private final Collection coordinatorsToCheckpoint; + /** Map from checkpoint ID to the pending checkpoint. */ private final Map pendingCheckpoints; @@ -189,6 +194,10 @@ public class CheckpointCoordinator { private final Clock clock; + private final boolean isExactlyOnceMode; + + private final boolean isUnalignedCheckpoint; + /** Flag represents there is an in-flight trigger request. */ private boolean isTriggering = false; @@ -203,6 +212,7 @@ public class CheckpointCoordinator { ExecutionVertex[] tasksToTrigger, ExecutionVertex[] tasksToWaitFor, ExecutionVertex[] tasksToCommitTo, + Collection coordinatorsToCheckpoint, CheckpointIDCounter checkpointIDCounter, CompletedCheckpointStore completedCheckpointStore, StateBackend checkpointStateBackend, @@ -217,6 +227,7 @@ public class CheckpointCoordinator { tasksToTrigger, tasksToWaitFor, tasksToCommitTo, + coordinatorsToCheckpoint, checkpointIDCounter, completedCheckpointStore, checkpointStateBackend, @@ -234,6 +245,7 @@ public class CheckpointCoordinator { ExecutionVertex[] tasksToTrigger, ExecutionVertex[] tasksToWaitFor, ExecutionVertex[] tasksToCommitTo, + Collection coordinatorsToCheckpoint, CheckpointIDCounter checkpointIDCounter, CompletedCheckpointStore completedCheckpointStore, StateBackend checkpointStateBackend, @@ -267,6 +279,7 @@ public class CheckpointCoordinator { this.tasksToTrigger = checkNotNull(tasksToTrigger); this.tasksToWaitFor = checkNotNull(tasksToWaitFor); this.tasksToCommitTo = checkNotNull(tasksToCommitTo); + this.coordinatorsToCheckpoint = Collections.unmodifiableCollection(coordinatorsToCheckpoint); this.pendingCheckpoints = new LinkedHashMap<>(); this.checkpointIdCounter = checkNotNull(checkpointIDCounter); this.completedCheckpointStore = checkNotNull(completedCheckpointStore); @@ -276,6 +289,8 @@ public class CheckpointCoordinator { this.isPreferCheckpointForRecovery = chkConfig.isPreferCheckpointForRecovery(); this.failureManager = checkNotNull(failureManager); this.clock = checkNotNull(clock); + this.isExactlyOnceMode = chkConfig.isExactlyOnce(); + this.isUnalignedCheckpoint = chkConfig.isUnalignedCheckpoint(); this.recentPendingCheckpoints = new ArrayDeque<>(NUM_GHOST_CHECKPOINT_IDS); this.masterHooks = new HashMap<>(); @@ -548,8 +563,16 @@ public class CheckpointCoordinator { onCompletionPromise), timer); - pendingCheckpointCompletableFuture - .thenCompose(this::snapshotMasterState) + final CompletableFuture masterStatesComplete = pendingCheckpointCompletableFuture + .thenCompose(this::snapshotMasterState); + + final CompletableFuture coordinatorCheckpointsComplete = pendingCheckpointCompletableFuture + .thenComposeAsync((pendingCheckpoint) -> + OperatorCoordinatorCheckpoints.triggerAndAcknowledgeAllCoordinatorCheckpointsWithCompletion( + coordinatorsToCheckpoint, pendingCheckpoint, timer), + timer); + + CompletableFuture.allOf(masterStatesComplete, coordinatorCheckpointsComplete) .whenCompleteAsync( (ignored, throwable) -> { final PendingCheckpoint checkpoint = @@ -634,6 +657,7 @@ public class CheckpointCoordinator { checkpointID, timestamp, ackTasks, + OperatorCoordinatorCheckpointContext.getIds(coordinatorsToCheckpoint), masterHooks.keySet(), props, checkpointStorageLocation, @@ -736,7 +760,9 @@ public class CheckpointCoordinator { final CheckpointOptions checkpointOptions = new CheckpointOptions( props.getCheckpointType(), - checkpointStorageLocation.getLocationReference()); + checkpointStorageLocation.getLocationReference(), + isExactlyOnceMode, + isUnalignedCheckpoint); // send the messages to the tasks that trigger their checkpoint for (Execution execution: executions) { @@ -1080,15 +1106,23 @@ public class CheckpointCoordinator { LOG.debug(builder.toString()); } - // send the "notify complete" call to all vertices - final long timestamp = completedCheckpoint.getTimestamp(); + // send the "notify complete" call to all vertices, coordinators, etc. + sendAcknowledgeMessages(checkpointId, completedCheckpoint.getTimestamp()); + } + private void sendAcknowledgeMessages(long checkpointId, long timestamp) { + // commit tasks for (ExecutionVertex ev : tasksToCommitTo) { Execution ee = ev.getCurrentExecutionAttempt(); if (ee != null) { ee.notifyCheckpointComplete(checkpointId, timestamp); } } + + // commit coordinators + for (OperatorCoordinatorCheckpointContext coordinatorContext : coordinatorsToCheckpoint) { + coordinatorContext.coordinator().checkpointComplete(checkpointId); + } } /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointMetrics.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointMetrics.java index d18feb8f9e19b179a8594fa92b06eb4be65705e9..9420ef9e5486a1ecea6d074e1818ddb003d6faee 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointMetrics.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointMetrics.java @@ -30,9 +30,6 @@ public class CheckpointMetrics implements Serializable { private static final long serialVersionUID = 1L; - /** The number of bytes that were buffered during the checkpoint alignment phase. */ - private long bytesBufferedInAlignment; - /** The duration (in nanoseconds) that the stream alignment for the checkpoint took. */ private long alignmentDurationNanos; @@ -59,21 +56,11 @@ public class CheckpointMetrics implements Serializable { checkArgument(bytesBufferedInAlignment >= -1); checkArgument(alignmentDurationNanos >= -1); - this.bytesBufferedInAlignment = bytesBufferedInAlignment; this.alignmentDurationNanos = alignmentDurationNanos; this.syncDurationMillis = syncDurationMillis; this.asyncDurationMillis = asyncDurationMillis; } - public long getBytesBufferedInAlignment() { - return bytesBufferedInAlignment; - } - - public CheckpointMetrics setBytesBufferedInAlignment(long bytesBufferedInAlignment) { - this.bytesBufferedInAlignment = bytesBufferedInAlignment; - return this; - } - public long getAlignmentDurationNanos() { return alignmentDurationNanos; } @@ -121,8 +108,7 @@ public class CheckpointMetrics implements Serializable { CheckpointMetrics that = (CheckpointMetrics) o; - return bytesBufferedInAlignment == that.bytesBufferedInAlignment && - alignmentDurationNanos == that.alignmentDurationNanos && + return alignmentDurationNanos == that.alignmentDurationNanos && syncDurationMillis == that.syncDurationMillis && asyncDurationMillis == that.asyncDurationMillis && checkpointStartDelayNanos == that.checkpointStartDelayNanos; @@ -132,7 +118,6 @@ public class CheckpointMetrics implements Serializable { @Override public int hashCode() { return Objects.hash( - bytesBufferedInAlignment, alignmentDurationNanos, syncDurationMillis, asyncDurationMillis, @@ -142,7 +127,6 @@ public class CheckpointMetrics implements Serializable { @Override public String toString() { return "CheckpointMetrics{" + - "bytesBufferedInAlignment=" + bytesBufferedInAlignment + ", alignmentDurationNanos=" + alignmentDurationNanos + ", syncDurationMillis=" + syncDurationMillis + ", asyncDurationMillis=" + asyncDurationMillis + diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointOptions.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointOptions.java index 09d351686f18b37fc57d3930461094441cbe96fe..1f2cd9034fa74381c32288fd280c4db5511838a2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointOptions.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointOptions.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.checkpoint; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.state.CheckpointStorageLocationReference; @@ -42,12 +43,27 @@ public class CheckpointOptions implements Serializable { /** Target location for the checkpoint. */ private final CheckpointStorageLocationReference targetLocation; + private final boolean isExactlyOnceMode; + + private final boolean isUnalignedCheckpoint; + + @VisibleForTesting public CheckpointOptions( CheckpointType checkpointType, CheckpointStorageLocationReference targetLocation) { + this(checkpointType, targetLocation, true, false); + } + + public CheckpointOptions( + CheckpointType checkpointType, + CheckpointStorageLocationReference targetLocation, + boolean isExactlyOnceMode, + boolean isUnalignedCheckpoint) { this.checkpointType = checkNotNull(checkpointType); this.targetLocation = checkNotNull(targetLocation); + this.isExactlyOnceMode = isExactlyOnceMode; + this.isUnalignedCheckpoint = isUnalignedCheckpoint; } // ------------------------------------------------------------------------ @@ -66,11 +82,24 @@ public class CheckpointOptions implements Serializable { return targetLocation; } + public boolean isExactlyOnceMode() { + return isExactlyOnceMode; + } + + public boolean isUnalignedCheckpoint() { + return isUnalignedCheckpoint; + } + // ------------------------------------------------------------------------ @Override public int hashCode() { - return 31 * targetLocation.hashCode() + checkpointType.hashCode(); + int result = 1; + result = 31 * result + targetLocation.hashCode(); + result = 31 * result + checkpointType.hashCode(); + result = 31 * result + (isExactlyOnceMode ? 1 : 0); + result = 31 * result + (isUnalignedCheckpoint ? 1 : 0); + return result; } @Override @@ -81,7 +110,9 @@ public class CheckpointOptions implements Serializable { else if (obj != null && obj.getClass() == CheckpointOptions.class) { final CheckpointOptions that = (CheckpointOptions) obj; return this.checkpointType == that.checkpointType && - this.targetLocation.equals(that.targetLocation); + this.targetLocation.equals(that.targetLocation) && + this.isExactlyOnceMode == that.isExactlyOnceMode && + this.isUnalignedCheckpoint == that.isUnalignedCheckpoint; } else { return false; @@ -90,7 +121,12 @@ public class CheckpointOptions implements Serializable { @Override public String toString() { - return "CheckpointOptions: " + checkpointType + " @ " + targetLocation; + return "CheckpointOptions {" + + "checkpointType = " + checkpointType + + ", targetLocation = " + targetLocation + + ", isExactlyOnceMode = " + isExactlyOnceMode + + ", isUnalignedCheckpoint = " + isUnalignedCheckpoint + + "}"; } // ------------------------------------------------------------------------ @@ -100,7 +136,18 @@ public class CheckpointOptions implements Serializable { private static final CheckpointOptions CHECKPOINT_AT_DEFAULT_LOCATION = new CheckpointOptions(CheckpointType.CHECKPOINT, CheckpointStorageLocationReference.getDefault()); + @VisibleForTesting public static CheckpointOptions forCheckpointWithDefaultLocation() { return CHECKPOINT_AT_DEFAULT_LOCATION; } + + public static CheckpointOptions forCheckpointWithDefaultLocation( + boolean isExactlyOnceMode, + boolean isUnalignedCheckpoint) { + return new CheckpointOptions( + CheckpointType.CHECKPOINT, + CheckpointStorageLocationReference.getDefault(), + isExactlyOnceMode, + isUnalignedCheckpoint); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointStatsTracker.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointStatsTracker.java index 9be1f69581403589ee5106009721f8e4e60c2840..e6e78a4a6f041b08c061df118141686935cd927a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointStatsTracker.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointStatsTracker.java @@ -341,9 +341,6 @@ public class CheckpointStatsTracker { @VisibleForTesting static final String LATEST_COMPLETED_CHECKPOINT_DURATION_METRIC = "lastCheckpointDuration"; - @VisibleForTesting - static final String LATEST_COMPLETED_CHECKPOINT_ALIGNMENT_BUFFERED_METRIC = "lastCheckpointAlignmentBuffered"; - @VisibleForTesting static final String LATEST_COMPLETED_CHECKPOINT_EXTERNAL_PATH_METRIC = "lastCheckpointExternalPath"; @@ -360,7 +357,6 @@ public class CheckpointStatsTracker { metricGroup.gauge(LATEST_RESTORED_CHECKPOINT_TIMESTAMP_METRIC, new LatestRestoredCheckpointTimestampGauge()); metricGroup.gauge(LATEST_COMPLETED_CHECKPOINT_SIZE_METRIC, new LatestCompletedCheckpointSizeGauge()); metricGroup.gauge(LATEST_COMPLETED_CHECKPOINT_DURATION_METRIC, new LatestCompletedCheckpointDurationGauge()); - metricGroup.gauge(LATEST_COMPLETED_CHECKPOINT_ALIGNMENT_BUFFERED_METRIC, new LatestCompletedCheckpointAlignmentBufferedGauge()); metricGroup.gauge(LATEST_COMPLETED_CHECKPOINT_EXTERNAL_PATH_METRIC, new LatestCompletedCheckpointExternalPathGauge()); } @@ -428,18 +424,6 @@ public class CheckpointStatsTracker { } } - private class LatestCompletedCheckpointAlignmentBufferedGauge implements Gauge { - @Override - public Long getValue() { - CompletedCheckpointStats completed = latestCompletedCheckpoint; - if (completed != null) { - return completed.getAlignmentBuffered(); - } else { - return -1L; - } - } - } - private class LatestCompletedCheckpointExternalPathGauge implements Gauge { @Override public String getValue() { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/Checkpoints.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/Checkpoints.java index 225b7b9f6dadcd3cf073e23dcfea12453b95b079..17db5e6ce6a4047ae1a4ed2812695d080d14d28f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/Checkpoints.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/Checkpoints.java @@ -23,7 +23,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.checkpoint.metadata.CheckpointMetadata; import org.apache.flink.runtime.checkpoint.metadata.MetadataSerializer; import org.apache.flink.runtime.checkpoint.metadata.MetadataSerializers; -import org.apache.flink.runtime.checkpoint.metadata.MetadataV2Serializer; +import org.apache.flink.runtime.checkpoint.metadata.MetadataV3Serializer; import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.OperatorID; @@ -85,8 +85,8 @@ public class Checkpoints { // write generic header out.writeInt(HEADER_MAGIC_NUMBER); - out.writeInt(MetadataV2Serializer.VERSION); - MetadataV2Serializer.serialize(checkpointMetadata, out); + out.writeInt(MetadataV3Serializer.VERSION); + MetadataV3Serializer.serialize(checkpointMetadata, out); } // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStats.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStats.java index 5ca355b6bb6a232bcdc8ce2728712118999f3f43..0aabfb3d743b8dad6da9991df4d903db8ac180c8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStats.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStats.java @@ -41,9 +41,6 @@ public class CompletedCheckpointStats extends AbstractCheckpointStats { /** Total checkpoint state size over all subtasks. */ private final long stateSize; - /** Buffered bytes during alignment over all subtasks. */ - private final long alignmentBuffered; - /** The latest acknowledged subtask stats. */ private final SubtaskStateStats latestAcknowledgedSubtask; @@ -63,7 +60,6 @@ public class CompletedCheckpointStats extends AbstractCheckpointStats { * @param taskStats Task stats for each involved operator. * @param numAcknowledgedSubtasks Number of acknowledged subtasks. * @param stateSize Total checkpoint state size over all subtasks. - * @param alignmentBuffered Buffered bytes during alignment over all subtasks. * @param latestAcknowledgedSubtask The latest acknowledged subtask stats. * @param externalPointer Optional external path if persisted externally. */ @@ -75,7 +71,6 @@ public class CompletedCheckpointStats extends AbstractCheckpointStats { Map taskStats, int numAcknowledgedSubtasks, long stateSize, - long alignmentBuffered, SubtaskStateStats latestAcknowledgedSubtask, String externalPointer) { @@ -83,7 +78,6 @@ public class CompletedCheckpointStats extends AbstractCheckpointStats { checkArgument(numAcknowledgedSubtasks == totalSubtaskCount, "Did not acknowledge all subtasks."); checkArgument(stateSize >= 0, "Negative state size"); this.stateSize = stateSize; - this.alignmentBuffered = alignmentBuffered; this.latestAcknowledgedSubtask = checkNotNull(latestAcknowledgedSubtask); this.externalPointer = externalPointer; } @@ -103,11 +97,6 @@ public class CompletedCheckpointStats extends AbstractCheckpointStats { return stateSize; } - @Override - public long getAlignmentBuffered() { - return alignmentBuffered; - } - @Override @Nullable public SubtaskStateStats getLatestAcknowledgedSubtaskStats() { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStatsSummary.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStatsSummary.java index 7e3f4b4e7832521fde50893cfa6c72a000c1e5b0..65694fbaa9e68b89f9090537c83d6b162eb98b05 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStatsSummary.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStatsSummary.java @@ -35,21 +35,16 @@ public class CompletedCheckpointStatsSummary implements Serializable { /** Duration statistics for all completed checkpoints. */ private final MinMaxAvgStats duration; - /** Byte buffered during alignment for all completed checkpoints. */ - private final MinMaxAvgStats alignmentBuffered; - CompletedCheckpointStatsSummary() { - this(new MinMaxAvgStats(), new MinMaxAvgStats(), new MinMaxAvgStats()); + this(new MinMaxAvgStats(), new MinMaxAvgStats()); } private CompletedCheckpointStatsSummary( MinMaxAvgStats stateSize, - MinMaxAvgStats duration, - MinMaxAvgStats alignmentBuffered) { + MinMaxAvgStats duration) { this.stateSize = checkNotNull(stateSize); this.duration = checkNotNull(duration); - this.alignmentBuffered = checkNotNull(alignmentBuffered); } /** @@ -60,7 +55,6 @@ public class CompletedCheckpointStatsSummary implements Serializable { void updateSummary(CompletedCheckpointStats completed) { stateSize.add(completed.getStateSize()); duration.add(completed.getEndToEndDuration()); - alignmentBuffered.add(completed.getAlignmentBuffered()); } /** @@ -71,8 +65,7 @@ public class CompletedCheckpointStatsSummary implements Serializable { CompletedCheckpointStatsSummary createSnapshot() { return new CompletedCheckpointStatsSummary( stateSize.createSnapshot(), - duration.createSnapshot(), - alignmentBuffered.createSnapshot()); + duration.createSnapshot()); } /** @@ -92,16 +85,4 @@ public class CompletedCheckpointStatsSummary implements Serializable { public MinMaxAvgStats getEndToEndDurationStats() { return duration; } - - /** - * Returns the summary stats for the bytes buffered during alignment. - * - *

If no alignments are reported or happen (at least once mode), the - * returned stats are in their initial state. - * - * @return Summary stats for the bytes buffered during alignment. - */ - public MinMaxAvgStats getAlignmentBufferedStats() { - return alignmentBuffered; - } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/FailedCheckpointStats.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/FailedCheckpointStats.java index 1eb0131971c6c93005c9849e72dc8e740c9a7181..edd83b63f5d804edaa9776d05fd63171e02f0ed4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/FailedCheckpointStats.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/FailedCheckpointStats.java @@ -41,9 +41,6 @@ public class FailedCheckpointStats extends AbstractCheckpointStats { /** Total checkpoint state size over all subtasks. */ private final long stateSize; - /** Buffered bytes during alignment over all subtasks. */ - private final long alignmentBuffered; - /** Timestamp when the checkpoint was failed at the coordinator. */ private final long failureTimestamp; @@ -68,7 +65,6 @@ public class FailedCheckpointStats extends AbstractCheckpointStats { * @param taskStats Task stats for each involved operator. * @param numAcknowledgedSubtasks Number of acknowledged subtasks. * @param stateSize Total checkpoint state size over all subtasks. - * @param alignmentBuffered Buffered bytes during alignment over all subtasks. * @param failureTimestamp Timestamp when this checkpoint failed. * @param latestAcknowledgedSubtask The latest acknowledged subtask stats or null. * @param cause Cause of the checkpoint failure or null. @@ -81,7 +77,6 @@ public class FailedCheckpointStats extends AbstractCheckpointStats { Map taskStats, int numAcknowledgedSubtasks, long stateSize, - long alignmentBuffered, long failureTimestamp, @Nullable SubtaskStateStats latestAcknowledgedSubtask, @Nullable Throwable cause) { @@ -91,7 +86,6 @@ public class FailedCheckpointStats extends AbstractCheckpointStats { this.numAcknowledgedSubtasks = numAcknowledgedSubtasks; checkArgument(stateSize >= 0, "Negative state size"); this.stateSize = stateSize; - this.alignmentBuffered = alignmentBuffered; this.failureTimestamp = failureTimestamp; this.latestAcknowledgedSubtask = latestAcknowledgedSubtask; this.failureMsg = cause != null ? cause.getMessage() : null; @@ -112,11 +106,6 @@ public class FailedCheckpointStats extends AbstractCheckpointStats { return stateSize; } - @Override - public long getAlignmentBuffered() { - return alignmentBuffered; - } - @Override @Nullable public SubtaskStateStats getLatestAcknowledgedSubtaskStats() { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverRegion.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorCoordinatorCheckpointContext.java similarity index 39% rename from flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverRegion.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorCoordinatorCheckpointContext.java index 2e416a4fe8667da95593c620338103f2541d3888..16acbb2b75b95da45720cf425c77059dd95c10a5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverRegion.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorCoordinatorCheckpointContext.java @@ -16,52 +16,61 @@ * limitations under the License. */ -package org.apache.flink.runtime.executiongraph.failover.flip1; +package org.apache.flink.runtime.checkpoint; -import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; -import java.util.HashSet; -import java.util.Set; +import java.util.Collection; +import java.util.stream.Collectors; import static org.apache.flink.util.Preconditions.checkNotNull; /** - * FailoverRegion is a subset of all the vertices in the job topology. + * An {@link OperatorCoordinator} and its contextual information needed to trigger and + * acknowledge a checkpoint. */ -public class FailoverRegion { - - /** All vertex IDs in this region. */ - private final Set executionVertexIDs; - - /** All vertices in this region. */ - private final Set> executionVertices; - - /** - * Creates a new failover region containing a set of vertices. - * - * @param executionVertices to be contained in this region - */ - public FailoverRegion(Set> executionVertices) { - this.executionVertices = checkNotNull(executionVertices); - this.executionVertexIDs = new HashSet<>(); - executionVertices.forEach(v -> this.executionVertexIDs.add(v.getId())); +public final class OperatorCoordinatorCheckpointContext { + + private final OperatorCoordinator coordinator; + + private final OperatorID operatorId; + + private final int maxParallelism; + + private final int currentParallelism; + + public OperatorCoordinatorCheckpointContext( + OperatorCoordinator coordinator, + OperatorID operatorId, + int maxParallelism, + int currentParallelism) { + + this.coordinator = checkNotNull(coordinator); + this.operatorId = checkNotNull(operatorId); + this.maxParallelism = maxParallelism; + this.currentParallelism = currentParallelism; + } + + public OperatorCoordinator coordinator() { + return coordinator; + } + + public OperatorID operatorId() { + return operatorId; + } + + public int maxParallelism() { + return maxParallelism; } - /** - * Returns IDs of all vertices in this region. - * - * @return IDs of all vertices in this region - */ - public Set getAllExecutionVertexIDs() { - return executionVertexIDs; + public int currentParallelism() { + return currentParallelism; } - /** - * Returns all vertices in this region. - * - * @return all vertices in this region - */ - public Set> getAllExecutionVertices() { - return executionVertices; + public static Collection getIds(Collection infos) { + return infos.stream() + .map(OperatorCoordinatorCheckpointContext::operatorId) + .collect(Collectors.toList()); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorCoordinatorCheckpoints.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorCoordinatorCheckpoints.java new file mode 100644 index 0000000000000000000000000000000000000000..cfacec867a3ee2972b2ab007f605bda578f8f300 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorCoordinatorCheckpoints.java @@ -0,0 +1,144 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.checkpoint; + +import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; +import org.apache.flink.runtime.state.StreamStateHandle; +import org.apache.flink.runtime.state.memory.ByteStreamStateHandle; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.Executor; + +/** + * All the logic related to taking checkpoints of the {@link OperatorCoordinator}s. + * + *

NOTE: This class has a simplified error handling logic. If one of the several coordinator checkpoints + * fail, no cleanup is triggered for the other concurrent ones. That is okay, since they all produce just byte[] + * as the result. We have to change that once we allow then to create external resources that actually need + * to be cleaned up. + */ +final class OperatorCoordinatorCheckpoints { + + public static CompletableFuture triggerCoordinatorCheckpoint( + final OperatorCoordinatorCheckpointContext coordinatorInfo, + final long checkpointId) throws Exception { + + final CompletableFuture checkpointFuture = + coordinatorInfo.coordinator().checkpointCoordinator(checkpointId); + + return checkpointFuture.thenApply( + (state) -> new CoordinatorSnapshot( + coordinatorInfo, new ByteStreamStateHandle(coordinatorInfo.operatorId().toString(), state)) + ); + } + + public static CompletableFuture triggerAllCoordinatorCheckpoints( + final Collection coordinators, + final long checkpointId) throws Exception { + + final Collection> individualSnapshots = new ArrayList<>(coordinators.size()); + + for (final OperatorCoordinatorCheckpointContext coordinator : coordinators) { + individualSnapshots.add(triggerCoordinatorCheckpoint(coordinator, checkpointId)); + } + + return FutureUtils.combineAll(individualSnapshots).thenApply(AllCoordinatorSnapshots::new); + } + + public static CompletableFuture triggerAndAcknowledgeAllCoordinatorCheckpoints( + final Collection coordinators, + final PendingCheckpoint checkpoint, + final Executor acknowledgeExecutor) throws Exception { + + final CompletableFuture snapshots = + triggerAllCoordinatorCheckpoints(coordinators, checkpoint.getCheckpointId()); + + return snapshots + .thenAcceptAsync( + (allSnapshots) -> { + try { + acknowledgeAllCoordinators(checkpoint, allSnapshots.snapshots); + } + catch (Exception e) { + throw new CompletionException(e); + } + }, + acknowledgeExecutor); + } + + public static CompletableFuture triggerAndAcknowledgeAllCoordinatorCheckpointsWithCompletion( + final Collection coordinators, + final PendingCheckpoint checkpoint, + final Executor acknowledgeExecutor) throws CompletionException { + + try { + return triggerAndAcknowledgeAllCoordinatorCheckpoints(coordinators, checkpoint, acknowledgeExecutor); + } catch (Exception e) { + throw new CompletionException(e); + } + } + + // ------------------------------------------------------------------------ + + private static void acknowledgeAllCoordinators(PendingCheckpoint checkpoint, Collection snapshots) throws CheckpointException { + for (final CoordinatorSnapshot snapshot : snapshots) { + final PendingCheckpoint.TaskAcknowledgeResult result = + checkpoint.acknowledgeCoordinatorState(snapshot.coordinator, snapshot.state); + + if (result != PendingCheckpoint.TaskAcknowledgeResult.SUCCESS) { + throw new CheckpointException("Coordinator state not acknowledged successfully: " + result, + CheckpointFailureReason.TRIGGER_CHECKPOINT_FAILURE); + } + } + } + + // ------------------------------------------------------------------------ + + static final class AllCoordinatorSnapshots { + + private final Collection snapshots; + + AllCoordinatorSnapshots(Collection snapshots) { + this.snapshots = snapshots; + } + + public Iterable snapshots() { + return snapshots; + } + } + + static final class CoordinatorSnapshot { + + final OperatorCoordinatorCheckpointContext coordinator; + final StreamStateHandle state; + + CoordinatorSnapshot(OperatorCoordinatorCheckpointContext coordinator, StreamStateHandle state) { + // if this is not true any more, we need more elaborate dispose/cleanup handling + // see comment above the class. + assert state instanceof ByteStreamStateHandle; + + this.coordinator = coordinator; + this.state = state; + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorState.java index de011c111718db8254a4ff0a9e7b5155d54d39f3..998a3bdbd04adf49fd61c2f8c613eac77d5afa0f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorState.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorState.java @@ -21,14 +21,19 @@ package org.apache.flink.runtime.checkpoint; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.state.CompositeStateHandle; import org.apache.flink.runtime.state.SharedStateRegistry; +import org.apache.flink.runtime.state.StreamStateHandle; import org.apache.flink.util.Preconditions; +import javax.annotation.Nullable; + import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.Objects; +import static org.apache.flink.util.Preconditions.checkState; + /** * Simple container class which contains the raw/managed operator state and key-group state handles from all sub * tasks of an operator and therefore represents the complete state of a logical operator. @@ -43,6 +48,10 @@ public class OperatorState implements CompositeStateHandle { /** The handles to states created by the parallel tasks: subtaskIndex -> subtaskstate. */ private final Map operatorSubtaskStates; + /** The state of the operator coordinator. Null, if no such state exists. */ + @Nullable + private StreamStateHandle coordinatorState; + /** The parallelism of the operator when it was checkpointed. */ private final int parallelism; @@ -87,6 +96,16 @@ public class OperatorState implements CompositeStateHandle { } } + public void setCoordinatorState(@Nullable StreamStateHandle coordinatorState) { + checkState(this.coordinatorState == null, "coordinator state already set"); + this.coordinatorState = coordinatorState; + } + + @Nullable + public StreamStateHandle getCoordinatorState() { + return coordinatorState; + } + public Map getSubtaskStates() { return Collections.unmodifiableMap(operatorSubtaskStates); } @@ -112,6 +131,10 @@ public class OperatorState implements CompositeStateHandle { for (OperatorSubtaskState operatorSubtaskState : operatorSubtaskStates.values()) { operatorSubtaskState.discardState(); } + + if (coordinatorState != null) { + coordinatorState.discardState(); + } } @Override @@ -123,7 +146,7 @@ public class OperatorState implements CompositeStateHandle { @Override public long getStateSize() { - long result = 0L; + long result = coordinatorState == null ? 0L : coordinatorState.getStateSize(); for (int i = 0; i < parallelism; i++) { OperatorSubtaskState operatorSubtaskState = operatorSubtaskStates.get(i); @@ -142,6 +165,7 @@ public class OperatorState implements CompositeStateHandle { return operatorID.equals(other.operatorID) && parallelism == other.parallelism + && Objects.equals(coordinatorState, other.coordinatorState) && operatorSubtaskStates.equals(other.operatorSubtaskStates); } else { return false; @@ -161,6 +185,7 @@ public class OperatorState implements CompositeStateHandle { "operatorID: " + operatorID + ", parallelism: " + parallelism + ", maxParallelism: " + maxParallelism + + ", coordinatorState: " + (coordinatorState == null ? "(none)" : coordinatorState.getStateSize() + " bytes") + ", sub task states: " + operatorSubtaskStates.size() + ", total size (bytes): " + getStateSize() + ')'; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorStateRepartitioner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorStateRepartitioner.java index a073521bc2d6110bb80543ddb27a713bd1e11229..b9d2eb1138440b17671f39a972620a4021f8419f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorStateRepartitioner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorStateRepartitioner.java @@ -19,7 +19,6 @@ package org.apache.flink.runtime.checkpoint; import org.apache.flink.annotation.Internal; -import org.apache.flink.runtime.state.OperatorStateHandle; import java.util.List; @@ -27,7 +26,7 @@ import java.util.List; * Interface that allows to implement different strategies for repartitioning of operator state as parallelism changes. */ @Internal -public interface OperatorStateRepartitioner { +public interface OperatorStateRepartitioner { /** * @param previousParallelSubtaskStates List with one entry of state handles per parallel subtask of an operator, as they @@ -38,8 +37,8 @@ public interface OperatorStateRepartitioner { * @return List with one entry per parallel subtask. Each subtask receives now one collection of states that build * of the new total state for this subtask. */ - List> repartitionState( - List> previousParallelSubtaskStates, + List> repartitionState( + List> previousParallelSubtaskStates, int oldParallelism, int newParallelism); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorSubtaskState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorSubtaskState.java index 87ab061c070444a7412079a45d9cd3705322f3f2..4099ec11a28dd097a4b900e5f276399c00a5a936 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorSubtaskState.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorSubtaskState.java @@ -19,8 +19,10 @@ package org.apache.flink.runtime.checkpoint; import org.apache.flink.runtime.state.CompositeStateHandle; +import org.apache.flink.runtime.state.InputChannelStateHandle; import org.apache.flink.runtime.state.KeyedStateHandle; import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.ResultSubpartitionStateHandle; import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.state.StateObject; import org.apache.flink.runtime.state.StateUtil; @@ -35,6 +37,8 @@ import javax.annotation.Nullable; import java.util.ArrayList; import java.util.List; +import static org.apache.flink.runtime.checkpoint.StateObjectCollection.emptyIfNull; + /** * This class encapsulates the state for one parallel instance of an operator. The complete state of a (logical) * operator (e.g. a flatmap operator) consists of the union of all {@link OperatorSubtaskState}s from all @@ -80,6 +84,12 @@ public class OperatorSubtaskState implements CompositeStateHandle { @Nonnull private final StateObjectCollection rawKeyedState; + @Nonnull + private final StateObjectCollection inputChannelState; + + @Nonnull + private final StateObjectCollection resultSubpartitionState; + /** * The state size. This is also part of the deserialized state handle. * We store it here in order to not deserialize the state handle when @@ -95,6 +105,22 @@ public class OperatorSubtaskState implements CompositeStateHandle { StateObjectCollection.empty(), StateObjectCollection.empty(), StateObjectCollection.empty(), + StateObjectCollection.empty(), + StateObjectCollection.empty(), + StateObjectCollection.empty()); + } + + public OperatorSubtaskState( + @Nonnull StateObjectCollection managedOperatorState, + @Nonnull StateObjectCollection rawOperatorState, + @Nonnull StateObjectCollection managedKeyedState, + @Nonnull StateObjectCollection rawKeyedState) { + this( + managedOperatorState, + rawOperatorState, + managedKeyedState, + rawKeyedState, + StateObjectCollection.empty(), StateObjectCollection.empty()); } @@ -102,17 +128,23 @@ public class OperatorSubtaskState implements CompositeStateHandle { @Nonnull StateObjectCollection managedOperatorState, @Nonnull StateObjectCollection rawOperatorState, @Nonnull StateObjectCollection managedKeyedState, - @Nonnull StateObjectCollection rawKeyedState) { + @Nonnull StateObjectCollection rawKeyedState, + @Nonnull StateObjectCollection inputChannelState, + @Nonnull StateObjectCollection resultSubpartitionState) { this.managedOperatorState = Preconditions.checkNotNull(managedOperatorState); this.rawOperatorState = Preconditions.checkNotNull(rawOperatorState); this.managedKeyedState = Preconditions.checkNotNull(managedKeyedState); this.rawKeyedState = Preconditions.checkNotNull(rawKeyedState); + this.inputChannelState = Preconditions.checkNotNull(inputChannelState); + this.resultSubpartitionState = Preconditions.checkNotNull(resultSubpartitionState); long calculateStateSize = managedOperatorState.getStateSize(); calculateStateSize += rawOperatorState.getStateSize(); calculateStateSize += managedKeyedState.getStateSize(); calculateStateSize += rawKeyedState.getStateSize(); + calculateStateSize += inputChannelState.getStateSize(); + calculateStateSize += resultSubpartitionState.getStateSize(); stateSize = calculateStateSize; } @@ -121,16 +153,19 @@ public class OperatorSubtaskState implements CompositeStateHandle { * Collections. */ public OperatorSubtaskState( - @Nullable OperatorStateHandle managedOperatorState, - @Nullable OperatorStateHandle rawOperatorState, - @Nullable KeyedStateHandle managedKeyedState, - @Nullable KeyedStateHandle rawKeyedState) { - + @Nullable OperatorStateHandle managedOperatorState, + @Nullable OperatorStateHandle rawOperatorState, + @Nullable KeyedStateHandle managedKeyedState, + @Nullable KeyedStateHandle rawKeyedState, + @Nullable StateObjectCollection inputChannelState, + @Nullable StateObjectCollection resultSubpartitionState) { this( singletonOrEmptyOnNull(managedOperatorState), singletonOrEmptyOnNull(rawOperatorState), singletonOrEmptyOnNull(managedKeyedState), - singletonOrEmptyOnNull(rawKeyedState)); + singletonOrEmptyOnNull(rawKeyedState), + emptyIfNull(inputChannelState), + emptyIfNull(resultSubpartitionState)); } private static StateObjectCollection singletonOrEmptyOnNull(T element) { @@ -171,6 +206,16 @@ public class OperatorSubtaskState implements CompositeStateHandle { return rawKeyedState; } + @Nonnull + public StateObjectCollection getInputChannelState() { + return inputChannelState; + } + + @Nonnull + public StateObjectCollection getResultSubpartitionState() { + return resultSubpartitionState; + } + @Override public void discardState() { try { @@ -179,11 +224,15 @@ public class OperatorSubtaskState implements CompositeStateHandle { managedOperatorState.size() + rawOperatorState.size() + managedKeyedState.size() + - rawKeyedState.size()); + rawKeyedState.size() + + inputChannelState.size() + + resultSubpartitionState.size()); toDispose.addAll(managedOperatorState); toDispose.addAll(rawOperatorState); toDispose.addAll(managedKeyedState); toDispose.addAll(rawKeyedState); + toDispose.addAll(inputChannelState); + toDispose.addAll(resultSubpartitionState); StateUtil.bestEffortDiscardAllStateObjects(toDispose); } catch (Exception e) { LOG.warn("Error while discarding operator states.", e); @@ -236,6 +285,12 @@ public class OperatorSubtaskState implements CompositeStateHandle { if (!getManagedKeyedState().equals(that.getManagedKeyedState())) { return false; } + if (!getInputChannelState().equals(that.getInputChannelState())) { + return false; + } + if (!getResultSubpartitionState().equals(that.getResultSubpartitionState())) { + return false; + } return getRawKeyedState().equals(that.getRawKeyedState()); } @@ -245,6 +300,8 @@ public class OperatorSubtaskState implements CompositeStateHandle { result = 31 * result + getRawOperatorState().hashCode(); result = 31 * result + getManagedKeyedState().hashCode(); result = 31 * result + getRawKeyedState().hashCode(); + result = 31 * result + getInputChannelState().hashCode(); + result = 31 * result + getResultSubpartitionState().hashCode(); result = 31 * result + (int) (getStateSize() ^ (getStateSize() >>> 32)); return result; } @@ -256,6 +313,8 @@ public class OperatorSubtaskState implements CompositeStateHandle { ", operatorStateFromStream=" + rawOperatorState + ", keyedStateFromBackend=" + managedKeyedState + ", keyedStateFromStream=" + rawKeyedState + + ", inputChannelState=" + inputChannelState + + ", resultSubpartitionState=" + resultSubpartitionState + ", stateSize=" + stateSize + '}'; } @@ -264,6 +323,8 @@ public class OperatorSubtaskState implements CompositeStateHandle { return managedOperatorState.hasState() || rawOperatorState.hasState() || managedKeyedState.hasState() - || rawKeyedState.hasState(); + || rawKeyedState.hasState() + || inputChannelState.hasState() + || resultSubpartitionState.hasState(); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java index a596acba9375e852ff87af6931c1657f0d42b718..52844f8cfa374a4da56d141d18c4a640508b9857 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java @@ -27,6 +27,7 @@ import org.apache.flink.runtime.state.CheckpointMetadataOutputStream; import org.apache.flink.runtime.state.CheckpointStorageLocation; import org.apache.flink.runtime.state.CompletedCheckpointStorageLocation; import org.apache.flink.runtime.state.StateUtil; +import org.apache.flink.runtime.state.StreamStateHandle; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.Preconditions; @@ -38,6 +39,7 @@ import javax.annotation.Nullable; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -88,6 +90,8 @@ public class PendingCheckpoint { private final Map notYetAcknowledgedTasks; + private final Set notYetAcknowledgedOperatorCoordinators; + private final List masterStates; private final Set notYetAcknowledgedMasterStates; @@ -117,36 +121,16 @@ public class PendingCheckpoint { private volatile ScheduledFuture cancellerHandle; - private CheckpointException failureCause = null; + private CheckpointException failureCause; // -------------------------------------------------------------------------------------------- - public PendingCheckpoint( - JobID jobId, - long checkpointId, - long checkpointTimestamp, - Map verticesToConfirm, - Collection masterStateIdentifiers, - CheckpointProperties props, - CheckpointStorageLocation targetLocation, - Executor executor) { - - this( - jobId, - checkpointId, - checkpointTimestamp, - verticesToConfirm, - masterStateIdentifiers, - props, - targetLocation, - executor, - new CompletableFuture<>()); - } public PendingCheckpoint( JobID jobId, long checkpointId, long checkpointTimestamp, Map verticesToConfirm, + Collection operatorCoordinatorsToConfirm, Collection masterStateIdentifiers, CheckpointProperties props, CheckpointStorageLocation targetLocation, @@ -166,7 +150,10 @@ public class PendingCheckpoint { this.operatorStates = new HashMap<>(); this.masterStates = new ArrayList<>(masterStateIdentifiers.size()); - this.notYetAcknowledgedMasterStates = new HashSet<>(masterStateIdentifiers); + this.notYetAcknowledgedMasterStates = masterStateIdentifiers.isEmpty() + ? Collections.emptySet() : new HashSet<>(masterStateIdentifiers); + this.notYetAcknowledgedOperatorCoordinators = operatorCoordinatorsToConfirm.isEmpty() + ? Collections.emptySet() : new HashSet<>(operatorCoordinatorsToConfirm); this.acknowledgedTasks = new HashSet<>(verticesToConfirm.size()); this.onCompletionPromise = checkNotNull(onCompletionPromise); } @@ -197,6 +184,10 @@ public class PendingCheckpoint { return notYetAcknowledgedTasks.size(); } + public int getNumberOfNonAcknowledgedOperatorCoordinators() { + return notYetAcknowledgedOperatorCoordinators.size(); + } + public int getNumberOfAcknowledgedTasks() { return numAcknowledgedTasks; } @@ -209,11 +200,21 @@ public class PendingCheckpoint { return masterStates; } - public boolean areMasterStatesFullyAcknowledged() { + public boolean isFullyAcknowledged() { + return areTasksFullyAcknowledged() && + areCoordinatorsFullyAcknowledged() && + areMasterStatesFullyAcknowledged(); + } + + boolean areMasterStatesFullyAcknowledged() { return notYetAcknowledgedMasterStates.isEmpty() && !discarded; } - public boolean areTasksFullyAcknowledged() { + boolean areCoordinatorsFullyAcknowledged() { + return notYetAcknowledgedOperatorCoordinators.isEmpty() && !discarded; + } + + boolean areTasksFullyAcknowledged() { return notYetAcknowledgedTasks.isEmpty() && !discarded; } @@ -291,10 +292,8 @@ public class PendingCheckpoint { public CompletedCheckpoint finalizeCheckpoint() throws IOException { synchronized (lock) { - checkState(areMasterStatesFullyAcknowledged(), - "Pending checkpoint has not been fully acknowledged by master states yet."); - checkState(areTasksFullyAcknowledged(), - "Pending checkpoint has not been fully acknowledged by tasks yet."); + checkState(!isDiscarded(), "checkpoint is discarded"); + checkState(isFullyAcknowledged(), "Pending checkpoint has not been fully acknowledged yet"); // make sure we fulfill the promise with an exception if something fails try { @@ -420,7 +419,6 @@ public class PendingCheckpoint { stateSize, metrics.getSyncDurationMillis(), metrics.getAsyncDurationMillis(), - metrics.getBytesBufferedInAlignment(), alignmentDurationMillis, checkpointStartDelayMillis); @@ -431,6 +429,38 @@ public class PendingCheckpoint { } } + public TaskAcknowledgeResult acknowledgeCoordinatorState( + OperatorCoordinatorCheckpointContext coordinatorInfo, + @Nullable StreamStateHandle stateHandle) { + + synchronized (lock) { + if (discarded) { + return TaskAcknowledgeResult.DISCARDED; + } + + final OperatorID operatorId = coordinatorInfo.operatorId(); + OperatorState operatorState = operatorStates.get(operatorId); + + // sanity check for better error reporting + if (!notYetAcknowledgedOperatorCoordinators.remove(operatorId)) { + return operatorState != null && operatorState.getCoordinatorState() != null + ? TaskAcknowledgeResult.DUPLICATE + : TaskAcknowledgeResult.UNKNOWN; + } + + if (stateHandle != null) { + if (operatorState == null) { + operatorState = new OperatorState( + operatorId, coordinatorInfo.currentParallelism(), coordinatorInfo.maxParallelism()); + operatorStates.put(operatorId, operatorState); + } + operatorState.setCoordinatorState(stateHandle); + } + + return TaskAcknowledgeResult.SUCCESS; + } + } + /** * Acknowledges a master state (state generated on the checkpoint coordinator) to * the pending checkpoint. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpointStats.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpointStats.java index 323979d61a468b63f21ec36499f72fba33867d35..1ec405a9ba009a879bafb49e6bf53913729c6518 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpointStats.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpointStats.java @@ -54,9 +54,6 @@ public class PendingCheckpointStats extends AbstractCheckpointStats { /** Current checkpoint state size over all collected subtasks. */ private volatile long currentStateSize; - /** Current buffered bytes during alignment over all collected subtasks. */ - private volatile long currentAlignmentBuffered; - /** Stats of the latest acknowledged subtask. */ private volatile SubtaskStateStats latestAcknowledgedSubtask; @@ -97,11 +94,6 @@ public class PendingCheckpointStats extends AbstractCheckpointStats { return currentStateSize; } - @Override - public long getAlignmentBuffered() { - return currentAlignmentBuffered; - } - @Override public SubtaskStateStats getLatestAcknowledgedSubtaskStats() { return latestAcknowledgedSubtask; @@ -127,11 +119,6 @@ public class PendingCheckpointStats extends AbstractCheckpointStats { currentStateSize += subtask.getStateSize(); - long alignmentBuffered = subtask.getAlignmentBuffered(); - if (alignmentBuffered > 0) { - currentAlignmentBuffered += alignmentBuffered; - } - return true; } else { return false; @@ -153,7 +140,6 @@ public class PendingCheckpointStats extends AbstractCheckpointStats { new HashMap<>(taskStats), currentNumAcknowledgedSubtasks, currentStateSize, - currentAlignmentBuffered, latestAcknowledgedSubtask, externalPointer); @@ -177,7 +163,6 @@ public class PendingCheckpointStats extends AbstractCheckpointStats { new HashMap<>(taskStats), currentNumAcknowledgedSubtasks, currentStateSize, - currentAlignmentBuffered, failureTimestamp, latestAcknowledgedSubtask, cause); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PrioritizedOperatorSubtaskState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PrioritizedOperatorSubtaskState.java index 0a949ad62e641dddb2708e37efd5709322bba013..bb133e74b3c45030d45f03c0a83f5b021030d962 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PrioritizedOperatorSubtaskState.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PrioritizedOperatorSubtaskState.java @@ -19,8 +19,10 @@ package org.apache.flink.runtime.checkpoint; import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.state.InputChannelStateHandle; import org.apache.flink.runtime.state.KeyedStateHandle; import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.ResultSubpartitionStateHandle; import org.apache.flink.runtime.state.StateObject; import org.apache.commons.lang3.BooleanUtils; @@ -31,6 +33,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.function.BiFunction; +import java.util.function.Function; /** * This class is a wrapper over multiple alternative {@link OperatorSubtaskState} that are (partial) substitutes for @@ -60,20 +63,28 @@ public class PrioritizedOperatorSubtaskState { /** List of prioritized snapshot alternatives for raw keyed state. */ private final List> prioritizedRawKeyedState; + private final List> prioritizedInputChannelState; + + private final List> prioritizedResultSubpartitionState; + /** Signal flag if this represents state for a restored operator. */ private final boolean restored; PrioritizedOperatorSubtaskState( - @Nonnull List> prioritizedManagedKeyedState, - @Nonnull List> prioritizedRawKeyedState, - @Nonnull List> prioritizedManagedOperatorState, - @Nonnull List> prioritizedRawOperatorState, - boolean restored) { + @Nonnull List> prioritizedManagedKeyedState, + @Nonnull List> prioritizedRawKeyedState, + @Nonnull List> prioritizedManagedOperatorState, + @Nonnull List> prioritizedRawOperatorState, + @Nonnull List> prioritizedInputChannelState, + @Nonnull List> prioritizedResultSubpartitionState, + boolean restored) { this.prioritizedManagedOperatorState = prioritizedManagedOperatorState; this.prioritizedRawOperatorState = prioritizedRawOperatorState; this.prioritizedManagedKeyedState = prioritizedManagedKeyedState; this.prioritizedRawKeyedState = prioritizedRawKeyedState; + this.prioritizedInputChannelState = prioritizedInputChannelState; + this.prioritizedResultSubpartitionState = prioritizedResultSubpartitionState; this.restored = restored; } @@ -153,6 +164,16 @@ public class PrioritizedOperatorSubtaskState { return lastElement(prioritizedRawKeyedState); } + @Nonnull + public StateObjectCollection getPrioritizedInputChannelState() { + return lastElement(prioritizedInputChannelState); + } + + @Nonnull + public StateObjectCollection getPrioritizedResultSubpartitionState() { + return lastElement(prioritizedResultSubpartitionState); + } + // ----------------------------------------------------------------------------------------------------------------- /** @@ -213,6 +234,8 @@ public class PrioritizedOperatorSubtaskState { List> managedKeyedAlternatives = new ArrayList<>(size); List> rawOperatorAlternatives = new ArrayList<>(size); List> rawKeyedAlternatives = new ArrayList<>(size); + List> inputChannelStateAlternatives = new ArrayList<>(size); + List> resultSubpartitionStateAlternatives = new ArrayList<>(size); for (OperatorSubtaskState subtaskState : alternativesByPriority) { @@ -221,34 +244,36 @@ public class PrioritizedOperatorSubtaskState { rawKeyedAlternatives.add(subtaskState.getRawKeyedState()); managedOperatorAlternatives.add(subtaskState.getManagedOperatorState()); rawOperatorAlternatives.add(subtaskState.getRawOperatorState()); + inputChannelStateAlternatives.add(subtaskState.getInputChannelState()); + resultSubpartitionStateAlternatives.add(subtaskState.getResultSubpartitionState()); } } - // Key-groups should match. - BiFunction keyedStateApprover = - (ref, alt) -> ref.getKeyGroupRange().equals(alt.getKeyGroupRange()); - - // State meta data should match. - BiFunction operatorStateApprover = - (ref, alt) -> ref.getStateNameToPartitionOffsets().equals(alt.getStateNameToPartitionOffsets()); - return new PrioritizedOperatorSubtaskState( resolvePrioritizedAlternatives( jobManagerState.getManagedKeyedState(), managedKeyedAlternatives, - keyedStateApprover), + eqStateApprover(KeyedStateHandle::getKeyGroupRange)), resolvePrioritizedAlternatives( jobManagerState.getRawKeyedState(), rawKeyedAlternatives, - keyedStateApprover), + eqStateApprover(KeyedStateHandle::getKeyGroupRange)), resolvePrioritizedAlternatives( jobManagerState.getManagedOperatorState(), managedOperatorAlternatives, - operatorStateApprover), + eqStateApprover(OperatorStateHandle::getStateNameToPartitionOffsets)), resolvePrioritizedAlternatives( jobManagerState.getRawOperatorState(), rawOperatorAlternatives, - operatorStateApprover), + eqStateApprover(OperatorStateHandle::getStateNameToPartitionOffsets)), + resolvePrioritizedAlternatives( + jobManagerState.getInputChannelState(), + inputChannelStateAlternatives, + eqStateApprover(InputChannelStateHandle::getInfo)), + resolvePrioritizedAlternatives( + jobManagerState.getResultSubpartitionState(), + resultSubpartitionStateAlternatives, + eqStateApprover(ResultSubpartitionStateHandle::getInfo)), restored); } @@ -297,4 +322,8 @@ public class PrioritizedOperatorSubtaskState { return Collections.unmodifiableList(approved); } } + + private static BiFunction eqStateApprover(Function identityExtractor) { + return (ref, alt) -> identityExtractor.apply(ref).equals(identityExtractor.apply(alt)); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/RoundRobinOperatorStateRepartitioner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/RoundRobinOperatorStateRepartitioner.java index cea2663cd4d492cc7402854d95ecf351f7c49a13..a3025185632ec253f7eb5a5cb4de6aec765d5f79 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/RoundRobinOperatorStateRepartitioner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/RoundRobinOperatorStateRepartitioner.java @@ -39,9 +39,9 @@ import java.util.stream.Collectors; * Current default implementation of {@link OperatorStateRepartitioner} that redistributes state in round robin fashion. */ @Internal -public class RoundRobinOperatorStateRepartitioner implements OperatorStateRepartitioner { +public class RoundRobinOperatorStateRepartitioner implements OperatorStateRepartitioner { - public static final OperatorStateRepartitioner INSTANCE = new RoundRobinOperatorStateRepartitioner(); + public static final OperatorStateRepartitioner INSTANCE = new RoundRobinOperatorStateRepartitioner(); private static final boolean OPTIMIZE_MEMORY_USE = false; @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateAssignmentOperation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateAssignmentOperation.java index 8e7b3558ad8b27de405f85846ce5453325a54fca..7f498281bc7b0d43ad380f72aa98eb98ff017b57 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateAssignmentOperation.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateAssignmentOperation.java @@ -19,16 +19,21 @@ package org.apache.flink.runtime.checkpoint; import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.executiongraph.Execution; import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.jobgraph.OperatorInstanceID; +import org.apache.flink.runtime.state.AbstractChannelStateHandle; +import org.apache.flink.runtime.state.InputChannelStateHandle; import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.KeyGroupRangeAssignment; import org.apache.flink.runtime.state.KeyGroupsStateHandle; import org.apache.flink.runtime.state.KeyedStateHandle; import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.ResultSubpartitionStateHandle; +import org.apache.flink.runtime.state.StateObject; import org.apache.flink.util.Preconditions; import org.slf4j.Logger; @@ -36,13 +41,14 @@ import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.function.Function; +import static java.util.Collections.emptyList; import static org.apache.flink.util.Preconditions.checkNotNull; import static org.apache.flink.util.Preconditions.checkState; @@ -100,11 +106,9 @@ public class StateAssignmentOperation { } operatorStates.add(operatorState); } - if (statelessTask) { // skip tasks where no operator has any state - continue; + if (!statelessTask) { // skip tasks where no operator has any state + assignAttemptState(executionJobVertex, operatorStates); } - - assignAttemptState(executionJobVertex, operatorStates); } } @@ -143,22 +147,33 @@ public class StateAssignmentOperation { * op2 state2,0 state2,1 state2,2 state2,3 * op3 state3,0 state3,1 state3,2 state3,3 */ - Map> newManagedOperatorStates = - new HashMap<>(expectedNumberOfSubTasks); - Map> newRawOperatorStates = - new HashMap<>(expectedNumberOfSubTasks); - - reDistributePartitionableStates( + Map> newManagedOperatorStates = reDistributePartitionableStates( operatorStates, newParallelism, operatorIDs, - newManagedOperatorStates, - newRawOperatorStates); + OperatorSubtaskState::getManagedOperatorState, + RoundRobinOperatorStateRepartitioner.INSTANCE); + Map> newRawOperatorStates = reDistributePartitionableStates( + operatorStates, + newParallelism, + operatorIDs, + OperatorSubtaskState::getRawOperatorState, + RoundRobinOperatorStateRepartitioner.INSTANCE); + final Map> newInputChannelState = reDistributePartitionableStates( + operatorStates, + newParallelism, + operatorIDs, + OperatorSubtaskState::getInputChannelState, + channelStateNonRescalingRepartitioner("input channel")); + final Map> newResultSubpartitionState = reDistributePartitionableStates( + operatorStates, + newParallelism, + operatorIDs, + OperatorSubtaskState::getResultSubpartitionState, + channelStateNonRescalingRepartitioner("result subpartition")); - Map> newManagedKeyedState = - new HashMap<>(expectedNumberOfSubTasks); - Map> newRawKeyedState = - new HashMap<>(expectedNumberOfSubTasks); + Map> newManagedKeyedState = new HashMap<>(expectedNumberOfSubTasks); + Map> newRawKeyedState = new HashMap<>(expectedNumberOfSubTasks); reDistributeKeyedStates( operatorStates, @@ -182,6 +197,8 @@ public class StateAssignmentOperation { executionJobVertex, newManagedOperatorStates, newRawOperatorStates, + newInputChannelState, + newResultSubpartitionState, newManagedKeyedState, newRawKeyedState, newParallelism); @@ -191,6 +208,8 @@ public class StateAssignmentOperation { ExecutionJobVertex executionJobVertex, Map> subManagedOperatorState, Map> subRawOperatorState, + Map> inputChannelStates, + Map> resultSubpartitionStates, Map> subManagedKeyedState, Map> subRawKeyedState, int newParallelism) { @@ -212,6 +231,8 @@ public class StateAssignmentOperation { instanceID, subManagedOperatorState, subRawOperatorState, + inputChannelStates, + resultSubpartitionStates, subManagedKeyedState, subRawKeyedState); @@ -232,11 +253,15 @@ public class StateAssignmentOperation { OperatorInstanceID instanceID, Map> subManagedOperatorState, Map> subRawOperatorState, + Map> inputChannelStates, + Map> resultSubpartitionStates, Map> subManagedKeyedState, Map> subRawKeyedState) { if (!subManagedOperatorState.containsKey(instanceID) && !subRawOperatorState.containsKey(instanceID) && + !inputChannelStates.containsKey(instanceID) && + !resultSubpartitionStates.containsKey(instanceID) && !subManagedKeyedState.containsKey(instanceID) && !subRawKeyedState.containsKey(instanceID)) { @@ -246,10 +271,12 @@ public class StateAssignmentOperation { checkState(!subRawKeyedState.containsKey(instanceID)); } return new OperatorSubtaskState( - new StateObjectCollection<>(subManagedOperatorState.getOrDefault(instanceID, Collections.emptyList())), - new StateObjectCollection<>(subRawOperatorState.getOrDefault(instanceID, Collections.emptyList())), - new StateObjectCollection<>(subManagedKeyedState.getOrDefault(instanceID, Collections.emptyList())), - new StateObjectCollection<>(subRawKeyedState.getOrDefault(instanceID, Collections.emptyList()))); + new StateObjectCollection<>(subManagedOperatorState.getOrDefault(instanceID, emptyList())), + new StateObjectCollection<>(subRawOperatorState.getOrDefault(instanceID, emptyList())), + new StateObjectCollection<>(subManagedKeyedState.getOrDefault(instanceID, emptyList())), + new StateObjectCollection<>(subRawKeyedState.getOrDefault(instanceID, emptyList())), + new StateObjectCollection<>(inputChannelStates.getOrDefault(instanceID, emptyList())), + new StateObjectCollection<>(resultSubpartitionStates.getOrDefault(instanceID, emptyList()))); } public void checkParallelismPreconditions(List operatorStates, ExecutionJobVertex executionJobVertex) { @@ -302,8 +329,8 @@ public class StateAssignmentOperation { subManagedKeyedState = operatorState.getState(subTaskIndex).getManagedKeyedState().asList(); subRawKeyedState = operatorState.getState(subTaskIndex).getRawKeyedState().asList(); } else { - subManagedKeyedState = Collections.emptyList(); - subRawKeyedState = Collections.emptyList(); + subManagedKeyedState = emptyList(); + subRawKeyedState = emptyList(); } } else { subManagedKeyedState = getManagedKeyedStateHandles(operatorState, keyGroupPartitions.get(subTaskIndex)); @@ -311,79 +338,54 @@ public class StateAssignmentOperation { } if (subManagedKeyedState.isEmpty() && subRawKeyedState.isEmpty()) { - return new Tuple2<>(Collections.emptyList(), Collections.emptyList()); + return new Tuple2<>(emptyList(), emptyList()); } else { return new Tuple2<>(subManagedKeyedState, subRawKeyedState); } } - public static void reDistributePartitionableStates( + public static Map> reDistributePartitionableStates( List oldOperatorStates, int newParallelism, List newOperatorIDs, - Map> newManagedOperatorStates, - Map> newRawOperatorStates) { + Function> extractHandle, + OperatorStateRepartitioner stateRepartitioner) { //TODO: rewrite this method to only use OperatorID checkState(newOperatorIDs.size() == oldOperatorStates.size(), "This method still depends on the order of the new and old operators"); // The nested list wraps as the level of operator -> subtask -> state object collection - List>> oldManagedOperatorStates = new ArrayList<>(oldOperatorStates.size()); - List>> oldRawOperatorStates = new ArrayList<>(oldOperatorStates.size()); - - splitManagedAndRawOperatorStates(oldOperatorStates, oldManagedOperatorStates, oldRawOperatorStates); - OperatorStateRepartitioner opStateRepartitioner = RoundRobinOperatorStateRepartitioner.INSTANCE; + List>> oldStates = splitManagedAndRawOperatorStates(oldOperatorStates, extractHandle); + Map> result = new HashMap<>(); for (int operatorIndex = 0; operatorIndex < newOperatorIDs.size(); operatorIndex++) { - OperatorState operatorState = oldOperatorStates.get(operatorIndex); - int oldParallelism = operatorState.getParallelism(); - - OperatorID operatorID = newOperatorIDs.get(operatorIndex); - - newManagedOperatorStates.putAll(applyRepartitioner( - operatorID, - opStateRepartitioner, - oldManagedOperatorStates.get(operatorIndex), - oldParallelism, - newParallelism)); - - newRawOperatorStates.putAll(applyRepartitioner( - operatorID, - opStateRepartitioner, - oldRawOperatorStates.get(operatorIndex), - oldParallelism, + result.putAll(applyRepartitioner( + newOperatorIDs.get(operatorIndex), + stateRepartitioner, + oldStates.get(operatorIndex), + oldOperatorStates.get(operatorIndex).getParallelism(), newParallelism)); } + + return result; } - private static void splitManagedAndRawOperatorStates( - List operatorStates, - List>> managedOperatorStates, - List>> rawOperatorStates) { + private static List>> splitManagedAndRawOperatorStates( + List operatorStates, + Function> extracthandle) { + List>> result = new ArrayList<>(); for (OperatorState operatorState : operatorStates) { + List> statePerSubtask = new ArrayList<>(operatorState.getParallelism()); - final int parallelism = operatorState.getParallelism(); - List> managedOpStatePerSubtasks = new ArrayList<>(parallelism); - List> rawOpStatePerSubtasks = new ArrayList<>(parallelism); - - for (int subTaskIndex = 0; subTaskIndex < parallelism; subTaskIndex++) { - OperatorSubtaskState operatorSubtaskState = operatorState.getState(subTaskIndex); - if (operatorSubtaskState == null) { - managedOpStatePerSubtasks.add(Collections.emptyList()); - rawOpStatePerSubtasks.add(Collections.emptyList()); - } else { - StateObjectCollection managed = operatorSubtaskState.getManagedOperatorState(); - StateObjectCollection raw = operatorSubtaskState.getRawOperatorState(); - - managedOpStatePerSubtasks.add(managed.asList()); - rawOpStatePerSubtasks.add(raw.asList()); - } + for (int subTaskIndex = 0; subTaskIndex < operatorState.getParallelism(); subTaskIndex++) { + OperatorSubtaskState subtaskState = operatorState.getState(subTaskIndex); + statePerSubtask.add(subtaskState == null ? emptyList() : extracthandle.apply(subtaskState).asList()); } - managedOperatorStates.add(managedOpStatePerSubtasks); - rawOperatorStates.add(rawOpStatePerSubtasks); + result.add(statePerSubtask); } + return result; } /** @@ -395,8 +397,8 @@ public class StateAssignmentOperation { * @return all managedKeyedStateHandles which have intersection with given KeyGroupRange */ public static List getManagedKeyedStateHandles( - OperatorState operatorState, - KeyGroupRange subtaskKeyGroupRange) { + OperatorState operatorState, + KeyGroupRange subtaskKeyGroupRange) { final int parallelism = operatorState.getParallelism(); @@ -430,8 +432,8 @@ public class StateAssignmentOperation { * @return all rawKeyedStateHandles which have intersection with given KeyGroupRange */ public static List getRawKeyedStateHandles( - OperatorState operatorState, - KeyGroupRange subtaskKeyGroupRange) { + OperatorState operatorState, + KeyGroupRange subtaskKeyGroupRange) { final int parallelism = operatorState.getParallelism(); @@ -459,10 +461,11 @@ public class StateAssignmentOperation { /** * Extracts certain key group ranges from the given state handles and adds them to the collector. */ - private static void extractIntersectingState( - Collection originalSubtaskStateHandles, - KeyGroupRange rangeToExtract, - List extractedStateCollector) { + @VisibleForTesting + public static void extractIntersectingState( + Collection originalSubtaskStateHandles, + KeyGroupRange rangeToExtract, + List extractedStateCollector) { for (KeyedStateHandle keyedStateHandle : originalSubtaskStateHandles) { @@ -568,20 +571,20 @@ public class StateAssignmentOperation { } } - public static Map> applyRepartitioner( - OperatorID operatorID, - OperatorStateRepartitioner opStateRepartitioner, - List> chainOpParallelStates, - int oldParallelism, - int newParallelism) { + public static Map> applyRepartitioner( + OperatorID operatorID, + OperatorStateRepartitioner opStateRepartitioner, + List> chainOpParallelStates, + int oldParallelism, + int newParallelism) { - List> states = applyRepartitioner( + List> states = applyRepartitioner( opStateRepartitioner, chainOpParallelStates, oldParallelism, newParallelism); - Map> result = new HashMap<>(states.size()); + Map> result = new HashMap<>(states.size()); for (int subtaskIndex = 0; subtaskIndex < states.size(); subtaskIndex++) { checkNotNull(states.get(subtaskIndex) != null, "states.get(subtaskIndex) is null"); @@ -602,14 +605,14 @@ public class StateAssignmentOperation { * @return repartitioned state */ // TODO rewrite based on operator id - public static List> applyRepartitioner( - OperatorStateRepartitioner opStateRepartitioner, - List> chainOpParallelStates, - int oldParallelism, - int newParallelism) { + public static List> applyRepartitioner( + OperatorStateRepartitioner opStateRepartitioner, + List> chainOpParallelStates, + int oldParallelism, + int newParallelism) { if (chainOpParallelStates == null) { - return Collections.emptyList(); + return emptyList(); } return opStateRepartitioner.repartitionState( @@ -618,26 +621,16 @@ public class StateAssignmentOperation { newParallelism); } - /** - * Determine the subset of {@link KeyGroupsStateHandle KeyGroupsStateHandles} with correct - * key group index for the given subtask {@link KeyGroupRange}. - * - *

This is publicly visible to be used in tests. - */ - public static List getKeyedStateHandles( - Collection keyedStateHandles, - KeyGroupRange subtaskKeyGroupRange) { - - List subtaskKeyedStateHandles = new ArrayList<>(keyedStateHandles.size()); - - for (KeyedStateHandle keyedStateHandle : keyedStateHandles) { - KeyedStateHandle intersectedKeyedStateHandle = keyedStateHandle.getIntersection(subtaskKeyGroupRange); - - if (intersectedKeyedStateHandle != null) { - subtaskKeyedStateHandles.add(intersectedKeyedStateHandle); - } - } - - return subtaskKeyedStateHandles; + static > OperatorStateRepartitioner channelStateNonRescalingRepartitioner(String logStateName) { + return (previousParallelSubtaskStates, oldParallelism, newParallelism) -> { + Preconditions.checkArgument( + oldParallelism == newParallelism || + previousParallelSubtaskStates.stream() + .flatMap(s -> s.stream().map(l -> l.getOffsets())) + .allMatch(List::isEmpty), + String.format("rescaling not supported for %s state (old: %d, new: %d)", logStateName, oldParallelism, newParallelism)); + return previousParallelSubtaskStates; + }; } + } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateObjectCollection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateObjectCollection.java index 9d4c32b06d442ac6e795debf59894e8e86c37faa..02872eee650f58fcce0b2ec79111c13f59e1f06a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateObjectCollection.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateObjectCollection.java @@ -196,6 +196,10 @@ public class StateObjectCollection implements Collection< return (StateObjectCollection) EMPTY; } + public static StateObjectCollection emptyIfNull(StateObjectCollection collection) { + return collection == null ? empty() : collection; + } + public static StateObjectCollection singleton(T stateObject) { return new StateObjectCollection<>(Collections.singleton(stateObject)); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/SubtaskStateStats.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/SubtaskStateStats.java index 9b699044f6cd5df0be14c7276b2fa9a8f9fb2f9f..5db5632e8b132e0d96c212ed2ba965736357b43b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/SubtaskStateStats.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/SubtaskStateStats.java @@ -53,9 +53,6 @@ public class SubtaskStateStats implements Serializable { /** Checkpoint duration at the operator (async part) in milliseconds. */ private final long asyncCheckpointDuration; - /** Number of buffered bytes during alignment. */ - private final long alignmentBuffered; - /** Alignment duration in milliseconds. */ private final long alignmentDuration; @@ -68,7 +65,6 @@ public class SubtaskStateStats implements Serializable { long stateSize, long syncCheckpointDuration, long asyncCheckpointDuration, - long alignmentBuffered, long alignmentDuration, long checkpointStartDelay) { @@ -79,7 +75,6 @@ public class SubtaskStateStats implements Serializable { this.ackTimestamp = ackTimestamp; this.syncCheckpointDuration = syncCheckpointDuration; this.asyncCheckpointDuration = asyncCheckpointDuration; - this.alignmentBuffered = alignmentBuffered; this.alignmentDuration = alignmentDuration; this.checkpointStartDelay = checkpointStartDelay; } @@ -136,14 +131,6 @@ public class SubtaskStateStats implements Serializable { return asyncCheckpointDuration; } - /** - * @return Number of bytes buffered during stream alignment (for exactly-once only) or - * -1 if the runtime did not report this. - */ - public long getAlignmentBuffered() { - return alignmentBuffered; - } - /** * @return Duration of the stream alignment (for exactly-once only) or -1 if the * runtime did not report this. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/TaskStateStats.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/TaskStateStats.java index b1e2f992227f1700f57948875e05634ba0aa7f35..2895d357b038e199bf121919228b69a0a60a641b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/TaskStateStats.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/TaskStateStats.java @@ -120,14 +120,6 @@ public class TaskStateStats implements Serializable { return summaryStats.getStateSizeStats().getSum(); } - /** - * @return Total buffered bytes during alignment over all subtasks or -1 if the - * runtime did not report this.. - */ - public long getAlignmentBuffered() { - return summaryStats.getAlignmentBufferedStats().getSum(); - } - /** * Returns the duration of this checkpoint at the task/operator calculated * as the time since triggering until the latest acknowledged subtask @@ -176,7 +168,6 @@ public class TaskStateStats implements Serializable { private MinMaxAvgStats ackTimestamp = new MinMaxAvgStats(); private MinMaxAvgStats syncCheckpointDuration = new MinMaxAvgStats(); private MinMaxAvgStats asyncCheckpointDuration = new MinMaxAvgStats(); - private MinMaxAvgStats alignmentBuffered = new MinMaxAvgStats(); private MinMaxAvgStats alignmentDuration = new MinMaxAvgStats(); private MinMaxAvgStats checkpointStartDelay = new MinMaxAvgStats(); @@ -185,7 +176,6 @@ public class TaskStateStats implements Serializable { ackTimestamp.add(subtaskStats.getAckTimestamp()); syncCheckpointDuration.add(subtaskStats.getSyncCheckpointDuration()); asyncCheckpointDuration.add(subtaskStats.getAsyncCheckpointDuration()); - alignmentBuffered.add(subtaskStats.getAlignmentBuffered()); alignmentDuration.add(subtaskStats.getAlignmentDuration()); checkpointStartDelay.add(subtaskStats.getCheckpointStartDelay()); } @@ -206,10 +196,6 @@ public class TaskStateStats implements Serializable { return asyncCheckpointDuration; } - public MinMaxAvgStats getAlignmentBufferedStats() { - return alignmentBuffered; - } - public MinMaxAvgStats getAlignmentDurationStats() { return alignmentDuration; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateCheckpointWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateCheckpointWriter.java new file mode 100644 index 0000000000000000000000000000000000000000..ef0485768ab394a334aeacafa9016bc6d2a2269d --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateCheckpointWriter.java @@ -0,0 +1,202 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.checkpoint.channel; + +import org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter.ChannelStateWriteResult; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.state.AbstractChannelStateHandle; +import org.apache.flink.runtime.state.CheckpointStreamFactory; +import org.apache.flink.runtime.state.CheckpointStreamFactory.CheckpointStateOutputStream; +import org.apache.flink.runtime.state.InputChannelStateHandle; +import org.apache.flink.runtime.state.ResultSubpartitionStateHandle; +import org.apache.flink.runtime.state.StreamStateHandle; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.function.RunnableWithException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.concurrent.NotThreadSafe; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.function.BiFunction; + +import static org.apache.flink.runtime.state.CheckpointedStateScope.EXCLUSIVE; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; + +/** + * Writes channel state for a specific checkpoint-subtask-attempt triple. + */ +@NotThreadSafe +class ChannelStateCheckpointWriter { + private static final Logger LOG = LoggerFactory.getLogger(ChannelStateCheckpointWriter.class); + + private final DataOutputStream dataStream; + private final CheckpointStateOutputStream checkpointStream; + private final ChannelStateWriteResult result; + private final Map> inputChannelOffsets = new HashMap<>(); + private final Map> resultSubpartitionOffsets = new HashMap<>(); + private final ChannelStateSerializer serializer; + private final long checkpointId; + private boolean allInputsReceived = false; + private boolean allOutputsReceived = false; + private final RunnableWithException onComplete; + + ChannelStateCheckpointWriter( + CheckpointStartRequest startCheckpointItem, + CheckpointStreamFactory streamFactory, + ChannelStateSerializer serializer, + RunnableWithException onComplete) throws Exception { + this( + startCheckpointItem.getCheckpointId(), + startCheckpointItem.getTargetResult(), + streamFactory.createCheckpointStateOutputStream(EXCLUSIVE), + serializer, + onComplete); + } + + ChannelStateCheckpointWriter( + long checkpointId, + ChannelStateWriteResult result, + CheckpointStateOutputStream stream, + ChannelStateSerializer serializer, + RunnableWithException onComplete) throws Exception { + this(checkpointId, result, serializer, onComplete, stream, new DataOutputStream(stream)); + } + + ChannelStateCheckpointWriter( + long checkpointId, + ChannelStateWriteResult result, + ChannelStateSerializer serializer, + RunnableWithException onComplete, + CheckpointStateOutputStream checkpointStateOutputStream, + DataOutputStream dataStream) throws Exception { + this.checkpointId = checkpointId; + this.result = checkNotNull(result); + this.checkpointStream = checkNotNull(checkpointStateOutputStream); + this.serializer = checkNotNull(serializer); + this.dataStream = checkNotNull(dataStream); + this.onComplete = checkNotNull(onComplete); + runWithChecks(() -> serializer.writeHeader(dataStream)); + } + + void writeInput(InputChannelInfo info, Buffer... flinkBuffers) throws Exception { + write(inputChannelOffsets, info, flinkBuffers, !allInputsReceived); + } + + void writeOutput(ResultSubpartitionInfo info, Buffer... flinkBuffers) throws Exception { + write(resultSubpartitionOffsets, info, flinkBuffers, !allOutputsReceived); + } + + private void write(Map> offsets, K key, Buffer[] flinkBuffers, boolean precondition) throws Exception { + try { + if (result.isDone()) { + return; + } + runWithChecks(() -> { + checkState(precondition); + offsets + .computeIfAbsent(key, unused -> new ArrayList<>()) + .add(checkpointStream.getPos()); + serializer.writeData(dataStream, flinkBuffers); + }); + } finally { + for (Buffer flinkBuffer : flinkBuffers) { + flinkBuffer.recycleBuffer(); + } + } + } + + void completeInput() throws Exception { + LOG.debug("complete input, output completed: {}", allOutputsReceived); + complete(!allInputsReceived, () -> allInputsReceived = true); + } + + void completeOutput() throws Exception { + LOG.debug("complete output, input completed: {}", allInputsReceived); + complete(!allOutputsReceived, () -> allOutputsReceived = true); + } + + private void complete(boolean precondition, RunnableWithException complete) throws Exception { + if (result.isDone()) { + // likely after abort - only need to set the flag run onComplete callback + doComplete(precondition, complete, onComplete); + } else { + runWithChecks(() -> doComplete(precondition, complete, onComplete, this::finishWriteAndResult)); + } + } + + private void finishWriteAndResult() throws IOException { + dataStream.flush(); + StreamStateHandle underlying = checkpointStream.closeAndGetHandle(); + complete( + result.inputChannelStateHandles, + inputChannelOffsets, + (chan, offsets) -> new InputChannelStateHandle(chan, underlying, offsets)); + complete( + result.resultSubpartitionStateHandles, + resultSubpartitionOffsets, + (chan, offsets) -> new ResultSubpartitionStateHandle(chan, underlying, offsets)); + } + + private void doComplete(boolean precondition, RunnableWithException complete, RunnableWithException... callbacks) throws Exception { + Preconditions.checkArgument(precondition); + complete.run(); + if (allInputsReceived && allOutputsReceived) { + for (RunnableWithException callback : callbacks) { + callback.run(); + } + } + } + + private > void complete( + CompletableFuture> future, + Map> offsets, + BiFunction, H> buildHandle) { + final Collection handles = new ArrayList<>(); + for (Map.Entry> e : offsets.entrySet()) { + handles.add(buildHandle.apply(e.getKey(), e.getValue())); + } + future.complete(handles); + LOG.debug("channel state write completed, checkpointId: {}, handles: {}", checkpointId, handles); + } + + private void runWithChecks(RunnableWithException r) throws Exception { + try { + checkState(!result.isDone(), "result is already completed", result); + r.run(); + } catch (Exception e) { + fail(e); + throw e; + } + } + + public void fail(Throwable e) throws Exception { + result.fail(e); + checkpointStream.close(); + } + +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateReader.java new file mode 100644 index 0000000000000000000000000000000000000000..0753e7a533761ca60e88fa591664f5cd5456949b --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateReader.java @@ -0,0 +1,67 @@ +package org.apache.flink.runtime.checkpoint.channel; +/* + * 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. + */ + +import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.BufferBuilder; + +import java.io.IOException; + +/** + * Reads channel state saved during checkpoint/savepoint. + */ +@Internal +public interface ChannelStateReader extends AutoCloseable { + + /** + * Status of reading result. + */ + enum ReadResult { HAS_MORE_DATA, NO_MORE_DATA } + + /** + * Put data into the supplied buffer to be injected into + * {@link org.apache.flink.runtime.io.network.partition.consumer.InputChannel InputChannel}. + */ + ReadResult readInputData(InputChannelInfo info, Buffer buffer) throws IOException; + + /** + * Put data into the supplied buffer to be injected into + * {@link org.apache.flink.runtime.io.network.partition.ResultSubpartition ResultSubpartition}. + */ + ReadResult readOutputData(ResultSubpartitionInfo info, BufferBuilder bufferBuilder) throws IOException; + + @Override + void close() throws Exception; + + ChannelStateReader NO_OP = new ChannelStateReader() { + + @Override + public ReadResult readInputData(InputChannelInfo info, Buffer buffer) { + return ReadResult.NO_MORE_DATA; + } + + @Override + public ReadResult readOutputData(ResultSubpartitionInfo info, BufferBuilder bufferBuilder) { + return ReadResult.NO_MORE_DATA; + } + + @Override + public void close() { + } + }; +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateReaderImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateReaderImpl.java new file mode 100644 index 0000000000000000000000000000000000000000..d326853f09825533fbf73a39e3502184300f4ea9 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateReaderImpl.java @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.checkpoint.channel; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; +import org.apache.flink.runtime.checkpoint.channel.RefCountingFSDataInputStream.RefCountingFSDataInputStreamFactory; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.BufferBuilder; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.state.AbstractChannelStateHandle; +import org.apache.flink.util.Preconditions; + +import org.apache.flink.shaded.guava18.com.google.common.io.Closer; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.concurrent.NotThreadSafe; + +import java.io.IOException; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; + +import static java.util.Arrays.asList; +import static org.apache.flink.util.Preconditions.checkState; + +/** + * {@link ChannelStateReader} implementation. Usage considerations: + *

    + *
  1. state of a channel can be read once per instance of this class; once done it returns + * {@link org.apache.flink.runtime.checkpoint.channel.ChannelStateReader.ReadResult#NO_MORE_DATA NO_MORE_DATA}
  2. + *
  3. reader/writer indices of the passed buffer are respected and updated
  4. + *
  5. buffers must be prepared (cleared) before passing to reader
  6. + *
  7. buffers must be released after use
  8. + *
+ */ +@NotThreadSafe +@Internal +public class ChannelStateReaderImpl implements ChannelStateReader { + private static final Logger log = LoggerFactory.getLogger(ChannelStateReaderImpl.class); + + private final Map inputChannelHandleReaders; + private final Map resultSubpartitionHandleReaders; + private boolean isClosed = false; + + public ChannelStateReaderImpl(TaskStateSnapshot snapshot) { + this(snapshot, new ChannelStateSerializerImpl()); + } + + ChannelStateReaderImpl(TaskStateSnapshot snapshot, ChannelStateDeserializer serializer) { + RefCountingFSDataInputStreamFactory streamFactory = new RefCountingFSDataInputStreamFactory(serializer); + final HashMap inputChannelHandleReadersTmp = new HashMap<>(); + final HashMap resultSubpartitionHandleReadersTmp = new HashMap<>(); + for (Map.Entry e : snapshot.getSubtaskStateMappings()) { + addReaders(inputChannelHandleReadersTmp, e.getValue().getInputChannelState(), streamFactory); + addReaders(resultSubpartitionHandleReadersTmp, e.getValue().getResultSubpartitionState(), streamFactory); + } + inputChannelHandleReaders = inputChannelHandleReadersTmp; // memory barrier to allow another thread call clear() + resultSubpartitionHandleReaders = resultSubpartitionHandleReadersTmp; // memory barrier to allow another thread call clear() + } + + private void addReaders( + Map readerMap, + Collection> handles, + RefCountingFSDataInputStreamFactory streamFactory) { + for (AbstractChannelStateHandle handle : handles) { + checkState(!readerMap.containsKey(handle.getInfo()), "multiple states exist for channel: " + handle.getInfo()); + readerMap.put(handle.getInfo(), new ChannelStateStreamReader(handle, streamFactory)); + } + } + + @Override + public ReadResult readInputData(InputChannelInfo info, Buffer buffer) throws IOException { + Preconditions.checkState(!isClosed, "reader is closed"); + log.debug("readInputData, resultSubpartitionInfo: {} , buffer {}", info, buffer); + ChannelStateStreamReader reader = inputChannelHandleReaders.get(info); + return reader == null ? ReadResult.NO_MORE_DATA : reader.readInto(buffer); + } + + @Override + public ReadResult readOutputData(ResultSubpartitionInfo info, BufferBuilder bufferBuilder) throws IOException { + Preconditions.checkState(!isClosed, "reader is closed"); + log.debug("readOutputData, resultSubpartitionInfo: {} , bufferBuilder {}", info, bufferBuilder); + ChannelStateStreamReader reader = resultSubpartitionHandleReaders.get(info); + return reader == null ? ReadResult.NO_MORE_DATA : reader.readInto(bufferBuilder); + } + + @Override + public void close() throws Exception { + isClosed = true; + try (Closer closer = Closer.create()) { + for (Map map : asList(inputChannelHandleReaders, resultSubpartitionHandleReaders)) { + for (ChannelStateStreamReader reader : map.values()) { + closer.register(reader); + } + map.clear(); + } + } + } + +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateSerializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateSerializer.java new file mode 100644 index 0000000000000000000000000000000000000000..84e13d1bdacb3c5459c9336764aca0fd7fa3bed4 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateSerializer.java @@ -0,0 +1,177 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.checkpoint.channel; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.BufferBuilder; +import org.apache.flink.util.Preconditions; + +import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf; + +import javax.annotation.concurrent.NotThreadSafe; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.InputStream; + +import static java.lang.Math.addExact; +import static java.lang.Math.min; + +interface ChannelStateSerializer { + + void writeHeader(DataOutputStream dataStream) throws IOException; + + void writeData(DataOutputStream stream, Buffer... flinkBuffers) throws IOException; +} + +interface ChannelStateDeserializer { + + void readHeader(InputStream stream) throws IOException; + + int readLength(InputStream stream) throws IOException; + + int readData(InputStream stream, ChannelStateByteBuffer buffer, int bytes) throws IOException; +} + +/** + * Wrapper around various buffers to receive channel state data. + */ +@Internal +@NotThreadSafe +interface ChannelStateByteBuffer { + + boolean isWritable(); + + /** + * Read up to bytesToRead bytes into this buffer from the given {@link InputStream}. + * @return the total number of bytes read into this buffer. + */ + int writeBytes(InputStream input, int bytesToRead) throws IOException; + + static ChannelStateByteBuffer wrap(Buffer buffer) { + return new ChannelStateByteBuffer() { + + private final ByteBuf byteBuf = buffer.asByteBuf(); + + @Override + public boolean isWritable() { + return byteBuf.isWritable(); + } + + @Override + public int writeBytes(InputStream input, int bytesToRead) throws IOException { + return byteBuf.writeBytes(input, Math.min(bytesToRead, byteBuf.writableBytes())); + } + }; + } + + static ChannelStateByteBuffer wrap(BufferBuilder bufferBuilder) { + final byte[] buf = new byte[1024]; + return new ChannelStateByteBuffer() { + @Override + public boolean isWritable() { + return !bufferBuilder.isFull(); + } + + @Override + public int writeBytes(InputStream input, int bytesToRead) throws IOException { + int left = bytesToRead; + for (int toRead = getToRead(left); toRead > 0; toRead = getToRead(left)) { + int read = input.read(buf, 0, toRead); + int copied = bufferBuilder.append(java.nio.ByteBuffer.wrap(buf, 0, read)); + Preconditions.checkState(copied == read); + left -= read; + } + bufferBuilder.commit(); + return bytesToRead - left; + } + + private int getToRead(int bytesToRead) { + return min(bytesToRead, min(buf.length, bufferBuilder.getWritableBytes())); + } + }; + } + + static ChannelStateByteBuffer wrap(byte[] bytes) { + return new ChannelStateByteBuffer() { + private int written = 0; + + @Override + public boolean isWritable() { + return written < bytes.length; + } + + @Override + public int writeBytes(InputStream input, int bytesToRead) throws IOException { + final int bytesRead = input.read(bytes, written, bytes.length - written); + written += bytesRead; + return bytesRead; + } + }; + } +} + +class ChannelStateSerializerImpl implements ChannelStateSerializer, ChannelStateDeserializer { + private static final int SERIALIZATION_VERSION = 0; + + @Override + public void writeHeader(DataOutputStream dataStream) throws IOException { + dataStream.writeInt(SERIALIZATION_VERSION); + } + + @Override + public void writeData(DataOutputStream stream, Buffer... flinkBuffers) throws IOException { + stream.writeInt(getSize(flinkBuffers)); + for (Buffer buffer : flinkBuffers) { + ByteBuf nettyByteBuf = buffer.asByteBuf(); + nettyByteBuf.getBytes(nettyByteBuf.readerIndex(), stream, nettyByteBuf.readableBytes()); + } + } + + private int getSize(Buffer[] buffers) { + int len = 0; + for (Buffer buffer : buffers) { + len = addExact(len, buffer.readableBytes()); + } + return len; + } + + @Override + public void readHeader(InputStream stream) throws IOException { + int version = readInt(stream); + Preconditions.checkArgument(version == SERIALIZATION_VERSION, "unsupported version: " + version); + } + + @Override + public int readLength(InputStream stream) throws IOException { + int len = readInt(stream); + Preconditions.checkArgument(len >= 0, "negative state size"); + return len; + } + + @Override + public int readData(InputStream stream, ChannelStateByteBuffer buffer, int bytes) throws IOException { + return buffer.writeBytes(stream, bytes); + } + + private static int readInt(InputStream stream) throws IOException { + return new DataInputStream(stream).readInt(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateStreamReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateStreamReader.java new file mode 100644 index 0000000000000000000000000000000000000000..67477507f31ca58a994694e203a713ce186834a7 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateStreamReader.java @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.checkpoint.channel; + +import org.apache.flink.runtime.checkpoint.channel.ChannelStateReader.ReadResult; +import org.apache.flink.runtime.checkpoint.channel.RefCountingFSDataInputStream.RefCountingFSDataInputStreamFactory; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.BufferBuilder; +import org.apache.flink.runtime.state.AbstractChannelStateHandle; +import org.apache.flink.util.Preconditions; + +import javax.annotation.concurrent.NotThreadSafe; + +import java.io.Closeable; +import java.io.IOException; +import java.util.LinkedList; +import java.util.List; +import java.util.Queue; + +import static org.apache.flink.runtime.checkpoint.channel.ChannelStateByteBuffer.wrap; +import static org.apache.flink.runtime.checkpoint.channel.ChannelStateReader.ReadResult.HAS_MORE_DATA; +import static org.apache.flink.runtime.checkpoint.channel.ChannelStateReader.ReadResult.NO_MORE_DATA; + +/** + * Reads the state of a single channel pointed by {@link org.apache.flink.runtime.state.AbstractChannelStateHandle AbstractChannelStateHandle}. + * Once all data is read, this class can't be used anymore. + * Uses {@link RefCountingFSDataInputStream} internally. + */ +@NotThreadSafe +class ChannelStateStreamReader implements Closeable { + + private final RefCountingFSDataInputStream stream; + private final ChannelStateDeserializer serializer; + private final Queue offsets; + private int remainingBytes = -1; + private boolean closed = false; + + ChannelStateStreamReader(AbstractChannelStateHandle handle, RefCountingFSDataInputStreamFactory streamFactory) { + this(streamFactory.getOrCreate(handle), handle.getOffsets(), streamFactory.getSerializer()); + } + + private ChannelStateStreamReader(RefCountingFSDataInputStream stream, List offsets, ChannelStateDeserializer serializer) { + this.stream = stream; + this.stream.incRef(); + this.serializer = serializer; + this.offsets = new LinkedList<>(offsets); + } + + ReadResult readInto(Buffer buffer) throws IOException { + return readInto(wrap(buffer)); + } + + ReadResult readInto(BufferBuilder bufferBuilder) throws IOException { + return readInto(wrap(bufferBuilder)); + } + + private ReadResult readInto(ChannelStateByteBuffer buffer) throws IOException { + Preconditions.checkState(!closed, "reader is closed"); + readWhilePossible(buffer); + if (haveMoreData()) { + return HAS_MORE_DATA; + } else { + closed = true; + stream.decRef(); + return NO_MORE_DATA; + } + } + + private void readWhilePossible(ChannelStateByteBuffer buffer) throws IOException { + while (haveMoreData() && buffer.isWritable()) { + if (remainingBytes <= 0) { + advanceOffset(); + } + int bytesRead = serializer.readData(stream, buffer, remainingBytes); + remainingBytes -= bytesRead; + } + } + + private boolean haveMoreData() { + return remainingBytes > 0 || !offsets.isEmpty(); + } + + @SuppressWarnings("ConstantConditions") + private void advanceOffset() throws IOException { + stream.seek(offsets.poll()); + remainingBytes = serializer.readLength(stream); + } + + @Override + public void close() throws IOException { + if (!closed) { + closed = true; + stream.close(); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriteRequest.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriteRequest.java new file mode 100644 index 0000000000000000000000000000000000000000..bd6c7ba4567f718cbdd0997fac64f5c1d96ab959 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriteRequest.java @@ -0,0 +1,168 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.checkpoint.channel; + +import org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter.ChannelStateWriteResult; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.state.CheckpointStorageLocationReference; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.function.ThrowingConsumer; + +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; + +import static org.apache.flink.runtime.checkpoint.channel.CheckpointInProgressRequestState.CANCELLED; +import static org.apache.flink.runtime.checkpoint.channel.CheckpointInProgressRequestState.COMPLETED; +import static org.apache.flink.runtime.checkpoint.channel.CheckpointInProgressRequestState.EXECUTING; +import static org.apache.flink.runtime.checkpoint.channel.CheckpointInProgressRequestState.FAILED; +import static org.apache.flink.runtime.checkpoint.channel.CheckpointInProgressRequestState.NEW; +import static org.apache.flink.util.Preconditions.checkNotNull; + +interface ChannelStateWriteRequest { + long getCheckpointId(); + + void cancel(Throwable cause); + + static CheckpointInProgressRequest completeInput(long checkpointId) { + return new CheckpointInProgressRequest("completeInput", checkpointId, ChannelStateCheckpointWriter::completeInput, false); + } + + static CheckpointInProgressRequest completeOutput(long checkpointId) { + return new CheckpointInProgressRequest("completeOutput", checkpointId, ChannelStateCheckpointWriter::completeOutput, false); + } + + static ChannelStateWriteRequest write(long checkpointId, InputChannelInfo info, Buffer... flinkBuffers) { + return new CheckpointInProgressRequest("writeInput", checkpointId, writer -> writer.writeInput(info, flinkBuffers), recycle(flinkBuffers), false); + } + + static ChannelStateWriteRequest write(long checkpointId, ResultSubpartitionInfo info, Buffer... flinkBuffers) { + return new CheckpointInProgressRequest("writeOutput", checkpointId, writer -> writer.writeOutput(info, flinkBuffers), recycle(flinkBuffers), false); + } + + static ChannelStateWriteRequest start(long checkpointId, ChannelStateWriteResult targetResult, CheckpointStorageLocationReference locationReference) { + return new CheckpointStartRequest(checkpointId, targetResult, locationReference); + } + + static ChannelStateWriteRequest abort(long checkpointId, Throwable cause) { + return new CheckpointInProgressRequest("abort", checkpointId, writer -> writer.fail(cause), true); + } + + static Consumer recycle(Buffer[] flinkBuffers) { + return unused -> { + for (Buffer b : flinkBuffers) { + b.recycleBuffer(); + } + }; + } +} + +final class CheckpointStartRequest implements ChannelStateWriteRequest { + private final ChannelStateWriteResult targetResult; + private final CheckpointStorageLocationReference locationReference; + private final long checkpointId; + + CheckpointStartRequest(long checkpointId, ChannelStateWriteResult targetResult, CheckpointStorageLocationReference locationReference) { + this.checkpointId = checkpointId; + this.targetResult = checkNotNull(targetResult); + this.locationReference = checkNotNull(locationReference); + } + + @Override + public long getCheckpointId() { + return checkpointId; + } + + ChannelStateWriteResult getTargetResult() { + return targetResult; + } + + public CheckpointStorageLocationReference getLocationReference() { + return locationReference; + } + + @Override + public void cancel(Throwable cause) { + targetResult.fail(cause); + } + + @Override + public String toString() { + return "start " + checkpointId; + } +} + +enum CheckpointInProgressRequestState { + NEW, EXECUTING, COMPLETED, FAILED, CANCELLED +} + +final class CheckpointInProgressRequest implements ChannelStateWriteRequest { + private final ThrowingConsumer action; + private final Consumer discardAction; + private final long checkpointId; + private final String name; + private final boolean ignoreMissingWriter; + private final AtomicReference state = new AtomicReference<>(NEW); + + CheckpointInProgressRequest(String name, long checkpointId, ThrowingConsumer action, boolean ignoreMissingWriter) { + this(name, checkpointId, action, unused -> { + }, ignoreMissingWriter); + } + + CheckpointInProgressRequest(String name, long checkpointId, ThrowingConsumer action, Consumer discardAction, boolean ignoreMissingWriter) { + this.checkpointId = checkpointId; + this.action = checkNotNull(action); + this.discardAction = checkNotNull(discardAction); + this.name = checkNotNull(name); + this.ignoreMissingWriter = ignoreMissingWriter; + } + + @Override + public long getCheckpointId() { + return checkpointId; + } + + @Override + public void cancel(Throwable cause) { + if (state.compareAndSet(NEW, CANCELLED) || state.compareAndSet(FAILED, CANCELLED)) { + discardAction.accept(cause); + } + } + + void execute(ChannelStateCheckpointWriter channelStateCheckpointWriter) throws Exception { + Preconditions.checkState(state.compareAndSet(NEW, EXECUTING)); + try { + action.accept(channelStateCheckpointWriter); + state.set(COMPLETED); + } catch (Exception e) { + state.set(FAILED); + throw e; + } + } + + void onWriterMissing() { + if (!ignoreMissingWriter) { + throw new IllegalArgumentException("writer not found while processing request: " + toString()); + } + } + + @Override + public String toString() { + return name + " " + checkpointId; + } + +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriteRequestDispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriteRequestDispatcher.java new file mode 100644 index 0000000000000000000000000000000000000000..461acc40d4067cef6e18a5bb9ae9169009c40d0f --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriteRequestDispatcher.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.checkpoint.channel; + +interface ChannelStateWriteRequestDispatcher { + + void dispatch(ChannelStateWriteRequest request) throws Exception; + + void fail(Throwable cause); + + ChannelStateWriteRequestDispatcher NO_OP = new ChannelStateWriteRequestDispatcher() { + @Override + public void dispatch(ChannelStateWriteRequest request) { + } + + @Override + public void fail(Throwable cause) { + } + }; +} + diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriteRequestDispatcherImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriteRequestDispatcherImpl.java new file mode 100644 index 0000000000000000000000000000000000000000..843663e0c0c3d8656196bcb54ab7d84ac5bcbc94 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriteRequestDispatcherImpl.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.checkpoint.channel; + +import org.apache.flink.runtime.state.CheckpointStorageWorkerView; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.Map; + +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; + +/** + * Maintains a set of {@link ChannelStateCheckpointWriter writers} per checkpoint and translates incoming + * {@link ChannelStateWriteRequest requests} to their corresponding methods. + */ +final class ChannelStateWriteRequestDispatcherImpl implements ChannelStateWriteRequestDispatcher { + private static final Logger LOG = LoggerFactory.getLogger(ChannelStateWriteRequestDispatcherImpl.class); + + private final Map writers; // limited indirectly by results max size + private final CheckpointStorageWorkerView streamFactoryResolver; + private final ChannelStateSerializer serializer; + + ChannelStateWriteRequestDispatcherImpl(CheckpointStorageWorkerView streamFactoryResolver, ChannelStateSerializer serializer) { + this.writers = new HashMap<>(); + this.streamFactoryResolver = checkNotNull(streamFactoryResolver); + this.serializer = checkNotNull(serializer); + } + + @Override + public void dispatch(ChannelStateWriteRequest request) throws Exception { + LOG.debug("process {}", request); + try { + dispatchInternal(request); + } catch (Exception e) { + request.cancel(e); + throw e; + } + } + + private void dispatchInternal(ChannelStateWriteRequest request) throws Exception { + if (request instanceof CheckpointStartRequest) { + checkState(!writers.containsKey(request.getCheckpointId()), "writer not found for request " + request); + writers.put(request.getCheckpointId(), buildWriter((CheckpointStartRequest) request)); + } else if (request instanceof CheckpointInProgressRequest) { + ChannelStateCheckpointWriter writer = writers.get(request.getCheckpointId()); + CheckpointInProgressRequest req = (CheckpointInProgressRequest) request; + if (writer == null) { + req.onWriterMissing(); + } else { + req.execute(writer); + } + } else { + throw new IllegalArgumentException("unknown request type: " + request); + } + } + + private ChannelStateCheckpointWriter buildWriter(CheckpointStartRequest request) throws Exception { + return new ChannelStateCheckpointWriter( + request, + streamFactoryResolver.resolveCheckpointStorageLocation(request.getCheckpointId(), request.getLocationReference()), + serializer, + () -> writers.remove(request.getCheckpointId())); + } + + @Override + public void fail(Throwable cause) { + for (ChannelStateCheckpointWriter writer : writers.values()) { + try { + writer.fail(cause); + } catch (Exception ex) { + LOG.warn("unable to fail write channel state writer", cause); + } + } + writers.clear(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriteRequestExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriteRequestExecutor.java new file mode 100644 index 0000000000000000000000000000000000000000..e90876693d7197196e849c3707bdfd0f5f4f04ef --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriteRequestExecutor.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.checkpoint.channel; + +import java.io.Closeable; + +/** + * Executes {@link ChannelStateWriteRequest}s potentially asynchronously. An exception thrown during the execution + * should be re-thrown on any next call. + */ +interface ChannelStateWriteRequestExecutor extends Closeable { + + /** + * @throws IllegalStateException if called more than once or after {@link #close()} + */ + void start() throws IllegalStateException; + + /** + * Send {@link ChannelStateWriteRequest} to this worker. If this method throws an exception then client must + * {@link ChannelStateWriteRequest#cancel cancel} it. + * @throws IllegalStateException if worker is not running + * @throws Exception if any exception occurred during processing this or other items previously + */ + void submit(ChannelStateWriteRequest r) throws Exception; + + /** + * Send {@link ChannelStateWriteRequest} to this worker to be processed first. If this method throws an exception then client must + * {@link ChannelStateWriteRequest#cancel cancel} it. + * @throws IllegalStateException if worker is not running + * @throws Exception if any exception occurred during processing this or other items previously + */ + void submitPriority(ChannelStateWriteRequest r) throws Exception; + +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriteRequestExecutorImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriteRequestExecutorImpl.java new file mode 100644 index 0000000000000000000000000000000000000000..cbcc3f78b89e30e74dac21405dd6c1a4e085cafb --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriteRequestExecutorImpl.java @@ -0,0 +1,157 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.checkpoint.channel; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.function.RunnableWithException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.concurrent.ThreadSafe; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.BlockingDeque; +import java.util.concurrent.CancellationException; +import java.util.concurrent.LinkedBlockingDeque; + +/** + * Executes {@link ChannelStateWriteRequest}s in a separate thread. Any exception occurred during execution causes this + * thread to stop and the exception to be re-thrown on any subsequent call. + */ +@ThreadSafe +class ChannelStateWriteRequestExecutorImpl implements ChannelStateWriteRequestExecutor { + + private static final Logger LOG = LoggerFactory.getLogger(ChannelStateWriteRequestExecutorImpl.class); + private static final int DEFAULT_HANDOVER_CAPACITY = 10_000; + + private final ChannelStateWriteRequestDispatcher dispatcher; + private final BlockingDeque deque; + private final Thread thread; + private volatile Exception thrown = null; + private volatile boolean wasClosed = false; + + ChannelStateWriteRequestExecutorImpl(ChannelStateWriteRequestDispatcher dispatcher) { + this(dispatcher, new LinkedBlockingDeque<>(DEFAULT_HANDOVER_CAPACITY)); + } + + ChannelStateWriteRequestExecutorImpl(ChannelStateWriteRequestDispatcher dispatcher, BlockingDeque deque) { + this.dispatcher = dispatcher; + this.deque = deque; + this.thread = new Thread(this::run); + this.thread.setDaemon(true); + } + + @VisibleForTesting + void run() { + try { + loop(); + } catch (Exception ex) { + thrown = ex; + } finally { + cleanupRequests(); + dispatcher.fail(thrown == null ? new CancellationException() : thrown); + } + LOG.debug("loop terminated"); + } + + private void loop() throws Exception { + while (!wasClosed) { + try { + dispatcher.dispatch(deque.take()); + } catch (InterruptedException e) { + if (!wasClosed) { + LOG.debug("interrupted while waiting for a request (continue waiting)", e); + } else { + Thread.currentThread().interrupt(); + } + } + } + } + + private void cleanupRequests() { + Throwable cause = thrown == null ? new CancellationException() : thrown; + List drained = new ArrayList<>(); + deque.drainTo(drained); + LOG.info("discarding {} drained requests", drained.size()); + for (ChannelStateWriteRequest request : drained) { + request.cancel(cause); + } + } + + @Override + public void start() throws IllegalStateException { + this.thread.start(); + } + + @Override + public void submit(ChannelStateWriteRequest request) throws Exception { + submitInternal(request, () -> deque.add(request)); + } + + @Override + public void submitPriority(ChannelStateWriteRequest request) throws Exception { + submitInternal(request, () -> deque.addFirst(request)); + } + + private void submitInternal(ChannelStateWriteRequest request, RunnableWithException action) throws Exception { + try { + action.run(); + } catch (Exception ex) { + request.cancel(ex); + throw ex; + } + ensureRunning(); + } + + private void ensureRunning() throws Exception { + // this check should be performed *at least after* enqueuing a request + // checking before is not enough because (check + enqueue) is not atomic + if (wasClosed || !thread.isAlive()) { + cleanupRequests(); + throw ExceptionUtils.firstOrSuppressed(new IllegalStateException("not running"), thrown); + } + } + + @Override + public void close() throws IOException { + wasClosed = true; + while (thread.isAlive()) { + thread.interrupt(); + try { + thread.join(); + } catch (InterruptedException e) { + if (!thread.isAlive()) { + Thread.currentThread().interrupt(); + } + LOG.debug("interrupted while waiting for the writer thread to die", e); + } + } + if (thrown != null) { + throw new IOException(thrown); + } + } + + @VisibleForTesting + Thread getThread() { + return thread; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriter.java new file mode 100644 index 0000000000000000000000000000000000000000..829208994ec44b7fbbe059a907213a811ffde271 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriter.java @@ -0,0 +1,189 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.checkpoint.channel; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.checkpoint.CheckpointOptions; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.state.CheckpointListener; +import org.apache.flink.runtime.state.InputChannelStateHandle; +import org.apache.flink.runtime.state.ResultSubpartitionStateHandle; + +import java.io.Closeable; +import java.util.Collection; +import java.util.Collections; +import java.util.concurrent.CompletableFuture; + +/** + * Writes channel state during checkpoint/savepoint. + */ +@Internal +public interface ChannelStateWriter extends Closeable, CheckpointListener { + + /** + * Channel state write result. + */ + class ChannelStateWriteResult { + final CompletableFuture> inputChannelStateHandles; + final CompletableFuture> resultSubpartitionStateHandles; + + ChannelStateWriteResult() { + this(new CompletableFuture<>(), new CompletableFuture<>()); + } + + ChannelStateWriteResult( + CompletableFuture> inputChannelStateHandles, + CompletableFuture> resultSubpartitionStateHandles) { + this.inputChannelStateHandles = inputChannelStateHandles; + this.resultSubpartitionStateHandles = resultSubpartitionStateHandles; + } + + public CompletableFuture> getInputChannelStateHandles() { + return inputChannelStateHandles; + } + + public CompletableFuture> getResultSubpartitionStateHandles() { + return resultSubpartitionStateHandles; + } + + public static final ChannelStateWriteResult EMPTY = new ChannelStateWriteResult( + CompletableFuture.completedFuture(Collections.emptyList()), + CompletableFuture.completedFuture(Collections.emptyList()) + ); + + public void fail(Throwable e) { + inputChannelStateHandles.completeExceptionally(e); + resultSubpartitionStateHandles.completeExceptionally(e); + } + + boolean isDone() { + return inputChannelStateHandles.isDone() && resultSubpartitionStateHandles.isDone(); + } + } + + /** + * Sequence number for the buffers that were saved during the previous execution attempt; then restored; and now are + * to be saved again (as opposed to the buffers received from the upstream or from the operator). + */ + int SEQUENCE_NUMBER_RESTORED = -1; + + /** + * Signifies that buffer sequence number is unknown (e.g. if passing sequence numbers is not implemented). + */ + int SEQUENCE_NUMBER_UNKNOWN = -2; + + /** + * Initiate write of channel state for the given checkpoint id. + */ + void start(long checkpointId, CheckpointOptions checkpointOptions); + + /** + * Add in-flight buffers from the {@link org.apache.flink.runtime.io.network.partition.consumer.InputChannel InputChannel}. + * Must be called after {@link #start} (long)} and before {@link #finishInput(long)}. + * Buffers are recycled after they are written or exception occurs. + * @param startSeqNum sequence number of the 1st passed buffer. + * It is intended to use for incremental snapshots. + * If no data is passed it is ignored. + * @param data zero or more data buffers ordered by their sequence numbers + * @throws IllegalArgumentException if one or more passed buffers {@link Buffer#isBuffer() isn't a buffer} + * @see org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter#SEQUENCE_NUMBER_RESTORED + * @see org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter#SEQUENCE_NUMBER_UNKNOWN + */ + void addInputData(long checkpointId, InputChannelInfo info, int startSeqNum, Buffer... data) throws IllegalArgumentException; + + /** + * Add in-flight buffers from the {@link org.apache.flink.runtime.io.network.partition.ResultSubpartition ResultSubpartition}. + * Must be called after {@link #start} and before {@link #finishOutput(long)}. + * Buffers are recycled after they are written or exception occurs. + * @param startSeqNum sequence number of the 1st passed buffer. + * It is intended to use for incremental snapshots. + * If no data is passed it is ignored. + * @param data zero or more data buffers ordered by their sequence numbers + * @throws IllegalArgumentException if one or more passed buffers {@link Buffer#isBuffer() isn't a buffer} + * @see org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter#SEQUENCE_NUMBER_RESTORED + * @see org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter#SEQUENCE_NUMBER_UNKNOWN + */ + void addOutputData(long checkpointId, ResultSubpartitionInfo info, int startSeqNum, Buffer... data) throws IllegalArgumentException; + + /** + * Finalize write of channel state data for the given checkpoint id. + * Must be called after {@link #start(long, CheckpointOptions)} and all of the input data of the given checkpoint added. + * When both {@link #finishInput} and {@link #finishOutput} were called the results can be (eventually) obtained + * using {@link #getWriteResult} + */ + void finishInput(long checkpointId); + + /** + * Finalize write of channel state data for the given checkpoint id. + * Must be called after {@link #start(long, CheckpointOptions)} and all of the output data of the given checkpoint added. + * When both {@link #finishInput} and {@link #finishOutput} were called the results can be (eventually) obtained + * using {@link #getWriteResult} + */ + void finishOutput(long checkpointId); + + /** + * Aborts the checkpoint and fails pending result for this checkpoint. + */ + void abort(long checkpointId, Throwable cause); + + /** + * Must be called after {@link #start(long, CheckpointOptions)}. + */ + ChannelStateWriteResult getWriteResult(long checkpointId); + + ChannelStateWriter NO_OP = new ChannelStateWriter() { + @Override + public void start(long checkpointId, CheckpointOptions checkpointOptions) { + } + + @Override + public void addInputData(long checkpointId, InputChannelInfo info, int startSeqNum, Buffer... data) { + } + + @Override + public void addOutputData(long checkpointId, ResultSubpartitionInfo info, int startSeqNum, Buffer... data) { + } + + @Override + public void finishInput(long checkpointId) { + } + + @Override + public void finishOutput(long checkpointId) { + } + + @Override + public void abort(long checkpointId, Throwable cause) { + } + + @Override + public ChannelStateWriteResult getWriteResult(long checkpointId) { + return new ChannelStateWriteResult( + CompletableFuture.completedFuture(Collections.emptyList()), + CompletableFuture.completedFuture(Collections.emptyList())); + } + + @Override + public void close() { + } + + @Override + public void notifyCheckpointComplete(long checkpointId) { + } + }; +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriterImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriterImpl.java new file mode 100644 index 0000000000000000000000000000000000000000..bb1d7114165d9952f2f590ca697df2779e13a805 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriterImpl.java @@ -0,0 +1,192 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.checkpoint.channel; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.checkpoint.CheckpointOptions; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.state.CheckpointStorageWorkerView; +import org.apache.flink.runtime.state.CheckpointStreamFactory; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.concurrent.ThreadSafe; + +import java.io.IOException; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +import static org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.completeInput; +import static org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.completeOutput; +import static org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.write; + +/** + * {@link ChannelStateWriter} implemented using + * {@link CheckpointStreamFactory.CheckpointStateOutputStream CheckpointStateOutputStreams}. Internally, it has by default + *
    + *
  • one stream per checkpoint; having multiple streams would mean more files written and more connections opened + * (and more latency on restore)
  • + *
  • one thread; having multiple threads means more connections, couples with the implementation and increases complexity
  • + *
+ * Thread-safety: this class is thread-safe when used with a thread-safe {@link ChannelStateWriteRequestExecutor executor} + * (e.g. default {@link ChannelStateWriteRequestExecutorImpl}. + */ +@Internal +@ThreadSafe +public class ChannelStateWriterImpl implements ChannelStateWriter { + + private static final Logger LOG = LoggerFactory.getLogger(ChannelStateWriterImpl.class); + private static final int DEFAULT_MAX_CHECKPOINTS = 5; // currently, only single in-flight checkpoint is supported + + private final ChannelStateWriteRequestExecutor executor; + private final ConcurrentMap results; + private final int maxCheckpoints; + + /** + * Creates a {@link ChannelStateWriterImpl} with {@link #DEFAULT_MAX_CHECKPOINTS} as {@link #maxCheckpoints}. + */ + public ChannelStateWriterImpl(CheckpointStorageWorkerView streamFactoryResolver) { + this(streamFactoryResolver, DEFAULT_MAX_CHECKPOINTS); + } + + /** + * Creates a {@link ChannelStateWriterImpl} with {@link ChannelStateSerializerImpl default} {@link ChannelStateSerializer}, + * and a {@link ChannelStateWriteRequestExecutorImpl}. + * + * @param maxCheckpoints maximum number of checkpoints to be written currently or finished but not taken yet. + * @param streamFactoryResolver a factory to obtain output stream factory for a given checkpoint + */ + ChannelStateWriterImpl(CheckpointStorageWorkerView streamFactoryResolver, int maxCheckpoints) { + this( + new ConcurrentHashMap<>(maxCheckpoints), + new ChannelStateWriteRequestExecutorImpl(new ChannelStateWriteRequestDispatcherImpl(streamFactoryResolver, new ChannelStateSerializerImpl())), + maxCheckpoints + ); + } + + ChannelStateWriterImpl(ConcurrentMap results, ChannelStateWriteRequestExecutor executor, int maxCheckpoints) { + this.results = results; + this.maxCheckpoints = maxCheckpoints; + this.executor = executor; + } + + @Override + public void start(long checkpointId, CheckpointOptions checkpointOptions) { + LOG.debug("start checkpoint {} ({})", checkpointId, checkpointOptions); + ChannelStateWriteResult result = new ChannelStateWriteResult(); + ChannelStateWriteResult put = results.computeIfAbsent(checkpointId, id -> { + Preconditions.checkState(results.size() < maxCheckpoints, "results.size() > maxCheckpoints", results.size(), maxCheckpoints); + enqueue(new CheckpointStartRequest(checkpointId, result, checkpointOptions.getTargetLocation()), false); + return result; + }); + Preconditions.checkArgument(put == result, "result future already present for checkpoint id: " + checkpointId); + } + + @Override + public void addInputData(long checkpointId, InputChannelInfo info, int startSeqNum, Buffer... data) { + LOG.debug("add input data, checkpoint id: {}, channel: {}, startSeqNum: {}, num buffers: {}", + checkpointId, info, startSeqNum, data == null ? 0 : data.length); + enqueue(write(checkpointId, info, checkBufferType(data)), false); + } + + @Override + public void addOutputData(long checkpointId, ResultSubpartitionInfo info, int startSeqNum, Buffer... data) { + LOG.debug("add output data, checkpoint id: {}, channel: {}, startSeqNum: {}, num buffers: {}", + checkpointId, info, startSeqNum, data == null ? 0 : data.length); + enqueue(write(checkpointId, info, checkBufferType(data)), false); + } + + @Override + public void finishInput(long checkpointId) { + LOG.debug("finish input data, checkpoint id: {}", checkpointId); + enqueue(completeInput(checkpointId), false); + } + + @Override + public void finishOutput(long checkpointId) { + LOG.debug("finish output data, checkpoint id: {}", checkpointId); + enqueue(completeOutput(checkpointId), false); + } + + @Override + public void abort(long checkpointId, Throwable cause) { + LOG.debug("abort, checkpoint id: {}", checkpointId); + enqueue(ChannelStateWriteRequest.abort(checkpointId, cause), true); // abort already started + enqueue(ChannelStateWriteRequest.abort(checkpointId, cause), false); // abort enqueued but not started + results.remove(checkpointId); + } + + @Override + public ChannelStateWriteResult getWriteResult(long checkpointId) { + LOG.debug("requested write result, checkpoint id: {}", checkpointId); + ChannelStateWriteResult result = results.get(checkpointId); + Preconditions.checkArgument(result != null, "channel state write result not found for checkpoint id " + checkpointId); + return result; + } + + @Override + public void notifyCheckpointComplete(long checkpointId) { + results.remove(checkpointId); + } + + public void open() { + executor.start(); + } + + @Override + public void close() throws IOException { + results.clear(); + executor.close(); + } + + private void enqueue(ChannelStateWriteRequest request, boolean atTheFront) { + // state check and previous errors check are performed inside the worker + try { + if (atTheFront) { + executor.submitPriority(request); + } else { + executor.submit(request); + } + } catch (Exception e) { + request.cancel(e); + throw new RuntimeException("unable to send request to worker", e); + } + } + + private static Buffer[] checkBufferType(Buffer... data) { + if (data == null) { + return new Buffer[0]; + } + try { + for (Buffer buffer : data) { + Preconditions.checkArgument(buffer.isBuffer()); + } + } catch (Exception e) { + for (Buffer buffer : data) { + if (buffer.isBuffer()) { + buffer.recycleBuffer(); + } + } + throw e; + } + return data; + } + +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/InputChannelInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/InputChannelInfo.java new file mode 100644 index 0000000000000000000000000000000000000000..01313c7e3b5196a0f41fe1a95fcfc1e86fc77d41 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/InputChannelInfo.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.checkpoint.channel; + +import org.apache.flink.annotation.Internal; + +import java.io.Serializable; +import java.util.Objects; + +/** + * Identifies {@link org.apache.flink.runtime.io.network.partition.consumer.InputChannel} in a given subtask. + * Note that {@link org.apache.flink.runtime.io.network.partition.consumer.InputChannelID InputChannelID} + * can not be used because it is generated randomly. + */ +@Internal +public class InputChannelInfo implements Serializable { + private static final long serialVersionUID = 1L; + + private final int gateIdx; + private final int inputChannelIdx; + + public InputChannelInfo(int gateIdx, int inputChannelIdx) { + this.gateIdx = gateIdx; + this.inputChannelIdx = inputChannelIdx; + } + + public int getGateIdx() { + return gateIdx; + } + + public int getInputChannelIdx() { + return inputChannelIdx; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final InputChannelInfo that = (InputChannelInfo) o; + return gateIdx == that.gateIdx && inputChannelIdx == that.inputChannelIdx; + } + + @Override + public int hashCode() { + return Objects.hash(gateIdx, inputChannelIdx); + } + + @Override + public String toString() { + return "InputChannelInfo{" + "gateIdx=" + gateIdx + ", inputChannelIdx=" + inputChannelIdx + '}'; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/RefCountingFSDataInputStream.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/RefCountingFSDataInputStream.java new file mode 100644 index 0000000000000000000000000000000000000000..86181a29be97f88b63a5fe6f467954a657f60d29 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/RefCountingFSDataInputStream.java @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.checkpoint.channel; + +import org.apache.flink.core.fs.FSDataInputStream; +import org.apache.flink.runtime.state.AbstractChannelStateHandle; +import org.apache.flink.runtime.state.StreamStateHandle; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.function.SupplierWithException; + +import javax.annotation.concurrent.NotThreadSafe; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; + +@NotThreadSafe +class RefCountingFSDataInputStream extends FSDataInputStream { + + private enum State {NEW, OPENED, CLOSED} + + private final SupplierWithException streamSupplier; + private FSDataInputStream stream; + private final ChannelStateDeserializer serializer; + private int refCount = 0; + private State state = State.NEW; + + private RefCountingFSDataInputStream( + SupplierWithException streamSupplier, + ChannelStateDeserializer serializer) { + this.streamSupplier = checkNotNull(streamSupplier); + this.serializer = checkNotNull(serializer); + } + + void incRef() { + checkNotClosed(); + refCount++; + } + + void decRef() throws IOException { + checkNotClosed(); + refCount--; + if (refCount == 0) { + close(); + } + } + + @Override + public int read() throws IOException { + ensureOpen(); + return stream.read(); + } + + @Override + public void seek(long pos) throws IOException { + ensureOpen(); + stream.seek(pos); + } + + @Override + public long getPos() throws IOException { + ensureOpen(); + return stream.getPos(); + } + + public void close() throws IOException { + state = State.CLOSED; + if (stream != null) { + stream.close(); + stream = null; + } + } + + private void ensureOpen() throws IOException { + checkNotClosed(); + if (state == State.NEW) { + stream = Preconditions.checkNotNull(streamSupplier.get()); + serializer.readHeader(stream); + state = State.OPENED; + } + } + + private void checkNotClosed() { + checkState(state != State.CLOSED, "stream is closed"); + } + + @NotThreadSafe + static class RefCountingFSDataInputStreamFactory { + private final Map streams = new HashMap<>(); // not clearing: expecting short life + private final ChannelStateDeserializer serializer; + + RefCountingFSDataInputStreamFactory(ChannelStateDeserializer serializer) { + this.serializer = checkNotNull(serializer); + } + + RefCountingFSDataInputStream getOrCreate(AbstractChannelStateHandle handle) { + StreamStateHandle streamStateHandle = handle.getDelegate(); + RefCountingFSDataInputStream stream = streams.get(streamStateHandle); + if (stream == null) { + stream = new RefCountingFSDataInputStream(streamStateHandle::openInputStream, serializer); + streams.put(streamStateHandle, stream); + } + return stream; + } + + ChannelStateDeserializer getSerializer() { + return serializer; + } + } + +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ResultSubpartitionInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ResultSubpartitionInfo.java new file mode 100644 index 0000000000000000000000000000000000000000..036e987967ebbd9b64be215d343a8f25362f9431 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ResultSubpartitionInfo.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.checkpoint.channel; + +import org.apache.flink.annotation.Internal; + +import java.io.Serializable; +import java.util.Objects; + +/** + * Identifies {@link org.apache.flink.runtime.io.network.partition.ResultSubpartition ResultSubpartition} in a given subtask. + * Note that {@link org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID IntermediateResultPartitionID} + * can not be used because it: a) identifies the whole + * {@link org.apache.flink.runtime.io.network.partition.ResultPartition ResultPartition} b) is generated randomly. + */ +@Internal +public class ResultSubpartitionInfo implements Serializable { + private static final long serialVersionUID = 1L; + + private final int partitionIdx; + private final int subPartitionIdx; + + public ResultSubpartitionInfo(int partitionIdx, int subPartitionIdx) { + this.partitionIdx = partitionIdx; + this.subPartitionIdx = subPartitionIdx; + } + + public int getPartitionIdx() { + return partitionIdx; + } + + public int getSubPartitionIdx() { + return subPartitionIdx; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final ResultSubpartitionInfo that = (ResultSubpartitionInfo) o; + return partitionIdx == that.partitionIdx && subPartitionIdx == that.subPartitionIdx; + } + + @Override + public int hashCode() { + return Objects.hash(partitionIdx, subPartitionIdx); + } + + @Override + public String toString() { + return "ResultSubpartitionInfo{" + "partitionIdx=" + partitionIdx + ", subPartitionIdx=" + subPartitionIdx + '}'; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/ChannelStateHandleSerializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/ChannelStateHandleSerializer.java new file mode 100644 index 0000000000000000000000000000000000000000..d69241ee8564b4d65af6679cb16ac42e699b6a9f --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/ChannelStateHandleSerializer.java @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.checkpoint.metadata; + +import org.apache.flink.runtime.checkpoint.channel.InputChannelInfo; +import org.apache.flink.runtime.checkpoint.channel.ResultSubpartitionInfo; +import org.apache.flink.runtime.state.AbstractChannelStateHandle; +import org.apache.flink.runtime.state.InputChannelStateHandle; +import org.apache.flink.runtime.state.ResultSubpartitionStateHandle; +import org.apache.flink.runtime.state.StreamStateHandle; +import org.apache.flink.util.function.BiConsumerWithException; +import org.apache.flink.util.function.FunctionWithException; +import org.apache.flink.util.function.TriFunctionWithException; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import static org.apache.flink.runtime.checkpoint.metadata.MetadataV2V3SerializerBase.deserializeStreamStateHandle; +import static org.apache.flink.runtime.checkpoint.metadata.MetadataV2V3SerializerBase.serializeStreamStateHandle; + +class ChannelStateHandleSerializer { + + public void serialize(ResultSubpartitionStateHandle handle, DataOutputStream dataOutputStream) throws IOException { + serializeChannelStateHandle(handle, dataOutputStream, (info, out) -> { + out.writeInt(info.getPartitionIdx()); + out.writeInt(info.getSubPartitionIdx()); + }); + } + + ResultSubpartitionStateHandle deserializeResultSubpartitionStateHandle(DataInputStream dis) throws IOException { + return deserializeChannelStateHandle( + is -> new ResultSubpartitionInfo(is.readInt(), is.readInt()), + (streamStateHandle, longs, info) -> new ResultSubpartitionStateHandle(info, streamStateHandle, longs), + dis); + } + + public void serialize(InputChannelStateHandle handle, DataOutputStream dos) throws IOException { + serializeChannelStateHandle(handle, dos, (info, dataOutputStream) -> { + dos.writeInt(info.getGateIdx()); + dos.writeInt(info.getInputChannelIdx()); + }); + } + + InputChannelStateHandle deserializeInputChannelStateHandle(DataInputStream dis) throws IOException { + return deserializeChannelStateHandle( + is -> new InputChannelInfo(is.readInt(), is.readInt()), + (streamStateHandle, longs, inputChannelInfo) -> new InputChannelStateHandle(inputChannelInfo, streamStateHandle, longs), + dis); + } + + private static void serializeChannelStateHandle( + AbstractChannelStateHandle handle, + DataOutputStream dos, + BiConsumerWithException infoWriter) throws IOException { + infoWriter.accept(handle.getInfo(), dos); + dos.writeInt(handle.getOffsets().size()); + for (long offset : handle.getOffsets()) { + dos.writeLong(offset); + } + serializeStreamStateHandle(handle.getDelegate(), dos); + } + + private static > Handle deserializeChannelStateHandle( + FunctionWithException infoReader, + TriFunctionWithException, Info, Handle, IOException> handleBuilder, + DataInputStream dis) throws IOException { + final Info info = infoReader.apply(dis); + int offsetsSize = dis.readInt(); + final List offsets = new ArrayList<>(offsetsSize); + for (int i = 0; i < offsetsSize; i++) { + offsets.add(dis.readLong()); + } + return handleBuilder.apply(deserializeStreamStateHandle(dis), offsets, info); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataSerializers.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataSerializers.java index ba28b7f6ca38a11adafd0354382f27212b18b6f2..b8863bd2b724634cd1a8d38df8234a27a1d0138f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataSerializers.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataSerializers.java @@ -28,19 +28,18 @@ import java.util.Map; */ public class MetadataSerializers { - private static final Map SERIALIZERS = new HashMap<>(2); + private static final Map SERIALIZERS = new HashMap<>(3); static { - SERIALIZERS.put(MetadataV1Serializer.VERSION, MetadataV1Serializer.INSTANCE); - SERIALIZERS.put(MetadataV2Serializer.VERSION, MetadataV2Serializer.INSTANCE); + registerSerializer(MetadataV1Serializer.INSTANCE); + registerSerializer(MetadataV2Serializer.INSTANCE); + registerSerializer(MetadataV3Serializer.INSTANCE); } - private MetadataSerializers() { - throw new AssertionError(); + private static void registerSerializer(MetadataSerializer serializer) { + SERIALIZERS.put(serializer.getVersion(), serializer); } - // ------------------------------------------------------------------------ - /** * Returns the {@link MetadataSerializer} for the given savepoint version. * @@ -56,4 +55,9 @@ public class MetadataSerializers { throw new IllegalArgumentException("Unrecognized checkpoint version number: " + version); } } + + // ------------------------------------------------------------------------ + + /** Utility method class, not meant to be instantiated. */ + private MetadataSerializers() {} } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2Serializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2Serializer.java index c2ac522065b3ba10544345f5f9521cb8ba934dcc..bbf25d89061176f75cf6978b1e33fcc2c0acaee3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2Serializer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2Serializer.java @@ -19,76 +19,30 @@ package org.apache.flink.runtime.checkpoint.metadata; import org.apache.flink.annotation.Internal; -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.core.fs.Path; -import org.apache.flink.runtime.checkpoint.MasterState; import org.apache.flink.runtime.checkpoint.OperatorState; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; import org.apache.flink.runtime.jobgraph.OperatorID; -import org.apache.flink.runtime.state.IncrementalRemoteKeyedStateHandle; -import org.apache.flink.runtime.state.KeyGroupRange; -import org.apache.flink.runtime.state.KeyGroupRangeOffsets; -import org.apache.flink.runtime.state.KeyGroupsStateHandle; -import org.apache.flink.runtime.state.KeyedStateHandle; -import org.apache.flink.runtime.state.OperatorStateHandle; -import org.apache.flink.runtime.state.OperatorStreamStateHandle; -import org.apache.flink.runtime.state.StateHandleID; -import org.apache.flink.runtime.state.StreamStateHandle; -import org.apache.flink.runtime.state.filesystem.FileStateHandle; -import org.apache.flink.runtime.state.memory.ByteStreamStateHandle; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; import java.util.Map; -import java.util.UUID; /** * (De)serializer for checkpoint metadata format version 2. + * This format was introduced with Apache Flink 1.3.0. * - *

This format version adds - * - *

Basic checkpoint metadata layout: - *

- *  +--------------+---------------+-----------------+
- *  | checkpointID | master states | operator states |
- *  +--------------+---------------+-----------------+
- *
- *  Master state:
- *  +--------------+---------------------+---------+------+---------------+
- *  | magic number | num remaining bytes | version | name | payload bytes |
- *  +--------------+---------------------+---------+------+---------------+
- * 
+ *

See {@link MetadataV2V3SerializerBase} for a description of the format layout. */ @Internal -public class MetadataV2Serializer implements MetadataSerializer { +public class MetadataV2Serializer extends MetadataV2V3SerializerBase implements MetadataSerializer{ /** The metadata format version. */ public static final int VERSION = 2; - /** Random magic number for consistency checks. */ - private static final int MASTER_STATE_MAGIC_NUMBER = 0xc96b1696; - - private static final byte NULL_HANDLE = 0; - private static final byte BYTE_STREAM_STATE_HANDLE = 1; - private static final byte FILE_STREAM_STATE_HANDLE = 2; - private static final byte KEY_GROUPS_HANDLE = 3; - private static final byte PARTITIONABLE_OPERATOR_STATE_HANDLE = 4; - private static final byte INCREMENTAL_KEY_GROUPS_HANDLE = 5; - /** The singleton instance of the serializer. */ public static final MetadataV2Serializer INSTANCE = new MetadataV2Serializer(); - // ------------------------------------------------------------------------ - /** Singleton, not meant to be instantiated. */ private MetadataV2Serializer() {} @@ -98,448 +52,92 @@ public class MetadataV2Serializer implements MetadataSerializer { } // ------------------------------------------------------------------------ - // (De)serialization entry points + // Deserialization entry point // ------------------------------------------------------------------------ - public static void serialize(CheckpointMetadata checkpointMetadata, DataOutputStream dos) throws IOException { - // first: checkpoint ID - dos.writeLong(checkpointMetadata.getCheckpointId()); - - // second: master state - final Collection masterStates = checkpointMetadata.getMasterStates(); - dos.writeInt(masterStates.size()); - for (MasterState ms : masterStates) { - serializeMasterState(ms, dos); - } - - // third: operator states - Collection operatorStates = checkpointMetadata.getOperatorStates(); - dos.writeInt(operatorStates.size()); - - for (OperatorState operatorState : operatorStates) { - // Operator ID - dos.writeLong(operatorState.getOperatorID().getLowerPart()); - dos.writeLong(operatorState.getOperatorID().getUpperPart()); - - // Parallelism - int parallelism = operatorState.getParallelism(); - dos.writeInt(parallelism); - dos.writeInt(operatorState.getMaxParallelism()); - dos.writeInt(1); - - // Sub task states - Map subtaskStateMap = operatorState.getSubtaskStates(); - dos.writeInt(subtaskStateMap.size()); - for (Map.Entry entry : subtaskStateMap.entrySet()) { - dos.writeInt(entry.getKey()); - serializeSubtaskState(entry.getValue(), dos); - } - } - } - @Override - public CheckpointMetadata deserialize(DataInputStream dis, ClassLoader cl) throws IOException { - // first: checkpoint ID - final long checkpointId = dis.readLong(); - if (checkpointId < 0) { - throw new IOException("invalid checkpoint ID: " + checkpointId); - } - - // second: master state - final List masterStates; - final int numMasterStates = dis.readInt(); - - if (numMasterStates == 0) { - masterStates = Collections.emptyList(); - } - else if (numMasterStates > 0) { - masterStates = new ArrayList<>(numMasterStates); - for (int i = 0; i < numMasterStates; i++) { - masterStates.add(deserializeMasterState(dis)); - } - } - else { - throw new IOException("invalid number of master states: " + numMasterStates); - } - - // third: operator states - int numTaskStates = dis.readInt(); - List operatorStates = new ArrayList<>(numTaskStates); - - for (int i = 0; i < numTaskStates; i++) { - OperatorID jobVertexId = new OperatorID(dis.readLong(), dis.readLong()); - int parallelism = dis.readInt(); - int maxParallelism = dis.readInt(); - int chainLength = dis.readInt(); - - // Add task state - OperatorState taskState = new OperatorState(jobVertexId, parallelism, maxParallelism); - operatorStates.add(taskState); - - // Sub task states - int numSubTaskStates = dis.readInt(); - - for (int j = 0; j < numSubTaskStates; j++) { - int subtaskIndex = dis.readInt(); - - OperatorSubtaskState subtaskState = deserializeSubtaskState(dis); - taskState.putState(subtaskIndex, subtaskState); - } - } - - return new CheckpointMetadata(checkpointId, operatorStates, masterStates); + public CheckpointMetadata deserialize(DataInputStream dis, ClassLoader classLoader) throws IOException { + return deserializeMetadata(dis); } // ------------------------------------------------------------------------ - // master state (de)serialization methods + // version-specific serialization // ------------------------------------------------------------------------ - private static void serializeMasterState(MasterState state, DataOutputStream dos) throws IOException { - // magic number for error detection - dos.writeInt(MASTER_STATE_MAGIC_NUMBER); - - // for safety, we serialize first into an array and then write the array and its - // length into the checkpoint - final ByteArrayOutputStream baos = new ByteArrayOutputStream(); - final DataOutputStream out = new DataOutputStream(baos); - - out.writeInt(state.version()); - out.writeUTF(state.name()); - - final byte[] bytes = state.bytes(); - out.writeInt(bytes.length); - out.write(bytes, 0, bytes.length); - - out.close(); - byte[] data = baos.toByteArray(); + @Override + protected void serializeOperatorState(OperatorState operatorState, DataOutputStream dos) throws IOException { + // Operator ID + dos.writeLong(operatorState.getOperatorID().getLowerPart()); + dos.writeLong(operatorState.getOperatorID().getUpperPart()); - dos.writeInt(data.length); - dos.write(data, 0, data.length); - } + // Parallelism + int parallelism = operatorState.getParallelism(); + dos.writeInt(parallelism); + dos.writeInt(operatorState.getMaxParallelism()); - private MasterState deserializeMasterState(DataInputStream dis) throws IOException { - final int magicNumber = dis.readInt(); - if (magicNumber != MASTER_STATE_MAGIC_NUMBER) { - throw new IOException("incorrect magic number in master styte byte sequence"); - } + // this field was "chain length" before Flink 1.3, and it is still part + // of the format, despite being unused + dos.writeInt(1); - final int numBytes = dis.readInt(); - if (numBytes <= 0) { - throw new IOException("found zero or negative length for master state bytes"); + // Sub task states + Map subtaskStateMap = operatorState.getSubtaskStates(); + dos.writeInt(subtaskStateMap.size()); + for (Map.Entry entry : subtaskStateMap.entrySet()) { + dos.writeInt(entry.getKey()); + serializeSubtaskState(entry.getValue(), dos); } + } - final byte[] data = new byte[numBytes]; - dis.readFully(data); + @Override + protected OperatorState deserializeOperatorState(DataInputStream dis) throws IOException { + final OperatorID jobVertexId = new OperatorID(dis.readLong(), dis.readLong()); + final int parallelism = dis.readInt(); + final int maxParallelism = dis.readInt(); - final DataInputStream in = new DataInputStream(new ByteArrayInputStream(data)); + // this field was "chain length" before Flink 1.3, and it is still part + // of the format, despite being unused + dis.readInt(); - final int version = in.readInt(); - final String name = in.readUTF(); + // Add task state + final OperatorState taskState = new OperatorState(jobVertexId, parallelism, maxParallelism); - final byte[] bytes = new byte[in.readInt()]; - in.readFully(bytes); + // Sub task states + final int numSubTaskStates = dis.readInt(); - // check that the data is not corrupt - if (in.read() != -1) { - throw new IOException("found trailing bytes in master state"); + for (int j = 0; j < numSubTaskStates; j++) { + final int subtaskIndex = dis.readInt(); + final OperatorSubtaskState subtaskState = deserializeSubtaskState(dis); + taskState.putState(subtaskIndex, subtaskState); } - return new MasterState(name, bytes, version); + return taskState; } - // ------------------------------------------------------------------------ - // task state (de)serialization methods - // ------------------------------------------------------------------------ - - private static T extractSingleton(Collection collection) { - if (collection == null || collection.isEmpty()) { - return null; - } - - if (collection.size() == 1) { - return collection.iterator().next(); - } else { - throw new IllegalStateException("Expected singleton collection, but found size: " + collection.size()); - } - } - - private static void serializeSubtaskState(OperatorSubtaskState subtaskState, DataOutputStream dos) throws IOException { - + @Override + protected void serializeSubtaskState(OperatorSubtaskState subtaskState, DataOutputStream dos) throws IOException { + // write two unused fields for compatibility: + // - "duration" + // - number of legacy states dos.writeLong(-1); + dos.writeInt(0); - int len = 0; - dos.writeInt(len); - - OperatorStateHandle operatorStateBackend = extractSingleton(subtaskState.getManagedOperatorState()); - - len = operatorStateBackend != null ? 1 : 0; - dos.writeInt(len); - if (len == 1) { - serializeOperatorStateHandle(operatorStateBackend, dos); - } - - OperatorStateHandle operatorStateFromStream = extractSingleton(subtaskState.getRawOperatorState()); - - len = operatorStateFromStream != null ? 1 : 0; - dos.writeInt(len); - if (len == 1) { - serializeOperatorStateHandle(operatorStateFromStream, dos); - } - - KeyedStateHandle keyedStateBackend = extractSingleton(subtaskState.getManagedKeyedState()); - serializeKeyedStateHandle(keyedStateBackend, dos); - - KeyedStateHandle keyedStateStream = extractSingleton(subtaskState.getRawKeyedState()); - serializeKeyedStateHandle(keyedStateStream, dos); + super.serializeSubtaskState(subtaskState, dos); } - private static OperatorSubtaskState deserializeSubtaskState(DataInputStream dis) throws IOException { - // Duration field has been removed from SubtaskState, do not remove - long ignoredDuration = dis.readLong(); - + @Override + protected OperatorSubtaskState deserializeSubtaskState(DataInputStream dis) throws IOException { + // read two unused fields for compatibility: + // - "duration" + // - number of legacy states + dis.readLong(); final int numLegacyTaskStates = dis.readInt(); + if (numLegacyTaskStates > 0) { throw new IOException( "Legacy state (from Flink <= 1.1, created through the 'Checkpointed' interface) is " + - "no longer supported."); - } - - int len = dis.readInt(); - OperatorStateHandle operatorStateBackend = len == 0 ? null : deserializeOperatorStateHandle(dis); - - len = dis.readInt(); - OperatorStateHandle operatorStateStream = len == 0 ? null : deserializeOperatorStateHandle(dis); - - KeyedStateHandle keyedStateBackend = deserializeKeyedStateHandle(dis); - - KeyedStateHandle keyedStateStream = deserializeKeyedStateHandle(dis); - - return new OperatorSubtaskState( - operatorStateBackend, - operatorStateStream, - keyedStateBackend, - keyedStateStream); - } - - @VisibleForTesting - public static void serializeKeyedStateHandle( - KeyedStateHandle stateHandle, DataOutputStream dos) throws IOException { - - if (stateHandle == null) { - dos.writeByte(NULL_HANDLE); - } else if (stateHandle instanceof KeyGroupsStateHandle) { - KeyGroupsStateHandle keyGroupsStateHandle = (KeyGroupsStateHandle) stateHandle; - - dos.writeByte(KEY_GROUPS_HANDLE); - dos.writeInt(keyGroupsStateHandle.getKeyGroupRange().getStartKeyGroup()); - dos.writeInt(keyGroupsStateHandle.getKeyGroupRange().getNumberOfKeyGroups()); - for (int keyGroup : keyGroupsStateHandle.getKeyGroupRange()) { - dos.writeLong(keyGroupsStateHandle.getOffsetForKeyGroup(keyGroup)); - } - serializeStreamStateHandle(keyGroupsStateHandle.getDelegateStateHandle(), dos); - } else if (stateHandle instanceof IncrementalRemoteKeyedStateHandle) { - IncrementalRemoteKeyedStateHandle incrementalKeyedStateHandle = - (IncrementalRemoteKeyedStateHandle) stateHandle; - - dos.writeByte(INCREMENTAL_KEY_GROUPS_HANDLE); - - dos.writeLong(incrementalKeyedStateHandle.getCheckpointId()); - dos.writeUTF(String.valueOf(incrementalKeyedStateHandle.getBackendIdentifier())); - dos.writeInt(incrementalKeyedStateHandle.getKeyGroupRange().getStartKeyGroup()); - dos.writeInt(incrementalKeyedStateHandle.getKeyGroupRange().getNumberOfKeyGroups()); - - serializeStreamStateHandle(incrementalKeyedStateHandle.getMetaStateHandle(), dos); - - serializeStreamStateHandleMap(incrementalKeyedStateHandle.getSharedState(), dos); - serializeStreamStateHandleMap(incrementalKeyedStateHandle.getPrivateState(), dos); - } else { - throw new IllegalStateException("Unknown KeyedStateHandle type: " + stateHandle.getClass()); + "no longer supported."); } - } - - private static void serializeStreamStateHandleMap( - Map map, - DataOutputStream dos) throws IOException { - dos.writeInt(map.size()); - for (Map.Entry entry : map.entrySet()) { - dos.writeUTF(entry.getKey().toString()); - serializeStreamStateHandle(entry.getValue(), dos); - } - } - - private static Map deserializeStreamStateHandleMap( - DataInputStream dis) throws IOException { - - final int size = dis.readInt(); - Map result = new HashMap<>(size); - - for (int i = 0; i < size; ++i) { - StateHandleID stateHandleID = new StateHandleID(dis.readUTF()); - StreamStateHandle stateHandle = deserializeStreamStateHandle(dis); - result.put(stateHandleID, stateHandle); - } - - return result; - } - - @VisibleForTesting - public static KeyedStateHandle deserializeKeyedStateHandle(DataInputStream dis) throws IOException { - final int type = dis.readByte(); - if (NULL_HANDLE == type) { - - return null; - } else if (KEY_GROUPS_HANDLE == type) { - - int startKeyGroup = dis.readInt(); - int numKeyGroups = dis.readInt(); - KeyGroupRange keyGroupRange = - KeyGroupRange.of(startKeyGroup, startKeyGroup + numKeyGroups - 1); - long[] offsets = new long[numKeyGroups]; - for (int i = 0; i < numKeyGroups; ++i) { - offsets[i] = dis.readLong(); - } - KeyGroupRangeOffsets keyGroupRangeOffsets = new KeyGroupRangeOffsets( - keyGroupRange, offsets); - StreamStateHandle stateHandle = deserializeStreamStateHandle(dis); - return new KeyGroupsStateHandle(keyGroupRangeOffsets, stateHandle); - } else if (INCREMENTAL_KEY_GROUPS_HANDLE == type) { - - long checkpointId = dis.readLong(); - String backendId = dis.readUTF(); - int startKeyGroup = dis.readInt(); - int numKeyGroups = dis.readInt(); - KeyGroupRange keyGroupRange = - KeyGroupRange.of(startKeyGroup, startKeyGroup + numKeyGroups - 1); - StreamStateHandle metaDataStateHandle = deserializeStreamStateHandle(dis); - Map sharedStates = deserializeStreamStateHandleMap(dis); - Map privateStates = deserializeStreamStateHandleMap(dis); - - UUID uuid; - - try { - uuid = UUID.fromString(backendId); - } catch (Exception ex) { - // compatibility with old format pre FLINK-6964: - uuid = UUID.nameUUIDFromBytes(backendId.getBytes(StandardCharsets.UTF_8)); - } - - return new IncrementalRemoteKeyedStateHandle( - uuid, - keyGroupRange, - checkpointId, - sharedStates, - privateStates, - metaDataStateHandle); - } else { - throw new IllegalStateException("Reading invalid KeyedStateHandle, type: " + type); - } - } - - @VisibleForTesting - public static void serializeOperatorStateHandle( - OperatorStateHandle stateHandle, DataOutputStream dos) throws IOException { - - if (stateHandle != null) { - dos.writeByte(PARTITIONABLE_OPERATOR_STATE_HANDLE); - Map partitionOffsetsMap = - stateHandle.getStateNameToPartitionOffsets(); - dos.writeInt(partitionOffsetsMap.size()); - for (Map.Entry entry : partitionOffsetsMap.entrySet()) { - dos.writeUTF(entry.getKey()); - - OperatorStateHandle.StateMetaInfo stateMetaInfo = entry.getValue(); - - int mode = stateMetaInfo.getDistributionMode().ordinal(); - dos.writeByte(mode); - - long[] offsets = stateMetaInfo.getOffsets(); - dos.writeInt(offsets.length); - for (long offset : offsets) { - dos.writeLong(offset); - } - } - serializeStreamStateHandle(stateHandle.getDelegateStateHandle(), dos); - } else { - dos.writeByte(NULL_HANDLE); - } - } - - @VisibleForTesting - public static OperatorStateHandle deserializeOperatorStateHandle( - DataInputStream dis) throws IOException { - - final int type = dis.readByte(); - if (NULL_HANDLE == type) { - return null; - } else if (PARTITIONABLE_OPERATOR_STATE_HANDLE == type) { - int mapSize = dis.readInt(); - Map offsetsMap = new HashMap<>(mapSize); - for (int i = 0; i < mapSize; ++i) { - String key = dis.readUTF(); - - int modeOrdinal = dis.readByte(); - OperatorStateHandle.Mode mode = OperatorStateHandle.Mode.values()[modeOrdinal]; - - long[] offsets = new long[dis.readInt()]; - for (int j = 0; j < offsets.length; ++j) { - offsets[j] = dis.readLong(); - } - - OperatorStateHandle.StateMetaInfo metaInfo = - new OperatorStateHandle.StateMetaInfo(offsets, mode); - offsetsMap.put(key, metaInfo); - } - StreamStateHandle stateHandle = deserializeStreamStateHandle(dis); - return new OperatorStreamStateHandle(offsetsMap, stateHandle); - } else { - throw new IllegalStateException("Reading invalid OperatorStateHandle, type: " + type); - } - } - - @VisibleForTesting - public static void serializeStreamStateHandle( - StreamStateHandle stateHandle, DataOutputStream dos) throws IOException { - - if (stateHandle == null) { - dos.writeByte(NULL_HANDLE); - - } else if (stateHandle instanceof FileStateHandle) { - dos.writeByte(FILE_STREAM_STATE_HANDLE); - FileStateHandle fileStateHandle = (FileStateHandle) stateHandle; - dos.writeLong(stateHandle.getStateSize()); - dos.writeUTF(fileStateHandle.getFilePath().toString()); - - } else if (stateHandle instanceof ByteStreamStateHandle) { - dos.writeByte(BYTE_STREAM_STATE_HANDLE); - ByteStreamStateHandle byteStreamStateHandle = (ByteStreamStateHandle) stateHandle; - dos.writeUTF(byteStreamStateHandle.getHandleName()); - byte[] internalData = byteStreamStateHandle.getData(); - dos.writeInt(internalData.length); - dos.write(byteStreamStateHandle.getData()); - } else { - throw new IOException("Unknown implementation of StreamStateHandle: " + stateHandle.getClass()); - } - - dos.flush(); - } - - public static StreamStateHandle deserializeStreamStateHandle(DataInputStream dis) throws IOException { - final int type = dis.read(); - if (NULL_HANDLE == type) { - return null; - } else if (FILE_STREAM_STATE_HANDLE == type) { - long size = dis.readLong(); - String pathString = dis.readUTF(); - return new FileStateHandle(new Path(pathString), size); - } else if (BYTE_STREAM_STATE_HANDLE == type) { - String handleName = dis.readUTF(); - int numBytes = dis.readInt(); - byte[] data = new byte[numBytes]; - dis.readFully(data); - return new ByteStreamStateHandle(handleName, data); - } else { - throw new IOException("Unknown implementation of StreamStateHandle, code: " + type); - } + return super.deserializeSubtaskState(dis); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2V3SerializerBase.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2V3SerializerBase.java new file mode 100644 index 0000000000000000000000000000000000000000..3101bd00292b10825b68ac317972b14a16ef1e32 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2V3SerializerBase.java @@ -0,0 +1,530 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.checkpoint.metadata; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.checkpoint.MasterState; +import org.apache.flink.runtime.checkpoint.OperatorState; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.checkpoint.StateObjectCollection; +import org.apache.flink.runtime.state.IncrementalRemoteKeyedStateHandle; +import org.apache.flink.runtime.state.InputChannelStateHandle; +import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.runtime.state.KeyGroupRangeOffsets; +import org.apache.flink.runtime.state.KeyGroupsStateHandle; +import org.apache.flink.runtime.state.KeyedStateHandle; +import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.OperatorStreamStateHandle; +import org.apache.flink.runtime.state.ResultSubpartitionStateHandle; +import org.apache.flink.runtime.state.StateHandleID; +import org.apache.flink.runtime.state.StateObject; +import org.apache.flink.runtime.state.StreamStateHandle; +import org.apache.flink.runtime.state.filesystem.FileStateHandle; +import org.apache.flink.runtime.state.memory.ByteStreamStateHandle; +import org.apache.flink.util.function.BiConsumerWithException; +import org.apache.flink.util.function.FunctionWithException; + +import javax.annotation.Nullable; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; + +/** + * Base (De)serializer for checkpoint metadata format version 2 and 3. + * + *

The difference between versions 2 and 3 is minor. Version 3 includes + * operator coordinator state for each operator, and drops some minor unused fields. + * + *

Basic checkpoint metadata layout: + *

+ *  +--------------+---------------+-----------------+
+ *  | checkpointID | master states | operator states |
+ *  +--------------+---------------+-----------------+
+ *
+ *  Master state:
+ *  +--------------+---------------------+---------+------+---------------+
+ *  | magic number | num remaining bytes | version | name | payload bytes |
+ *  +--------------+---------------------+---------+------+---------------+
+ * 
+ */ +@Internal +public abstract class MetadataV2V3SerializerBase { + + /** Random magic number for consistency checks. */ + private static final int MASTER_STATE_MAGIC_NUMBER = 0xc96b1696; + + private static final byte NULL_HANDLE = 0; + private static final byte BYTE_STREAM_STATE_HANDLE = 1; + private static final byte FILE_STREAM_STATE_HANDLE = 2; + private static final byte KEY_GROUPS_HANDLE = 3; + private static final byte PARTITIONABLE_OPERATOR_STATE_HANDLE = 4; + private static final byte INCREMENTAL_KEY_GROUPS_HANDLE = 5; + + // ------------------------------------------------------------------------ + // (De)serialization entry points + // ------------------------------------------------------------------------ + + protected void serializeMetadata(CheckpointMetadata checkpointMetadata, DataOutputStream dos) throws IOException { + // first: checkpoint ID + dos.writeLong(checkpointMetadata.getCheckpointId()); + + // second: master state + final Collection masterStates = checkpointMetadata.getMasterStates(); + dos.writeInt(masterStates.size()); + for (MasterState ms : masterStates) { + serializeMasterState(ms, dos); + } + + // third: operator states + Collection operatorStates = checkpointMetadata.getOperatorStates(); + dos.writeInt(operatorStates.size()); + + for (OperatorState operatorState : operatorStates) { + serializeOperatorState(operatorState, dos); + } + } + + protected CheckpointMetadata deserializeMetadata(DataInputStream dis) throws IOException { + // first: checkpoint ID + final long checkpointId = dis.readLong(); + if (checkpointId < 0) { + throw new IOException("invalid checkpoint ID: " + checkpointId); + } + + // second: master state + final List masterStates; + final int numMasterStates = dis.readInt(); + + if (numMasterStates == 0) { + masterStates = Collections.emptyList(); + } + else if (numMasterStates > 0) { + masterStates = new ArrayList<>(numMasterStates); + for (int i = 0; i < numMasterStates; i++) { + masterStates.add(deserializeMasterState(dis)); + } + } + else { + throw new IOException("invalid number of master states: " + numMasterStates); + } + + // third: operator states + final int numTaskStates = dis.readInt(); + final List operatorStates = new ArrayList<>(numTaskStates); + + for (int i = 0; i < numTaskStates; i++) { + operatorStates.add(deserializeOperatorState(dis)); + } + + return new CheckpointMetadata(checkpointId, operatorStates, masterStates); + } + + // ------------------------------------------------------------------------ + // master state (de)serialization methods + // ------------------------------------------------------------------------ + + protected void serializeMasterState(MasterState state, DataOutputStream dos) throws IOException { + // magic number for error detection + dos.writeInt(MASTER_STATE_MAGIC_NUMBER); + + // for safety, we serialize first into an array and then write the array and its + // length into the checkpoint + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + final DataOutputStream out = new DataOutputStream(baos); + + out.writeInt(state.version()); + out.writeUTF(state.name()); + + final byte[] bytes = state.bytes(); + out.writeInt(bytes.length); + out.write(bytes, 0, bytes.length); + + out.close(); + byte[] data = baos.toByteArray(); + + dos.writeInt(data.length); + dos.write(data, 0, data.length); + } + + protected MasterState deserializeMasterState(DataInputStream dis) throws IOException { + final int magicNumber = dis.readInt(); + if (magicNumber != MASTER_STATE_MAGIC_NUMBER) { + throw new IOException("incorrect magic number in master styte byte sequence"); + } + + final int numBytes = dis.readInt(); + if (numBytes <= 0) { + throw new IOException("found zero or negative length for master state bytes"); + } + + final byte[] data = new byte[numBytes]; + dis.readFully(data); + + final DataInputStream in = new DataInputStream(new ByteArrayInputStream(data)); + + final int version = in.readInt(); + final String name = in.readUTF(); + + final byte[] bytes = new byte[in.readInt()]; + in.readFully(bytes); + + // check that the data is not corrupt + if (in.read() != -1) { + throw new IOException("found trailing bytes in master state"); + } + + return new MasterState(name, bytes, version); + } + + // ------------------------------------------------------------------------ + // operator state (de)serialization methods + // ------------------------------------------------------------------------ + + protected abstract void serializeOperatorState(OperatorState operatorState, DataOutputStream dos) throws IOException; + + protected abstract OperatorState deserializeOperatorState(DataInputStream dis) throws IOException; + + // ------------------------------------------------------------------------ + // operator subtask state (de)serialization methods + // ------------------------------------------------------------------------ + + protected void serializeSubtaskState(OperatorSubtaskState subtaskState, DataOutputStream dos) throws IOException { + serializeSingleton(subtaskState.getManagedOperatorState(), dos, MetadataV2V3SerializerBase::serializeOperatorStateHandle); + serializeSingleton(subtaskState.getRawOperatorState(), dos, MetadataV2V3SerializerBase::serializeOperatorStateHandle); + serializeKeyedStateCol(subtaskState.getManagedKeyedState(), dos); + serializeKeyedStateCol(subtaskState.getRawKeyedState(), dos); + } + + private void serializeKeyedStateCol(StateObjectCollection managedKeyedState, DataOutputStream dos) throws IOException { + serializeKeyedStateHandle(extractSingleton(managedKeyedState), dos); + } + + private void serializeSingleton( + StateObjectCollection stateObjectCollection, + DataOutputStream dos, + BiConsumerWithException cons) throws IOException { + final T state = extractSingleton(stateObjectCollection); + if (state != null) { + dos.writeInt(1); + cons.accept(state, dos); + } else { + dos.writeInt(0); + } + } + + protected OperatorSubtaskState deserializeSubtaskState(DataInputStream dis) throws IOException { + final boolean hasManagedOperatorState = dis.readInt() != 0; + final OperatorStateHandle managedOperatorState = hasManagedOperatorState ? deserializeOperatorStateHandle(dis) : null; + + final boolean hasRawOperatorState = dis.readInt() != 0; + final OperatorStateHandle rawOperatorState = hasRawOperatorState ? deserializeOperatorStateHandle(dis) : null; + + final KeyedStateHandle managedKeyedState = deserializeKeyedStateHandle(dis); + final KeyedStateHandle rawKeyedState = deserializeKeyedStateHandle(dis); + + StateObjectCollection inputChannelState = deserializeInputChannelStateHandle(dis); + + StateObjectCollection resultSubpartitionState = deserializeResultSubpartitionStateHandle(dis); + + return new OperatorSubtaskState( + managedOperatorState, + rawOperatorState, + managedKeyedState, + rawKeyedState, + inputChannelState, + resultSubpartitionState); + } + + @VisibleForTesting + public static void serializeKeyedStateHandle( + KeyedStateHandle stateHandle, DataOutputStream dos) throws IOException { + + if (stateHandle == null) { + dos.writeByte(NULL_HANDLE); + } else if (stateHandle instanceof KeyGroupsStateHandle) { + KeyGroupsStateHandle keyGroupsStateHandle = (KeyGroupsStateHandle) stateHandle; + + dos.writeByte(KEY_GROUPS_HANDLE); + dos.writeInt(keyGroupsStateHandle.getKeyGroupRange().getStartKeyGroup()); + dos.writeInt(keyGroupsStateHandle.getKeyGroupRange().getNumberOfKeyGroups()); + for (int keyGroup : keyGroupsStateHandle.getKeyGroupRange()) { + dos.writeLong(keyGroupsStateHandle.getOffsetForKeyGroup(keyGroup)); + } + serializeStreamStateHandle(keyGroupsStateHandle.getDelegateStateHandle(), dos); + } else if (stateHandle instanceof IncrementalRemoteKeyedStateHandle) { + IncrementalRemoteKeyedStateHandle incrementalKeyedStateHandle = + (IncrementalRemoteKeyedStateHandle) stateHandle; + + dos.writeByte(INCREMENTAL_KEY_GROUPS_HANDLE); + + dos.writeLong(incrementalKeyedStateHandle.getCheckpointId()); + dos.writeUTF(String.valueOf(incrementalKeyedStateHandle.getBackendIdentifier())); + dos.writeInt(incrementalKeyedStateHandle.getKeyGroupRange().getStartKeyGroup()); + dos.writeInt(incrementalKeyedStateHandle.getKeyGroupRange().getNumberOfKeyGroups()); + + serializeStreamStateHandle(incrementalKeyedStateHandle.getMetaStateHandle(), dos); + + serializeStreamStateHandleMap(incrementalKeyedStateHandle.getSharedState(), dos); + serializeStreamStateHandleMap(incrementalKeyedStateHandle.getPrivateState(), dos); + } else { + throw new IllegalStateException("Unknown KeyedStateHandle type: " + stateHandle.getClass()); + } + } + + private static void serializeStreamStateHandleMap( + Map map, + DataOutputStream dos) throws IOException { + + dos.writeInt(map.size()); + for (Map.Entry entry : map.entrySet()) { + dos.writeUTF(entry.getKey().toString()); + serializeStreamStateHandle(entry.getValue(), dos); + } + } + + private static Map deserializeStreamStateHandleMap( + DataInputStream dis) throws IOException { + + final int size = dis.readInt(); + Map result = new HashMap<>(size); + + for (int i = 0; i < size; ++i) { + StateHandleID stateHandleID = new StateHandleID(dis.readUTF()); + StreamStateHandle stateHandle = deserializeStreamStateHandle(dis); + result.put(stateHandleID, stateHandle); + } + + return result; + } + + @VisibleForTesting + public static KeyedStateHandle deserializeKeyedStateHandle(DataInputStream dis) throws IOException { + final int type = dis.readByte(); + if (NULL_HANDLE == type) { + + return null; + } else if (KEY_GROUPS_HANDLE == type) { + + int startKeyGroup = dis.readInt(); + int numKeyGroups = dis.readInt(); + KeyGroupRange keyGroupRange = + KeyGroupRange.of(startKeyGroup, startKeyGroup + numKeyGroups - 1); + long[] offsets = new long[numKeyGroups]; + for (int i = 0; i < numKeyGroups; ++i) { + offsets[i] = dis.readLong(); + } + KeyGroupRangeOffsets keyGroupRangeOffsets = new KeyGroupRangeOffsets( + keyGroupRange, offsets); + StreamStateHandle stateHandle = deserializeStreamStateHandle(dis); + return new KeyGroupsStateHandle(keyGroupRangeOffsets, stateHandle); + } else if (INCREMENTAL_KEY_GROUPS_HANDLE == type) { + + long checkpointId = dis.readLong(); + String backendId = dis.readUTF(); + int startKeyGroup = dis.readInt(); + int numKeyGroups = dis.readInt(); + KeyGroupRange keyGroupRange = + KeyGroupRange.of(startKeyGroup, startKeyGroup + numKeyGroups - 1); + + StreamStateHandle metaDataStateHandle = deserializeStreamStateHandle(dis); + Map sharedStates = deserializeStreamStateHandleMap(dis); + Map privateStates = deserializeStreamStateHandleMap(dis); + + UUID uuid; + + try { + uuid = UUID.fromString(backendId); + } catch (Exception ex) { + // compatibility with old format pre FLINK-6964: + uuid = UUID.nameUUIDFromBytes(backendId.getBytes(StandardCharsets.UTF_8)); + } + + return new IncrementalRemoteKeyedStateHandle( + uuid, + keyGroupRange, + checkpointId, + sharedStates, + privateStates, + metaDataStateHandle); + } else { + throw new IllegalStateException("Reading invalid KeyedStateHandle, type: " + type); + } + } + + @VisibleForTesting + public static void serializeOperatorStateHandle( + OperatorStateHandle stateHandle, DataOutputStream dos) throws IOException { + + if (stateHandle != null) { + dos.writeByte(PARTITIONABLE_OPERATOR_STATE_HANDLE); + Map partitionOffsetsMap = + stateHandle.getStateNameToPartitionOffsets(); + dos.writeInt(partitionOffsetsMap.size()); + for (Map.Entry entry : partitionOffsetsMap.entrySet()) { + dos.writeUTF(entry.getKey()); + + OperatorStateHandle.StateMetaInfo stateMetaInfo = entry.getValue(); + + int mode = stateMetaInfo.getDistributionMode().ordinal(); + dos.writeByte(mode); + + long[] offsets = stateMetaInfo.getOffsets(); + dos.writeInt(offsets.length); + for (long offset : offsets) { + dos.writeLong(offset); + } + } + serializeStreamStateHandle(stateHandle.getDelegateStateHandle(), dos); + } else { + dos.writeByte(NULL_HANDLE); + } + } + + @VisibleForTesting + public static OperatorStateHandle deserializeOperatorStateHandle( + DataInputStream dis) throws IOException { + + final int type = dis.readByte(); + if (NULL_HANDLE == type) { + return null; + } else if (PARTITIONABLE_OPERATOR_STATE_HANDLE == type) { + int mapSize = dis.readInt(); + Map offsetsMap = new HashMap<>(mapSize); + for (int i = 0; i < mapSize; ++i) { + String key = dis.readUTF(); + + int modeOrdinal = dis.readByte(); + OperatorStateHandle.Mode mode = OperatorStateHandle.Mode.values()[modeOrdinal]; + + long[] offsets = new long[dis.readInt()]; + for (int j = 0; j < offsets.length; ++j) { + offsets[j] = dis.readLong(); + } + + OperatorStateHandle.StateMetaInfo metaInfo = + new OperatorStateHandle.StateMetaInfo(offsets, mode); + offsetsMap.put(key, metaInfo); + } + StreamStateHandle stateHandle = deserializeStreamStateHandle(dis); + return new OperatorStreamStateHandle(offsetsMap, stateHandle); + } else { + throw new IllegalStateException("Reading invalid OperatorStateHandle, type: " + type); + } + } + + @VisibleForTesting + public static void serializeStreamStateHandle( + StreamStateHandle stateHandle, DataOutputStream dos) throws IOException { + + if (stateHandle == null) { + dos.writeByte(NULL_HANDLE); + + } else if (stateHandle instanceof FileStateHandle) { + dos.writeByte(FILE_STREAM_STATE_HANDLE); + FileStateHandle fileStateHandle = (FileStateHandle) stateHandle; + dos.writeLong(stateHandle.getStateSize()); + dos.writeUTF(fileStateHandle.getFilePath().toString()); + + } else if (stateHandle instanceof ByteStreamStateHandle) { + dos.writeByte(BYTE_STREAM_STATE_HANDLE); + ByteStreamStateHandle byteStreamStateHandle = (ByteStreamStateHandle) stateHandle; + dos.writeUTF(byteStreamStateHandle.getHandleName()); + byte[] internalData = byteStreamStateHandle.getData(); + dos.writeInt(internalData.length); + dos.write(byteStreamStateHandle.getData()); + } else { + throw new IOException("Unknown implementation of StreamStateHandle: " + stateHandle.getClass()); + } + + dos.flush(); + } + + public static StreamStateHandle deserializeStreamStateHandle(DataInputStream dis) throws IOException { + final int type = dis.read(); + if (NULL_HANDLE == type) { + return null; + } else if (FILE_STREAM_STATE_HANDLE == type) { + long size = dis.readLong(); + String pathString = dis.readUTF(); + return new FileStateHandle(new Path(pathString), size); + } else if (BYTE_STREAM_STATE_HANDLE == type) { + String handleName = dis.readUTF(); + int numBytes = dis.readInt(); + byte[] data = new byte[numBytes]; + dis.readFully(data); + return new ByteStreamStateHandle(handleName, data); + } else { + throw new IOException("Unknown implementation of StreamStateHandle, code: " + type); + } + } + + // ------------------------------------------------------------------------ + // utilities + // ------------------------------------------------------------------------ + + @Nullable + static T extractSingleton(Collection collection) { + if (collection == null || collection.isEmpty()) { + return null; + } + + if (collection.size() == 1) { + return collection.iterator().next(); + } else { + throw new IllegalStateException("Expected singleton collection, but found size: " + collection.size()); + } + } + + protected StateObjectCollection deserializeResultSubpartitionStateHandle(DataInputStream dis) throws IOException { + return StateObjectCollection.empty(); + } + + protected StateObjectCollection deserializeInputChannelStateHandle(DataInputStream dis) throws IOException { + return StateObjectCollection.empty(); + } + + protected void serializeResultSubpartitionStateHandle(ResultSubpartitionStateHandle resultSubpartitionStateHandle, DataOutputStream dos) throws IOException { + } + + protected void serializeInputChannelStateHandle(InputChannelStateHandle inputChannelStateHandle, DataOutputStream dos) throws IOException { + } + + static StateObjectCollection deserializeCollection(DataInputStream dis, FunctionWithException s) throws IOException { + int size = dis.readInt(); + List result = new ArrayList<>(); + for (int i = 0; i < size; i++) { + result.add(s.apply(dis)); + } + return new StateObjectCollection<>(result); + } + +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV3Serializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV3Serializer.java new file mode 100644 index 0000000000000000000000000000000000000000..cfd648fe2f8636ea7c3a94b3efbc223c29338311 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV3Serializer.java @@ -0,0 +1,172 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.checkpoint.metadata; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.runtime.checkpoint.OperatorState; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.checkpoint.StateObjectCollection; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.state.InputChannelStateHandle; +import org.apache.flink.runtime.state.ResultSubpartitionStateHandle; +import org.apache.flink.runtime.state.StateObject; +import org.apache.flink.util.function.BiConsumerWithException; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.Map; + +/** + * (De)serializer for checkpoint metadata format version 3. + * This format was introduced with Apache Flink 1.11.0. + * + *

Compared to format version 2, this drops some unused fields and introduces + * operator coordinator state. + * + *

See {@link MetadataV2V3SerializerBase} for a description of the format layout. + */ +@Internal +public class MetadataV3Serializer extends MetadataV2V3SerializerBase implements MetadataSerializer { + + /** The metadata format version. */ + public static final int VERSION = 3; + + /** The singleton instance of the serializer. */ + public static final MetadataV3Serializer INSTANCE = new MetadataV3Serializer(); + + private final ChannelStateHandleSerializer channelStateHandleSerializer = new ChannelStateHandleSerializer(); + + /** Singleton, not meant to be instantiated. */ + private MetadataV3Serializer() {} + + @Override + public int getVersion() { + return VERSION; + } + + // ------------------------------------------------------------------------ + // (De)serialization entry points + // ------------------------------------------------------------------------ + + public static void serialize(CheckpointMetadata checkpointMetadata, DataOutputStream dos) throws IOException { + INSTANCE.serializeMetadata(checkpointMetadata, dos); + } + + @Override + public CheckpointMetadata deserialize(DataInputStream dis, ClassLoader classLoader) throws IOException { + return deserializeMetadata(dis); + } + + // ------------------------------------------------------------------------ + // version-specific serialization formats + // ------------------------------------------------------------------------ + + @Override + protected void serializeOperatorState(OperatorState operatorState, DataOutputStream dos) throws IOException { + // Operator ID + dos.writeLong(operatorState.getOperatorID().getLowerPart()); + dos.writeLong(operatorState.getOperatorID().getUpperPart()); + + // Parallelism + dos.writeInt(operatorState.getParallelism()); + dos.writeInt(operatorState.getMaxParallelism()); + + // Coordinator state + serializeStreamStateHandle(operatorState.getCoordinatorState(), dos); + + // Sub task states + final Map subtaskStateMap = operatorState.getSubtaskStates(); + dos.writeInt(subtaskStateMap.size()); + for (Map.Entry entry : subtaskStateMap.entrySet()) { + dos.writeInt(entry.getKey()); + serializeSubtaskState(entry.getValue(), dos); + } + } + + @Override + protected void serializeSubtaskState(OperatorSubtaskState subtaskState, DataOutputStream dos) throws IOException { + super.serializeSubtaskState(subtaskState, dos); + serializeCollection(subtaskState.getInputChannelState(), dos, this::serializeInputChannelStateHandle); + serializeCollection(subtaskState.getResultSubpartitionState(), dos, this::serializeResultSubpartitionStateHandle); + } + + @Override + protected OperatorState deserializeOperatorState(DataInputStream dis) throws IOException { + final OperatorID jobVertexId = new OperatorID(dis.readLong(), dis.readLong()); + final int parallelism = dis.readInt(); + final int maxParallelism = dis.readInt(); + + final OperatorState operatorState = new OperatorState(jobVertexId, parallelism, maxParallelism); + + // Coordinator state + operatorState.setCoordinatorState(deserializeStreamStateHandle(dis)); + + // Sub task states + final int numSubTaskStates = dis.readInt(); + + for (int j = 0; j < numSubTaskStates; j++) { + final int subtaskIndex = dis.readInt(); + final OperatorSubtaskState subtaskState = deserializeSubtaskState(dis); + operatorState.putState(subtaskIndex, subtaskState); + } + + return operatorState; + } + + @VisibleForTesting + @Override + public void serializeResultSubpartitionStateHandle(ResultSubpartitionStateHandle handle, DataOutputStream dos) throws IOException { + channelStateHandleSerializer.serialize(handle, dos); + } + + @VisibleForTesting + @Override + public StateObjectCollection deserializeResultSubpartitionStateHandle(DataInputStream dis) throws IOException { + return deserializeCollection(dis, channelStateHandleSerializer::deserializeResultSubpartitionStateHandle); + } + + @VisibleForTesting + @Override + public void serializeInputChannelStateHandle(InputChannelStateHandle handle, DataOutputStream dos) throws IOException { + channelStateHandleSerializer.serialize(handle, dos); + } + + @VisibleForTesting + @Override + public StateObjectCollection deserializeInputChannelStateHandle(DataInputStream dis) throws IOException { + return deserializeCollection(dis, channelStateHandleSerializer::deserializeInputChannelStateHandle); + } + + private void serializeCollection( + StateObjectCollection stateObjectCollection, + DataOutputStream dos, + BiConsumerWithException cons) throws IOException { + if (stateObjectCollection == null) { + dos.writeInt(0); + } else { + dos.writeInt(stateObjectCollection.size()); + for (T stateObject : stateObjectCollection) { + cons.accept(stateObject, dos); + } + } + } + +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java index 88f2676b4b5ede4167837f5c6536696f875ff6ba..4c7db29f39df8179688bb58e1874a427b9afd900 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java @@ -18,15 +18,16 @@ package org.apache.flink.runtime.clusterframework; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.configuration.AkkaOptions; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.CoreOptions; -import org.apache.flink.configuration.ResourceManagerOptions; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.entrypoint.parser.CommandLineOptions; +import org.apache.flink.runtime.util.config.memory.ProcessMemoryUtils; import org.apache.flink.util.NetUtils; import org.apache.flink.util.OperatingSystem; @@ -41,7 +42,6 @@ import org.apache.commons.cli.Option; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.io.File; @@ -52,6 +52,7 @@ import java.net.BindException; import java.util.HashMap; import java.util.Iterator; import java.util.Map; +import java.util.Optional; import java.util.stream.Stream; import scala.Some; @@ -82,92 +83,80 @@ public class BootstrapTools { .build(); /** - * Starts an ActorSystem with the given configuration listening at the address/ports. + * Starts a remote ActorSystem at given address and specific port range. * @param configuration The Flink configuration - * @param listeningAddress The address to listen at. - * @param portRangeDefinition The port range to choose a port from. + * @param externalAddress The external address to access the ActorSystem. + * @param externalPortRange The choosing range of the external port to access the ActorSystem. * @param logger The logger to output log information. * @return The ActorSystem which has been started * @throws Exception Thrown when actor system cannot be started in specified port range */ - public static ActorSystem startActorSystem( + @VisibleForTesting + public static ActorSystem startRemoteActorSystem( Configuration configuration, - String listeningAddress, - String portRangeDefinition, + String externalAddress, + String externalPortRange, Logger logger) throws Exception { - return startActorSystem( - configuration, - listeningAddress, - portRangeDefinition, - logger, - ForkJoinExecutorConfiguration.fromConfiguration(configuration)); - } - - /** - * Starts an ActorSystem with the given configuration listening at the address/ports. - * - * @param configuration The Flink configuration - * @param listeningAddress The address to listen at. - * @param portRangeDefinition The port range to choose a port from. - * @param logger The logger to output log information. - * @param actorSystemExecutorConfiguration configuration for the ActorSystem's underlying executor - * @return The ActorSystem which has been started - * @throws Exception Thrown when actor system cannot be started in specified port range - */ - public static ActorSystem startActorSystem( - Configuration configuration, - String listeningAddress, - String portRangeDefinition, - Logger logger, - @Nonnull ActorSystemExecutorConfiguration actorSystemExecutorConfiguration) throws Exception { - return startActorSystem( + return startRemoteActorSystem( configuration, AkkaUtils.getFlinkActorSystemName(), - listeningAddress, - portRangeDefinition, + externalAddress, + externalPortRange, + NetUtils.getWildcardIPAddress(), + Optional.empty(), logger, - actorSystemExecutorConfiguration); + ForkJoinExecutorConfiguration.fromConfiguration(configuration), + null); } /** - * Starts an ActorSystem with the given configuration listening at the address/ports. + * Starts a remote ActorSystem at given address and specific port range. * * @param configuration The Flink configuration * @param actorSystemName Name of the started {@link ActorSystem} - * @param listeningAddress The address to listen at. - * @param portRangeDefinition The port range to choose a port from. + * @param externalAddress The external address to access the ActorSystem. + * @param externalPortRange The choosing range of the external port to access the ActorSystem. + * @param bindAddress The local address to bind to. + * @param bindPort The local port to bind to. If not present, then the external port will be used. * @param logger The logger to output log information. * @param actorSystemExecutorConfiguration configuration for the ActorSystem's underlying executor + * @param customConfig Custom Akka config to be combined with the config derived from Flink configuration. * @return The ActorSystem which has been started * @throws Exception Thrown when actor system cannot be started in specified port range */ - public static ActorSystem startActorSystem( + public static ActorSystem startRemoteActorSystem( Configuration configuration, String actorSystemName, - String listeningAddress, - String portRangeDefinition, + String externalAddress, + String externalPortRange, + String bindAddress, + @SuppressWarnings("OptionalUsedAsFieldOrParameterType") Optional bindPort, Logger logger, - @Nonnull ActorSystemExecutorConfiguration actorSystemExecutorConfiguration) throws Exception { + ActorSystemExecutorConfiguration actorSystemExecutorConfiguration, + Config customConfig) throws Exception { // parse port range definition and create port iterator Iterator portsIterator; try { - portsIterator = NetUtils.getPortRangeFromString(portRangeDefinition); + portsIterator = NetUtils.getPortRangeFromString(externalPortRange); } catch (Exception e) { - throw new IllegalArgumentException("Invalid port range definition: " + portRangeDefinition); + throw new IllegalArgumentException("Invalid port range definition: " + externalPortRange); } while (portsIterator.hasNext()) { - final int port = portsIterator.next(); + final int externalPort = portsIterator.next(); try { - return startActorSystem( + return startRemoteActorSystem( configuration, actorSystemName, - listeningAddress, - port, + externalAddress, + externalPort, + bindAddress, + bindPort.orElse(externalPort), logger, - actorSystemExecutorConfiguration); + actorSystemExecutorConfiguration, + customConfig); } catch (Exception e) { // we can continue to try if this contains a netty channel exception @@ -181,104 +170,115 @@ public class BootstrapTools { // if we come here, we have exhausted the port range throw new BindException("Could not start actor system on any port in port range " - + portRangeDefinition); + + externalPortRange); } /** - * Starts an Actor System at a specific port. - * + * Starts a remote Actor System at given address and specific port. * @param configuration The Flink configuration. - * @param listeningAddress The address to listen at. - * @param listeningPort The port to listen at. + * @param actorSystemName Name of the started {@link ActorSystem} + * @param externalAddress The external address to access the ActorSystem. + * @param externalPort The external port to access the ActorSystem. + * @param bindAddress The local address to bind to. + * @param bindPort The local port to bind to. * @param logger the logger to output log information. + * @param actorSystemExecutorConfiguration configuration for the ActorSystem's underlying executor + * @param customConfig Custom Akka config to be combined with the config derived from Flink configuration. * @return The ActorSystem which has been started. * @throws Exception */ - public static ActorSystem startActorSystem( + private static ActorSystem startRemoteActorSystem( Configuration configuration, - String listeningAddress, - int listeningPort, - Logger logger) throws Exception { - return startActorSystem( - configuration, - listeningAddress, - listeningPort, - logger, - ForkJoinExecutorConfiguration.fromConfiguration(configuration)); - } + String actorSystemName, + String externalAddress, + int externalPort, + String bindAddress, + int bindPort, + Logger logger, + ActorSystemExecutorConfiguration actorSystemExecutorConfiguration, + Config customConfig) throws Exception { - /** - * Starts an Actor System at a specific port. - * @param configuration The Flink configuration. - * @param listeningAddress The address to listen at. - * @param listeningPort The port to listen at. - * @param logger the logger to output log information. - * @param actorSystemExecutorConfiguration configuration for the ActorSystem's underlying executor - * @return The ActorSystem which has been started. - * @throws Exception - */ - public static ActorSystem startActorSystem( - Configuration configuration, - String listeningAddress, - int listeningPort, - Logger logger, - ActorSystemExecutorConfiguration actorSystemExecutorConfiguration) throws Exception { - return startActorSystem( - configuration, - AkkaUtils.getFlinkActorSystemName(), - listeningAddress, - listeningPort, - logger, - actorSystemExecutorConfiguration); + String externalHostPortUrl = NetUtils.unresolvedHostAndPortToNormalizedString(externalAddress, externalPort); + String bindHostPortUrl = NetUtils.unresolvedHostAndPortToNormalizedString(bindAddress, bindPort); + logger.info("Trying to start actor system, external address {}, bind address {}.", externalHostPortUrl, bindHostPortUrl); + + try { + Config akkaConfig = AkkaUtils.getAkkaConfig( + configuration, + new Some<>(new Tuple2<>(externalAddress, externalPort)), + new Some<>(new Tuple2<>(bindAddress, bindPort)), + actorSystemExecutorConfiguration.getAkkaConfig()); + + if (customConfig != null) { + akkaConfig = customConfig.withFallback(akkaConfig); + } + + return startActorSystem(akkaConfig, actorSystemName, logger); + } + catch (Throwable t) { + if (t instanceof ChannelException) { + Throwable cause = t.getCause(); + if (cause != null && t.getCause() instanceof BindException) { + throw new IOException("Unable to create ActorSystem at address " + bindHostPortUrl + + " : " + cause.getMessage(), t); + } + } + throw new Exception("Could not create actor system", t); + } } /** - * Starts an Actor System at a specific port. + * Starts a local Actor System. * @param configuration The Flink configuration. - * @param actorSystemName Name of the started {@link ActorSystem} - * @param listeningAddress The address to listen at. - * @param listeningPort The port to listen at. - * @param logger the logger to output log information. - * @param actorSystemExecutorConfiguration configuration for the ActorSystem's underlying executor + * @param actorSystemName Name of the started ActorSystem. + * @param logger The logger to output log information. + * @param actorSystemExecutorConfiguration Configuration for the ActorSystem's underlying executor. + * @param customConfig Custom Akka config to be combined with the config derived from Flink configuration. * @return The ActorSystem which has been started. * @throws Exception */ - public static ActorSystem startActorSystem( + public static ActorSystem startLocalActorSystem( Configuration configuration, String actorSystemName, - String listeningAddress, - int listeningPort, Logger logger, - ActorSystemExecutorConfiguration actorSystemExecutorConfiguration) throws Exception { + ActorSystemExecutorConfiguration actorSystemExecutorConfiguration, + Config customConfig) throws Exception { - String hostPortUrl = NetUtils.unresolvedHostAndPortToNormalizedString(listeningAddress, listeningPort); - logger.info("Trying to start actor system at {}", hostPortUrl); + logger.info("Trying to start local actor system"); try { Config akkaConfig = AkkaUtils.getAkkaConfig( configuration, - new Some<>(new Tuple2<>(listeningAddress, listeningPort)), + scala.Option.empty(), + scala.Option.empty(), actorSystemExecutorConfiguration.getAkkaConfig()); - logger.debug("Using akka configuration\n {}", akkaConfig); - - ActorSystem actorSystem = AkkaUtils.createActorSystem(actorSystemName, akkaConfig); + if (customConfig != null) { + akkaConfig = customConfig.withFallback(akkaConfig); + } - logger.info("Actor system started at {}", AkkaUtils.getAddress(actorSystem)); - return actorSystem; + return startActorSystem(akkaConfig, actorSystemName, logger); } catch (Throwable t) { - if (t instanceof ChannelException) { - Throwable cause = t.getCause(); - if (cause != null && t.getCause() instanceof BindException) { - throw new IOException("Unable to create ActorSystem at address " + hostPortUrl + - " : " + cause.getMessage(), t); - } - } throw new Exception("Could not create actor system", t); } } + /** + * Starts an Actor System with given Akka config. + * @param akkaConfig Config of the started ActorSystem. + * @param actorSystemName Name of the started ActorSystem. + * @param logger The logger to output log information. + * @return The ActorSystem which has been started. + */ + private static ActorSystem startActorSystem(Config akkaConfig, String actorSystemName, Logger logger) { + logger.debug("Using akka configuration\n {}", akkaConfig); + ActorSystem actorSystem = AkkaUtils.createActorSystem(actorSystemName, akkaConfig); + + logger.info("Actor system started at {}", AkkaUtils.getAddress(actorSystem)); + return actorSystem; + } + /** * Writes a Flink YAML config file from a Flink Configuration object. * @param cfg The Flink config @@ -401,7 +401,7 @@ public class BootstrapTools { startCommandValues.put("java", "$JAVA_HOME/bin/java"); final TaskExecutorProcessSpec taskExecutorProcessSpec = tmParams.getTaskExecutorProcessSpec(); - startCommandValues.put("jvmmem", TaskExecutorProcessUtils.generateJvmParametersStr(taskExecutorProcessSpec)); + startCommandValues.put("jvmmem", ProcessMemoryUtils.generateJvmParametersStr(taskExecutorProcessSpec)); String javaOpts = flinkConfig.getString(CoreOptions.FLINK_JVM_OPTIONS); if (flinkConfig.getString(CoreOptions.FLINK_TM_JVM_OPTIONS).length() > 0) { @@ -686,49 +686,4 @@ public class BootstrapTools { public static String escapeWithDoubleQuote(String value) { return "\"" + WINDOWS_DOUBLE_QUOTE_ESCAPER.escape(value) + "\""; } - - /** - * Calculate heap size after cut-off. The heap size after cut-off will be used to set -Xms and -Xmx for jobmanager - * start command. - */ - public static int calculateHeapSize(int memory, Configuration conf) { - - final float memoryCutoffRatio = conf.getFloat(ResourceManagerOptions.CONTAINERIZED_HEAP_CUTOFF_RATIO); - final int minCutoff = conf.getInteger(ResourceManagerOptions.CONTAINERIZED_HEAP_CUTOFF_MIN); - - if (memoryCutoffRatio > 1 || memoryCutoffRatio < 0) { - throw new IllegalArgumentException("The configuration value '" - + ResourceManagerOptions.CONTAINERIZED_HEAP_CUTOFF_RATIO.key() - + "' must be between 0 and 1. Value given=" + memoryCutoffRatio); - } - if (minCutoff > memory) { - throw new IllegalArgumentException("The configuration value '" - + ResourceManagerOptions.CONTAINERIZED_HEAP_CUTOFF_MIN.key() - + "' is higher (" + minCutoff + ") than the requested amount of memory " + memory); - } - - int heapLimit = (int) ((float) memory * memoryCutoffRatio); - if (heapLimit < minCutoff) { - heapLimit = minCutoff; - } - return memory - heapLimit; - } - - /** - * Method to extract environment variables from the flinkConfiguration based on the given prefix String. - * - * @param envPrefix Prefix for the environment variables key - * @param flinkConfiguration The Flink config to get the environment variable definition from - */ - public static Map getEnvironmentVariables(String envPrefix, Configuration flinkConfiguration) { - Map result = new HashMap<>(); - for (Map.Entry entry: flinkConfiguration.toMap().entrySet()) { - if (entry.getKey().startsWith(envPrefix) && entry.getKey().length() > envPrefix.length()) { - // remove prefix - String key = entry.getKey().substring(envPrefix.length()); - result.put(key, entry.getValue()); - } - } - return result; - } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContaineredTaskManagerParameters.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContaineredTaskManagerParameters.java index bfdaffdb5b5a16c1e44136732ab46b2948f38b84..70826ae68a925048ed2582d0d9725d2ab639110f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContaineredTaskManagerParameters.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContaineredTaskManagerParameters.java @@ -31,9 +31,6 @@ public class ContaineredTaskManagerParameters implements java.io.Serializable { private static final long serialVersionUID = -3096987654278064670L; - /** The number of slots per TaskManager. */ - private final int numSlots; - /** Environment variables to add to the Java process. */ private final HashMap taskManagerEnv; @@ -41,11 +38,9 @@ public class ContaineredTaskManagerParameters implements java.io.Serializable { public ContaineredTaskManagerParameters( TaskExecutorProcessSpec taskExecutorProcessSpec, - int numSlots, HashMap taskManagerEnv) { this.taskExecutorProcessSpec = taskExecutorProcessSpec; - this.numSlots = numSlots; this.taskManagerEnv = taskManagerEnv; } @@ -55,10 +50,6 @@ public class ContaineredTaskManagerParameters implements java.io.Serializable { return taskExecutorProcessSpec; } - public int numSlots() { - return numSlots; - } - public Map taskManagerEnv() { return taskManagerEnv; } @@ -70,7 +61,6 @@ public class ContaineredTaskManagerParameters implements java.io.Serializable { public String toString() { return "TaskManagerParameters {" + "taskExecutorProcessSpec=" + taskExecutorProcessSpec + - ", numSlots=" + numSlots + ", taskManagerEnv=" + taskManagerEnv + '}'; } @@ -84,13 +74,11 @@ public class ContaineredTaskManagerParameters implements java.io.Serializable { * * @param config The Flink configuration. * @param taskExecutorProcessSpec The resource specifics of the task executor. - * @param numSlots Number of slots of the task executor. * @return The parameters to start the TaskManager processes with. */ public static ContaineredTaskManagerParameters create( Configuration config, - TaskExecutorProcessSpec taskExecutorProcessSpec, - int numSlots) { + TaskExecutorProcessSpec taskExecutorProcessSpec) { // obtain the additional environment variables from the configuration final HashMap envVars = new HashMap<>(); @@ -105,7 +93,6 @@ public class ContaineredTaskManagerParameters implements java.io.Serializable { } // done - return new ContaineredTaskManagerParameters( - taskExecutorProcessSpec, numSlots, envVars); + return new ContaineredTaskManagerParameters(taskExecutorProcessSpec, envVars); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/TaskExecutorProcessSpec.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/TaskExecutorProcessSpec.java index e530b8ee27a5e02e5df65b80a388e77834ac72e1..4b7a85611505bdcddb32cbcdc8db184d94500076 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/TaskExecutorProcessSpec.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/TaskExecutorProcessSpec.java @@ -18,10 +18,12 @@ package org.apache.flink.runtime.clusterframework; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.resources.CPUResource; import org.apache.flink.configuration.MemorySize; - -import java.io.Serializable; +import org.apache.flink.runtime.util.config.memory.CommonProcessMemorySpec; +import org.apache.flink.runtime.util.config.memory.JvmMetaspaceAndOverhead; +import org.apache.flink.runtime.util.config.memory.taskmanager.TaskExecutorFlinkMemory; /** * Describe the specifics of different resource dimensions of the TaskExecutor process. @@ -73,26 +75,12 @@ import java.io.Serializable; * └ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┘ * */ -public class TaskExecutorProcessSpec implements Serializable { +public class TaskExecutorProcessSpec extends CommonProcessMemorySpec { + private static final long serialVersionUID = 1L; private final CPUResource cpuCores; - private final MemorySize frameworkHeapSize; - - private final MemorySize frameworkOffHeapMemorySize; - - private final MemorySize taskHeapSize; - - private final MemorySize taskOffHeapSize; - - private final MemorySize networkMemSize; - - private final MemorySize managedMemorySize; - - private final MemorySize jvmMetaspaceSize; - - private final MemorySize jvmOverheadSize; - + @VisibleForTesting public TaskExecutorProcessSpec( CPUResource cpuCores, MemorySize frameworkHeapSize, @@ -104,81 +92,67 @@ public class TaskExecutorProcessSpec implements Serializable { MemorySize jvmMetaspaceSize, MemorySize jvmOverheadSize) { + this( + cpuCores, + new TaskExecutorFlinkMemory( + frameworkHeapSize, + frameworkOffHeapSize, + taskHeapSize, + taskOffHeapSize, + networkMemSize, + managedMemorySize), + new JvmMetaspaceAndOverhead(jvmMetaspaceSize, jvmOverheadSize)); + } + + protected TaskExecutorProcessSpec( + CPUResource cpuCores, + TaskExecutorFlinkMemory flinkMemory, + JvmMetaspaceAndOverhead jvmMetaspaceAndOverhead) { + + super(flinkMemory, jvmMetaspaceAndOverhead); this.cpuCores = cpuCores; - this.frameworkHeapSize = frameworkHeapSize; - this.frameworkOffHeapMemorySize = frameworkOffHeapSize; - this.taskHeapSize = taskHeapSize; - this.taskOffHeapSize = taskOffHeapSize; - this.networkMemSize = networkMemSize; - this.managedMemorySize = managedMemorySize; - this.jvmMetaspaceSize = jvmMetaspaceSize; - this.jvmOverheadSize = jvmOverheadSize; } public CPUResource getCpuCores() { return cpuCores; } - public MemorySize getFrameworkHeapSize() { - return frameworkHeapSize; + MemorySize getFrameworkHeapSize() { + return getFlinkMemory().getFrameworkHeap(); } - public MemorySize getFrameworkOffHeapMemorySize() { - return frameworkOffHeapMemorySize; + MemorySize getFrameworkOffHeapMemorySize() { + return getFlinkMemory().getFrameworkOffHeap(); } public MemorySize getTaskHeapSize() { - return taskHeapSize; + return getFlinkMemory().getTaskHeap(); } public MemorySize getTaskOffHeapSize() { - return taskOffHeapSize; + return getFlinkMemory().getTaskOffHeap(); } public MemorySize getNetworkMemSize() { - return networkMemSize; + return getFlinkMemory().getNetwork(); } public MemorySize getManagedMemorySize() { - return managedMemorySize; - } - - public MemorySize getJvmMetaspaceSize() { - return jvmMetaspaceSize; - } - - public MemorySize getJvmOverheadSize() { - return jvmOverheadSize; - } - - public MemorySize getTotalFlinkMemorySize() { - return frameworkHeapSize.add(frameworkOffHeapMemorySize).add(taskHeapSize).add(taskOffHeapSize).add(networkMemSize).add(getManagedMemorySize()); - } - - public MemorySize getTotalProcessMemorySize() { - return getTotalFlinkMemorySize().add(jvmMetaspaceSize).add(jvmOverheadSize); - } - - public MemorySize getJvmHeapMemorySize() { - return frameworkHeapSize.add(taskHeapSize); - } - - public MemorySize getJvmDirectMemorySize() { - return frameworkOffHeapMemorySize.add(taskOffHeapSize).add(networkMemSize); + return getFlinkMemory().getManaged(); } @Override public String toString() { return "TaskExecutorProcessSpec {" + "cpuCores=" + cpuCores.getValue().doubleValue() - + ", frameworkHeapSize=" + frameworkHeapSize.toHumanReadableString() - + ", frameworkOffHeapSize=" + frameworkOffHeapMemorySize.toHumanReadableString() - + ", taskHeapSize=" + taskHeapSize.toHumanReadableString() - + ", taskOffHeapSize=" + taskOffHeapSize.toHumanReadableString() - + ", networkMemSize=" + networkMemSize.toHumanReadableString() - + ", managedMemorySize=" + managedMemorySize.toHumanReadableString() - + ", jvmMetaspaceSize=" + jvmMetaspaceSize.toHumanReadableString() - + ", jvmOverheadSize=" + jvmOverheadSize.toHumanReadableString() - + "}"; + + ", frameworkHeapSize=" + getFrameworkHeapSize().toHumanReadableString() + + ", frameworkOffHeapSize=" + getFrameworkOffHeapMemorySize().toHumanReadableString() + + ", taskHeapSize=" + getTaskHeapSize().toHumanReadableString() + + ", taskOffHeapSize=" + getTaskOffHeapSize().toHumanReadableString() + + ", networkMemSize=" + getNetworkMemSize().toHumanReadableString() + + ", managedMemorySize=" + getManagedMemorySize().toHumanReadableString() + + ", jvmMetaspaceSize=" + getJvmMetaspaceSize().toHumanReadableString() + + ", jvmOverheadSize=" + getJvmOverheadSize().toHumanReadableString() + + '}'; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/TaskExecutorProcessUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/TaskExecutorProcessUtils.java index 9b7ae12512d8807577c4d14254ffeaab3a56a80d..e1794f8ec36a249fd33989ff8ce6ce5dea454b5b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/TaskExecutorProcessUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/TaskExecutorProcessUtils.java @@ -23,22 +23,21 @@ import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.IllegalConfigurationException; import org.apache.flink.configuration.MemorySize; -import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; import org.apache.flink.configuration.TaskManagerOptions; -import org.apache.flink.runtime.clusterframework.types.ResourceProfile; -import org.apache.flink.runtime.util.ConfigurationParserUtils; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Collections; +import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec; +import org.apache.flink.runtime.util.config.memory.CommonProcessMemorySpec; +import org.apache.flink.runtime.util.config.memory.JvmMetaspaceAndOverhead; +import org.apache.flink.runtime.util.config.memory.JvmMetaspaceAndOverheadOptions; +import org.apache.flink.runtime.util.config.memory.LegacyMemoryOptions; +import org.apache.flink.runtime.util.config.memory.MemoryBackwardsCompatibilityUtils; +import org.apache.flink.runtime.util.config.memory.ProcessMemoryOptions; +import org.apache.flink.runtime.util.config.memory.ProcessMemoryUtils; +import org.apache.flink.runtime.util.config.memory.taskmanager.TaskExecutorFlinkMemory; +import org.apache.flink.runtime.util.config.memory.taskmanager.TaskExecutorFlinkMemoryUtils; + +import java.util.Arrays; import java.util.HashMap; -import java.util.List; import java.util.Map; -import java.util.Optional; - -import static org.apache.flink.util.Preconditions.checkArgument; -import static org.apache.flink.util.Preconditions.checkNotNull; /** * Utility class for TaskExecutor memory configurations. @@ -46,24 +45,31 @@ import static org.apache.flink.util.Preconditions.checkNotNull; *

See {@link TaskExecutorProcessSpec} for details about memory components of TaskExecutor and their relationships. */ public class TaskExecutorProcessUtils { - private static final Logger LOG = LoggerFactory.getLogger(TaskExecutorProcessUtils.class); - private TaskExecutorProcessUtils() {} + static final ProcessMemoryOptions TM_PROCESS_MEMORY_OPTIONS = new ProcessMemoryOptions( + Arrays.asList(TaskManagerOptions.TASK_HEAP_MEMORY, TaskManagerOptions.MANAGED_MEMORY_SIZE), + TaskManagerOptions.TOTAL_FLINK_MEMORY, + TaskManagerOptions.TOTAL_PROCESS_MEMORY, + new JvmMetaspaceAndOverheadOptions( + TaskManagerOptions.JVM_METASPACE, + TaskManagerOptions.JVM_OVERHEAD_MIN, + TaskManagerOptions.JVM_OVERHEAD_MAX, + TaskManagerOptions.JVM_OVERHEAD_FRACTION)); - // ------------------------------------------------------------------------ - // Generating JVM Parameters - // ------------------------------------------------------------------------ + @SuppressWarnings("deprecation") + static final LegacyMemoryOptions TM_LEGACY_HEAP_OPTIONS = + new LegacyMemoryOptions( + "FLINK_TM_HEAP", + TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY, + TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY_MB); - public static String generateJvmParametersStr(final TaskExecutorProcessSpec taskExecutorProcessSpec) { - final MemorySize jvmHeapSize = taskExecutorProcessSpec.getJvmHeapMemorySize(); - final MemorySize jvmDirectSize = taskExecutorProcessSpec.getJvmDirectMemorySize(); - final MemorySize jvmMetaspaceSize = taskExecutorProcessSpec.getJvmMetaspaceSize(); + private static final ProcessMemoryUtils PROCESS_MEMORY_UTILS = new ProcessMemoryUtils<>( + TM_PROCESS_MEMORY_OPTIONS, + new TaskExecutorFlinkMemoryUtils()); - return "-Xmx" + jvmHeapSize.getBytes() - + " -Xms" + jvmHeapSize.getBytes() - + " -XX:MaxDirectMemorySize=" + jvmDirectSize.getBytes() - + " -XX:MaxMetaspaceSize=" + jvmMetaspaceSize.getBytes(); - } + private static final MemoryBackwardsCompatibilityUtils LEGACY_MEMORY_UTILS = new MemoryBackwardsCompatibilityUtils(TM_LEGACY_HEAP_OPTIONS); + + private TaskExecutorProcessUtils() {} // ------------------------------------------------------------------------ // Generating Dynamic Config Options @@ -91,40 +97,6 @@ public class TaskExecutorProcessUtils { return sb.toString(); } - // ------------------------------------------------------------------------ - // Generating Slot Resource Profiles - // ------------------------------------------------------------------------ - - public static List createDefaultWorkerSlotProfiles( - TaskExecutorProcessSpec taskExecutorProcessSpec, - int numberOfSlots) { - final ResourceProfile resourceProfile = - generateDefaultSlotResourceProfile(taskExecutorProcessSpec, numberOfSlots); - return Collections.nCopies(numberOfSlots, resourceProfile); - } - - public static ResourceProfile generateDefaultSlotResourceProfile( - TaskExecutorProcessSpec taskExecutorProcessSpec, - int numberOfSlots) { - return ResourceProfile.newBuilder() - .setCpuCores(taskExecutorProcessSpec.getCpuCores().divide(numberOfSlots)) - .setTaskHeapMemory(taskExecutorProcessSpec.getTaskHeapSize().divide(numberOfSlots)) - .setTaskOffHeapMemory(taskExecutorProcessSpec.getTaskOffHeapSize().divide(numberOfSlots)) - .setManagedMemory(taskExecutorProcessSpec.getManagedMemorySize().divide(numberOfSlots)) - .setNetworkMemory(taskExecutorProcessSpec.getNetworkMemSize().divide(numberOfSlots)) - .build(); - } - - public static ResourceProfile generateTotalAvailableResourceProfile(TaskExecutorProcessSpec taskExecutorProcessSpec) { - return ResourceProfile.newBuilder() - .setCpuCores(taskExecutorProcessSpec.getCpuCores()) - .setTaskHeapMemory(taskExecutorProcessSpec.getTaskHeapSize()) - .setTaskOffHeapMemory(taskExecutorProcessSpec.getTaskOffHeapSize()) - .setManagedMemory(taskExecutorProcessSpec.getManagedMemorySize()) - .setNetworkMemory(taskExecutorProcessSpec.getNetworkMemSize()) - .build(); - } - // ------------------------------------------------------------------------ // Memory Configuration Calculations // ------------------------------------------------------------------------ @@ -134,510 +106,36 @@ public class TaskExecutorProcessUtils { } public static TaskExecutorProcessSpec processSpecFromConfig(final Configuration config) { - if (isTaskHeapMemorySizeExplicitlyConfigured(config) && isManagedMemorySizeExplicitlyConfigured(config)) { - // both task heap memory and managed memory are configured, use these to derive total flink memory - return deriveProcessSpecWithExplicitTaskAndManagedMemory(config); - } else if (isTotalFlinkMemorySizeExplicitlyConfigured(config)) { - // either of task heap memory and managed memory is not configured, total flink memory is configured, - // derive from total flink memory - return deriveProcessSpecWithTotalFlinkMemory(config); - } else if (isTotalProcessMemorySizeExplicitlyConfigured(config)) { - // total flink memory is not configured, total process memory is configured, - // derive from total process memory - return deriveProcessSpecWithTotalProcessMemory(config); - } else { - throw new IllegalConfigurationException(String.format("Either Task Heap Memory size (%s) and Managed Memory size (%s), or Total Flink" - + " Memory size (%s), or Total Process Memory size (%s) need to be configured explicitly.", - TaskManagerOptions.TASK_HEAP_MEMORY.key(), - TaskManagerOptions.MANAGED_MEMORY_SIZE.key(), - TaskManagerOptions.TOTAL_FLINK_MEMORY.key(), - TaskManagerOptions.TOTAL_PROCESS_MEMORY.key())); - } - } - - public static boolean isTaskExecutorProcessResourceExplicitlyConfigured(final Configuration config) { - return (isTaskHeapMemorySizeExplicitlyConfigured(config) && isManagedMemorySizeExplicitlyConfigured(config)) - || isTotalFlinkMemorySizeExplicitlyConfigured(config) - || isTotalProcessMemorySizeExplicitlyConfigured(config); + return createMemoryProcessSpec(config, PROCESS_MEMORY_UTILS.memoryProcessSpecFromConfig(config)); } - private static TaskExecutorProcessSpec deriveProcessSpecWithExplicitTaskAndManagedMemory(final Configuration config) { - // derive flink internal memory from explicitly configure task heap memory size and managed memory size + public static TaskExecutorProcessSpec processSpecFromWorkerResourceSpec( + final Configuration config, final WorkerResourceSpec workerResourceSpec) { - final MemorySize taskHeapMemorySize = getTaskHeapMemorySize(config); - final MemorySize managedMemorySize = getManagedMemorySize(config); + final MemorySize frameworkHeapMemorySize = TaskExecutorFlinkMemoryUtils.getFrameworkHeapMemorySize(config); + final MemorySize frameworkOffHeapMemorySize = TaskExecutorFlinkMemoryUtils.getFrameworkOffHeapMemorySize(config); - final MemorySize frameworkHeapMemorySize = getFrameworkHeapMemorySize(config); - final MemorySize frameworkOffHeapMemorySize = getFrameworkOffHeapMemorySize(config); - final MemorySize taskOffHeapMemorySize = getTaskOffHeapMemorySize(config); - - final MemorySize networkMemorySize; - final MemorySize totalFlinkExcludeNetworkMemorySize = - frameworkHeapMemorySize.add(frameworkOffHeapMemorySize).add(taskHeapMemorySize).add(taskOffHeapMemorySize).add(managedMemorySize); - - if (isTotalFlinkMemorySizeExplicitlyConfigured(config)) { - // derive network memory from total flink memory, and check against network min/max - final MemorySize totalFlinkMemorySize = getTotalFlinkMemorySize(config); - if (totalFlinkExcludeNetworkMemorySize.getBytes() > totalFlinkMemorySize.getBytes()) { - throw new IllegalConfigurationException( - "Sum of configured Framework Heap Memory (" + frameworkHeapMemorySize.toHumanReadableString() - + "), Framework Off-Heap Memory (" + frameworkOffHeapMemorySize.toHumanReadableString() - + "), Task Heap Memory (" + taskHeapMemorySize.toHumanReadableString() - + "), Task Off-Heap Memory (" + taskOffHeapMemorySize.toHumanReadableString() - + ") and Managed Memory (" + managedMemorySize.toHumanReadableString() - + ") exceed configured Total Flink Memory (" + totalFlinkMemorySize.toHumanReadableString() + ")."); - } - networkMemorySize = totalFlinkMemorySize.subtract(totalFlinkExcludeNetworkMemorySize); - sanityCheckNetworkMemoryWithExplicitlySetTotalFlinkAndHeapMemory(config, networkMemorySize, totalFlinkMemorySize); - } else { - // derive network memory from network configs - if (isUsingLegacyNetworkConfigs(config)) { - networkMemorySize = getNetworkMemorySizeWithLegacyConfig(config); - } else { - networkMemorySize = deriveNetworkMemoryWithInverseFraction(config, totalFlinkExcludeNetworkMemorySize); - } - } - - final FlinkInternalMemory flinkInternalMemory = new FlinkInternalMemory( + final TaskExecutorFlinkMemory flinkMemory = new TaskExecutorFlinkMemory( frameworkHeapMemorySize, frameworkOffHeapMemorySize, - taskHeapMemorySize, - taskOffHeapMemorySize, - networkMemorySize, - managedMemorySize); - sanityCheckTotalFlinkMemory(config, flinkInternalMemory); - - // derive jvm metaspace and overhead - - final JvmMetaspaceAndOverhead jvmMetaspaceAndOverhead = deriveJvmMetaspaceAndOverheadFromTotalFlinkMemory(config, flinkInternalMemory.getTotalFlinkMemorySize()); - - return createTaskExecutorProcessSpec(config, flinkInternalMemory, jvmMetaspaceAndOverhead); - } - - private static TaskExecutorProcessSpec deriveProcessSpecWithTotalFlinkMemory(final Configuration config) { - // derive flink internal memory from explicitly configured total flink memory - - final MemorySize totalFlinkMemorySize = getTotalFlinkMemorySize(config); - final FlinkInternalMemory flinkInternalMemory = deriveInternalMemoryFromTotalFlinkMemory(config, totalFlinkMemorySize); - - // derive jvm metaspace and overhead + workerResourceSpec.getTaskHeapSize(), + workerResourceSpec.getTaskOffHeapSize(), + workerResourceSpec.getNetworkMemSize(), + workerResourceSpec.getManagedMemSize()); - final JvmMetaspaceAndOverhead jvmMetaspaceAndOverhead = deriveJvmMetaspaceAndOverheadFromTotalFlinkMemory(config, totalFlinkMemorySize); + final JvmMetaspaceAndOverhead jvmMetaspaceAndOverhead = + PROCESS_MEMORY_UTILS.deriveJvmMetaspaceAndOverheadFromTotalFlinkMemory( + config, flinkMemory.getTotalFlinkMemorySize()); - return createTaskExecutorProcessSpec(config, flinkInternalMemory, jvmMetaspaceAndOverhead); + return new TaskExecutorProcessSpec(workerResourceSpec.getCpuCores(), flinkMemory, jvmMetaspaceAndOverhead); } - private static TaskExecutorProcessSpec deriveProcessSpecWithTotalProcessMemory(final Configuration config) { - // derive total flink memory from explicitly configured total process memory size - - final MemorySize totalProcessMemorySize = getTotalProcessMemorySize(config); - final MemorySize jvmMetaspaceSize = getJvmMetaspaceSize(config); - final MemorySize jvmOverheadSize = deriveJvmOverheadWithFraction(config, totalProcessMemorySize); - final JvmMetaspaceAndOverhead jvmMetaspaceAndOverhead = new JvmMetaspaceAndOverhead(jvmMetaspaceSize, jvmOverheadSize); - - if (jvmMetaspaceAndOverhead.getTotalJvmMetaspaceAndOverheadSize().getBytes() > totalProcessMemorySize.getBytes()) { - throw new IllegalConfigurationException( - "Sum of configured JVM Metaspace (" + jvmMetaspaceAndOverhead.metaspace.toHumanReadableString() - + ") and JVM Overhead (" + jvmMetaspaceAndOverhead.overhead.toHumanReadableString() - + ") exceed configured Total Process Memory (" + totalProcessMemorySize.toHumanReadableString() + ")."); - } - final MemorySize totalFlinkMemorySize = totalProcessMemorySize.subtract(jvmMetaspaceAndOverhead.getTotalJvmMetaspaceAndOverheadSize()); - - // derive flink internal memory - - final FlinkInternalMemory flinkInternalMemory = deriveInternalMemoryFromTotalFlinkMemory(config, totalFlinkMemorySize); - - return createTaskExecutorProcessSpec(config, flinkInternalMemory, jvmMetaspaceAndOverhead); - } - - private static JvmMetaspaceAndOverhead deriveJvmMetaspaceAndOverheadFromTotalFlinkMemory( + private static TaskExecutorProcessSpec createMemoryProcessSpec( final Configuration config, - final MemorySize totalFlinkMemorySize) { - final MemorySize jvmMetaspaceSize = getJvmMetaspaceSize(config); - final MemorySize totalFlinkAndJvmMetaspaceSize = totalFlinkMemorySize.add(jvmMetaspaceSize); - final JvmMetaspaceAndOverhead jvmMetaspaceAndOverhead; - if (isTotalProcessMemorySizeExplicitlyConfigured(config)) { - final MemorySize totalProcessMemorySize = getTotalProcessMemorySize(config); - final MemorySize jvmOverheadSize = totalProcessMemorySize.subtract(totalFlinkAndJvmMetaspaceSize); - sanityCheckJvmOverhead(config, jvmOverheadSize, totalProcessMemorySize); - jvmMetaspaceAndOverhead = new JvmMetaspaceAndOverhead(jvmMetaspaceSize, jvmOverheadSize); - } else { - final MemorySize jvmOverheadSize = deriveJvmOverheadWithInverseFraction(config, totalFlinkAndJvmMetaspaceSize); - jvmMetaspaceAndOverhead = new JvmMetaspaceAndOverhead(jvmMetaspaceSize, jvmOverheadSize); - sanityCheckTotalProcessMemory(config, totalFlinkMemorySize, jvmMetaspaceAndOverhead); - } - return jvmMetaspaceAndOverhead; - } - - private static FlinkInternalMemory deriveInternalMemoryFromTotalFlinkMemory( - final Configuration config, - final MemorySize totalFlinkMemorySize) { - final MemorySize frameworkHeapMemorySize = getFrameworkHeapMemorySize(config); - final MemorySize frameworkOffHeapMemorySize = getFrameworkOffHeapMemorySize(config); - final MemorySize taskOffHeapMemorySize = getTaskOffHeapMemorySize(config); - - final MemorySize taskHeapMemorySize; - final MemorySize networkMemorySize; - final MemorySize managedMemorySize; - - if (isTaskHeapMemorySizeExplicitlyConfigured(config)) { - // task heap memory is configured, - // derive managed memory first, leave the remaining to network memory and check against network min/max - taskHeapMemorySize = getTaskHeapMemorySize(config); - managedMemorySize = deriveManagedMemoryAbsoluteOrWithFraction(config, totalFlinkMemorySize); - final MemorySize totalFlinkExcludeNetworkMemorySize = - frameworkHeapMemorySize.add(frameworkOffHeapMemorySize).add(taskHeapMemorySize).add(taskOffHeapMemorySize).add(managedMemorySize); - if (totalFlinkExcludeNetworkMemorySize.getBytes() > totalFlinkMemorySize.getBytes()) { - throw new IllegalConfigurationException( - "Sum of configured Framework Heap Memory (" + frameworkHeapMemorySize.toHumanReadableString() - + "), Framework Off-Heap Memory (" + frameworkOffHeapMemorySize.toHumanReadableString() - + "), Task Heap Memory (" + taskHeapMemorySize.toHumanReadableString() - + "), Task Off-Heap Memory (" + taskOffHeapMemorySize.toHumanReadableString() - + ") and Managed Memory (" + managedMemorySize.toHumanReadableString() - + ") exceed configured Total Flink Memory (" + totalFlinkMemorySize.toHumanReadableString() + ")."); - } - networkMemorySize = totalFlinkMemorySize.subtract(totalFlinkExcludeNetworkMemorySize); - sanityCheckNetworkMemoryWithExplicitlySetTotalFlinkAndHeapMemory(config, networkMemorySize, totalFlinkMemorySize); - } else { - // task heap memory is not configured - // derive managed memory and network memory, leave the remaining to task heap memory - managedMemorySize = deriveManagedMemoryAbsoluteOrWithFraction(config, totalFlinkMemorySize); - - if (isUsingLegacyNetworkConfigs(config)) { - networkMemorySize = getNetworkMemorySizeWithLegacyConfig(config); - } else { - networkMemorySize = deriveNetworkMemoryWithFraction(config, totalFlinkMemorySize); - } - final MemorySize totalFlinkExcludeTaskHeapMemorySize = - frameworkHeapMemorySize.add(frameworkOffHeapMemorySize).add(taskOffHeapMemorySize).add(managedMemorySize).add(networkMemorySize); - if (totalFlinkExcludeTaskHeapMemorySize.getBytes() > totalFlinkMemorySize.getBytes()) { - throw new IllegalConfigurationException( - "Sum of configured Framework Heap Memory (" + frameworkHeapMemorySize.toHumanReadableString() - + "), Framework Off-Heap Memory (" + frameworkOffHeapMemorySize.toHumanReadableString() - + "), Task Off-Heap Memory (" + taskOffHeapMemorySize.toHumanReadableString() - + "), Managed Memory (" + managedMemorySize.toHumanReadableString() - + ") and Network Memory (" + networkMemorySize.toHumanReadableString() - + ") exceed configured Total Flink Memory (" + totalFlinkMemorySize.toHumanReadableString() + ")."); - } - taskHeapMemorySize = totalFlinkMemorySize.subtract(totalFlinkExcludeTaskHeapMemorySize); - } - - final FlinkInternalMemory flinkInternalMemory = new FlinkInternalMemory( - frameworkHeapMemorySize, - frameworkOffHeapMemorySize, - taskHeapMemorySize, - taskOffHeapMemorySize, - networkMemorySize, - managedMemorySize); - sanityCheckTotalFlinkMemory(config, flinkInternalMemory); - - return flinkInternalMemory; - } - - private static MemorySize deriveManagedMemoryAbsoluteOrWithFraction(final Configuration config, final MemorySize base) { - if (isManagedMemorySizeExplicitlyConfigured(config)) { - return getManagedMemorySize(config); - } else { - return deriveWithFraction("managed memory", base, getManagedMemoryRangeFraction(config)); - } - } - - private static MemorySize deriveNetworkMemoryWithFraction(final Configuration config, final MemorySize base) { - return deriveWithFraction("network memory", base, getNetworkMemoryRangeFraction(config)); - } - - private static MemorySize deriveNetworkMemoryWithInverseFraction(final Configuration config, final MemorySize base) { - return deriveWithInverseFraction("network memory", base, getNetworkMemoryRangeFraction(config)); - } - - private static MemorySize deriveJvmOverheadWithFraction(final Configuration config, final MemorySize base) { - return deriveWithFraction("jvm overhead memory", base, getJvmOverheadRangeFraction(config)); - } - - private static MemorySize deriveJvmOverheadWithInverseFraction(final Configuration config, final MemorySize base) { - return deriveWithInverseFraction("jvm overhead memory", base, getJvmOverheadRangeFraction(config)); - } - - private static MemorySize deriveWithFraction( - final String memoryDescription, - final MemorySize base, - final RangeFraction rangeFraction) { - final MemorySize relative = base.multiply(rangeFraction.fraction); - return capToMinMax(memoryDescription, relative, rangeFraction); - } - - private static MemorySize deriveWithInverseFraction( - final String memoryDescription, - final MemorySize base, - final RangeFraction rangeFraction) { - checkArgument(rangeFraction.fraction < 1); - final MemorySize relative = base.multiply(rangeFraction.fraction / (1 - rangeFraction.fraction)); - return capToMinMax(memoryDescription, relative, rangeFraction); - } - - private static MemorySize capToMinMax( - final String memoryDescription, - final MemorySize relative, - final RangeFraction rangeFraction) { - long size = relative.getBytes(); - if (size > rangeFraction.maxSize.getBytes()) { - LOG.info( - "The derived from fraction {} ({}) is greater than its max value {}, max value will be used instead", - memoryDescription, - relative.toHumanReadableString(), - rangeFraction.maxSize.toHumanReadableString()); - size = rangeFraction.maxSize.getBytes(); - } else if (size < rangeFraction.minSize.getBytes()) { - LOG.info( - "The derived from fraction {} ({}) is less than its min value {}, min value will be used instead", - memoryDescription, - relative.toHumanReadableString(), - rangeFraction.minSize.toHumanReadableString()); - size = rangeFraction.minSize.getBytes(); - } - return new MemorySize(size); - } - - private static MemorySize getFrameworkHeapMemorySize(final Configuration config) { - return getMemorySizeFromConfig(config, TaskManagerOptions.FRAMEWORK_HEAP_MEMORY); - } - - private static MemorySize getFrameworkOffHeapMemorySize(final Configuration config) { - return getMemorySizeFromConfig(config, TaskManagerOptions.FRAMEWORK_OFF_HEAP_MEMORY); - } - - private static MemorySize getTaskHeapMemorySize(final Configuration config) { - checkArgument(isTaskHeapMemorySizeExplicitlyConfigured(config)); - return getMemorySizeFromConfig(config, TaskManagerOptions.TASK_HEAP_MEMORY); - } - - private static MemorySize getTaskOffHeapMemorySize(final Configuration config) { - return getMemorySizeFromConfig(config, TaskManagerOptions.TASK_OFF_HEAP_MEMORY); - } - - private static MemorySize getManagedMemorySize(final Configuration config) { - checkArgument(isManagedMemorySizeExplicitlyConfigured(config)); - return getMemorySizeFromConfig(config, TaskManagerOptions.MANAGED_MEMORY_SIZE); - } - - private static RangeFraction getManagedMemoryRangeFraction(final Configuration config) { - return getRangeFraction(MemorySize.ZERO, MemorySize.MAX_VALUE, TaskManagerOptions.MANAGED_MEMORY_FRACTION, config); - } - - private static MemorySize getNetworkMemorySizeWithLegacyConfig(final Configuration config) { - checkArgument(isUsingLegacyNetworkConfigs(config)); - @SuppressWarnings("deprecation") - final long numOfBuffers = config.getInteger(NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS); - final long pageSize = ConfigurationParserUtils.getPageSize(config); - return new MemorySize(numOfBuffers * pageSize); - } - - private static RangeFraction getNetworkMemoryRangeFraction(final Configuration config) { - final MemorySize minSize = getMemorySizeFromConfig(config, TaskManagerOptions.NETWORK_MEMORY_MIN); - final MemorySize maxSize = getMemorySizeFromConfig(config, TaskManagerOptions.NETWORK_MEMORY_MAX); - return getRangeFraction(minSize, maxSize, TaskManagerOptions.NETWORK_MEMORY_FRACTION, config); - } - - private static MemorySize getJvmMetaspaceSize(final Configuration config) { - return getMemorySizeFromConfig(config, TaskManagerOptions.JVM_METASPACE); - } - - private static RangeFraction getJvmOverheadRangeFraction(final Configuration config) { - final MemorySize minSize = getMemorySizeFromConfig(config, TaskManagerOptions.JVM_OVERHEAD_MIN); - final MemorySize maxSize = getMemorySizeFromConfig(config, TaskManagerOptions.JVM_OVERHEAD_MAX); - return getRangeFraction(minSize, maxSize, TaskManagerOptions.JVM_OVERHEAD_FRACTION, config); - } - - private static RangeFraction getRangeFraction( - final MemorySize minSize, - final MemorySize maxSize, - ConfigOption fractionOption, - final Configuration config) { - final double fraction = config.getFloat(fractionOption); - try { - return new RangeFraction(minSize, maxSize, fraction); - } catch (IllegalArgumentException e) { - throw new IllegalConfigurationException( - String.format( - "Inconsistently configured %s (%s) and its min (%s), max (%s) value", - fractionOption, - fraction, - minSize.toHumanReadableString(), - maxSize.toHumanReadableString()), - e); - } - } - - private static MemorySize getTotalFlinkMemorySize(final Configuration config) { - checkArgument(isTotalFlinkMemorySizeExplicitlyConfigured(config)); - return getMemorySizeFromConfig(config, TaskManagerOptions.TOTAL_FLINK_MEMORY); - } - - private static MemorySize getTotalProcessMemorySize(final Configuration config) { - checkArgument(isTotalProcessMemorySizeExplicitlyConfigured(config)); - return getMemorySizeFromConfig(config, TaskManagerOptions.TOTAL_PROCESS_MEMORY); - } - - private static MemorySize getMemorySizeFromConfig(final Configuration config, final ConfigOption option) { - try { - return config.get(option); - } catch (Throwable t) { - throw new IllegalConfigurationException("Cannot read memory size from config option '" + option.key() + "'.", t); - } - } - - private static boolean isTaskHeapMemorySizeExplicitlyConfigured(final Configuration config) { - return config.contains(TaskManagerOptions.TASK_HEAP_MEMORY); - } - - public static boolean isManagedMemorySizeExplicitlyConfigured(final Configuration config) { - return config.contains(TaskManagerOptions.MANAGED_MEMORY_SIZE); - } - - private static boolean isUsingLegacyNetworkConfigs(final Configuration config) { - // use the legacy number-of-buffer config option only when it is explicitly configured and - // none of new config options is explicitly configured - @SuppressWarnings("deprecation") - final boolean legacyConfigured = config.contains(NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS); - return !config.contains(TaskManagerOptions.NETWORK_MEMORY_MIN) && - !config.contains(TaskManagerOptions.NETWORK_MEMORY_MAX) && - !config.contains(TaskManagerOptions.NETWORK_MEMORY_FRACTION) && - legacyConfigured; - } - - private static boolean isNetworkMemoryFractionExplicitlyConfigured(final Configuration config) { - return config.contains(TaskManagerOptions.NETWORK_MEMORY_FRACTION); - } - - public static boolean isNetworkMemoryExplicitlyConfigured(final Configuration config) { - @SuppressWarnings("deprecation") - final boolean legacyConfigured = config.contains(NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS); - return config.contains(TaskManagerOptions.NETWORK_MEMORY_MAX) || - config.contains(TaskManagerOptions.NETWORK_MEMORY_MIN) || - config.contains(TaskManagerOptions.NETWORK_MEMORY_FRACTION) || - legacyConfigured; - } - - private static boolean isJvmOverheadFractionExplicitlyConfigured(final Configuration config) { - return config.contains(TaskManagerOptions.JVM_OVERHEAD_FRACTION); - } - - private static boolean isTotalFlinkMemorySizeExplicitlyConfigured(final Configuration config) { - return config.contains(TaskManagerOptions.TOTAL_FLINK_MEMORY); - } - - private static boolean isTotalProcessMemorySizeExplicitlyConfigured(final Configuration config) { - return config.contains(TaskManagerOptions.TOTAL_PROCESS_MEMORY); - } - - private static void sanityCheckTotalFlinkMemory(final Configuration config, final FlinkInternalMemory flinkInternalMemory) { - if (isTotalFlinkMemorySizeExplicitlyConfigured(config)) { - final MemorySize configuredTotalFlinkMemorySize = getTotalFlinkMemorySize(config); - if (!configuredTotalFlinkMemorySize.equals(flinkInternalMemory.getTotalFlinkMemorySize())) { - throw new IllegalConfigurationException( - "Configured/Derived Flink internal memory sizes (total " + flinkInternalMemory.getTotalFlinkMemorySize().toHumanReadableString() - + ") do not add up to the configured Total Flink Memory size (" + configuredTotalFlinkMemorySize.toHumanReadableString() - + "). Configured/Derived Flink internal memory sizes are: " - + "Framework Heap Memory (" + flinkInternalMemory.frameworkHeap.toHumanReadableString() - + "), Framework Off-Heap Memory (" + flinkInternalMemory.frameworkOffHeap.toHumanReadableString() - + "), Task Heap Memory (" + flinkInternalMemory.taskHeap.toHumanReadableString() - + "), Task Off-Heap Memory (" + flinkInternalMemory.taskOffHeap.toHumanReadableString() - + "), Network Memory (" + flinkInternalMemory.network.toHumanReadableString() - + "), Managed Memory (" + flinkInternalMemory.managed.toHumanReadableString() + ")."); - } - } - } - - private static void sanityCheckTotalProcessMemory( - final Configuration config, - final MemorySize totalFlinkMemory, - final JvmMetaspaceAndOverhead jvmMetaspaceAndOverhead) { - final MemorySize derivedTotalProcessMemorySize = - totalFlinkMemory.add(jvmMetaspaceAndOverhead.metaspace).add(jvmMetaspaceAndOverhead.overhead); - if (isTotalProcessMemorySizeExplicitlyConfigured(config)) { - final MemorySize configuredTotalProcessMemorySize = getTotalProcessMemorySize(config); - if (!configuredTotalProcessMemorySize.equals(derivedTotalProcessMemorySize)) { - throw new IllegalConfigurationException( - "Configured/Derived memory sizes (total " + derivedTotalProcessMemorySize.toHumanReadableString() - + ") do not add up to the configured Total Process Memory size (" + configuredTotalProcessMemorySize.toHumanReadableString() - + "). Configured/Derived memory sizes are: " - + "Total Flink Memory (" + totalFlinkMemory.toHumanReadableString() - + "), JVM Metaspace (" + jvmMetaspaceAndOverhead.metaspace.toHumanReadableString() - + "), JVM Overhead (" + jvmMetaspaceAndOverhead.overhead.toHumanReadableString() + ")."); - } - } - } - - private static void sanityCheckNetworkMemoryWithExplicitlySetTotalFlinkAndHeapMemory( - final Configuration config, - final MemorySize derivedNetworkMemorySize, - final MemorySize totalFlinkMemorySize) { - try { - sanityCheckNetworkMemory(config, derivedNetworkMemorySize, totalFlinkMemorySize); - } catch (IllegalConfigurationException e) { - throw new IllegalConfigurationException( - "If Total Flink, Task Heap and (or) Managed Memory sizes are explicitly configured then " + - "the Network Memory size is the rest of the Total Flink memory after subtracting all other " + - "configured types of memory, but the derived Network Memory is inconsistent with its configuration.", - e); - } - } - - private static void sanityCheckNetworkMemory( - final Configuration config, - final MemorySize derivedNetworkMemorySize, - final MemorySize totalFlinkMemorySize) { - if (isUsingLegacyNetworkConfigs(config)) { - final MemorySize configuredNetworkMemorySize = getNetworkMemorySizeWithLegacyConfig(config); - if (!configuredNetworkMemorySize.equals(derivedNetworkMemorySize)) { - throw new IllegalConfigurationException( - "Derived Network Memory size (" + derivedNetworkMemorySize.toHumanReadableString() - + ") does not match configured Network Memory size (" + configuredNetworkMemorySize.toHumanReadableString() + ")."); - } - } else { - final RangeFraction networkRangeFraction = getNetworkMemoryRangeFraction(config); - if (derivedNetworkMemorySize.getBytes() > networkRangeFraction.maxSize.getBytes() || - derivedNetworkMemorySize.getBytes() < networkRangeFraction.minSize.getBytes()) { - throw new IllegalConfigurationException("Derived Network Memory size (" - + derivedNetworkMemorySize.toHumanReadableString() + ") is not in configured Network Memory range [" - + networkRangeFraction.minSize.toHumanReadableString() + ", " - + networkRangeFraction.maxSize.toHumanReadableString() + "]."); - } - if (isNetworkMemoryFractionExplicitlyConfigured(config) && - !derivedNetworkMemorySize.equals(totalFlinkMemorySize.multiply(networkRangeFraction.fraction))) { - LOG.info( - "The derived Network Memory size ({}) does not match " + - "the configured Network Memory fraction ({}) from the configured Total Flink Memory size ({}). " + - "The derived Network Memory size will be used.", - derivedNetworkMemorySize.toHumanReadableString(), - networkRangeFraction.fraction, - totalFlinkMemorySize.toHumanReadableString()); - } - } - } - - private static void sanityCheckJvmOverhead( - final Configuration config, - final MemorySize derivedJvmOverheadSize, - final MemorySize totalProcessMemorySize) { - final RangeFraction jvmOverheadRangeFraction = getJvmOverheadRangeFraction(config); - if (derivedJvmOverheadSize.getBytes() > jvmOverheadRangeFraction.maxSize.getBytes() || - derivedJvmOverheadSize.getBytes() < jvmOverheadRangeFraction.minSize.getBytes()) { - throw new IllegalConfigurationException("Derived JVM Overhead size (" - + derivedJvmOverheadSize.toHumanReadableString() + ") is not in configured JVM Overhead range [" - + jvmOverheadRangeFraction.minSize.toHumanReadableString() + ", " - + jvmOverheadRangeFraction.maxSize.toHumanReadableString() + "]."); - } - if (isJvmOverheadFractionExplicitlyConfigured(config) && - !derivedJvmOverheadSize.equals(totalProcessMemorySize.multiply(jvmOverheadRangeFraction.fraction))) { - LOG.info( - "The derived JVM Overhead size ({}) does not match " + - "the configured JVM Overhead fraction ({}) from the configured Total Process Memory size ({}). " + - "The derived JVM OVerhead size will be used.", - derivedJvmOverheadSize.toHumanReadableString(), - jvmOverheadRangeFraction.fraction, - totalProcessMemorySize.toHumanReadableString()); - } + final CommonProcessMemorySpec processMemory) { + TaskExecutorFlinkMemory flinkMemory = processMemory.getFlinkMemory(); + JvmMetaspaceAndOverhead jvmMetaspaceAndOverhead = processMemory.getJvmMetaspaceAndOverhead(); + return new TaskExecutorProcessSpec(getCpuCores(config), flinkMemory, jvmMetaspaceAndOverhead); } private static CPUResource getCpuCores(final Configuration config) { @@ -646,7 +144,7 @@ public class TaskExecutorProcessUtils { public static double getCpuCoresWithFallbackConfigOption(final Configuration config, ConfigOption fallbackOption) { double fallbackValue = config.getDouble(fallbackOption); - return TaskExecutorProcessUtils.getCpuCoresWithFallback(config, fallbackValue).getValue().doubleValue(); + return getCpuCoresWithFallback(config, fallbackValue).getValue().doubleValue(); } public static CPUResource getCpuCoresWithFallback(final Configuration config, double fallback) { @@ -669,124 +167,9 @@ public class TaskExecutorProcessUtils { return new CPUResource(cpuCores); } - private static TaskExecutorProcessSpec createTaskExecutorProcessSpec( - final Configuration config, - final FlinkInternalMemory flinkInternalMemory, - final JvmMetaspaceAndOverhead jvmMetaspaceAndOverhead) { - return new TaskExecutorProcessSpec( - getCpuCores(config), - flinkInternalMemory.frameworkHeap, - flinkInternalMemory.frameworkOffHeap, - flinkInternalMemory.taskHeap, - flinkInternalMemory.taskOffHeap, - flinkInternalMemory.network, - flinkInternalMemory.managed, - jvmMetaspaceAndOverhead.metaspace, - jvmMetaspaceAndOverhead.overhead); - } - - private static class RangeFraction { - final MemorySize minSize; - final MemorySize maxSize; - final double fraction; - - RangeFraction(final MemorySize minSize, final MemorySize maxSize, final double fraction) { - this.minSize = minSize; - this.maxSize = maxSize; - this.fraction = fraction; - checkArgument(minSize.getBytes() <= maxSize.getBytes(), "min value must be less or equal to max value"); - checkArgument(fraction >= 0 && fraction < 1, "fraction must be in range [0, 1)"); - } - } - - private static class FlinkInternalMemory { - final MemorySize frameworkHeap; - final MemorySize frameworkOffHeap; - final MemorySize taskHeap; - final MemorySize taskOffHeap; - final MemorySize network; - final MemorySize managed; - - FlinkInternalMemory( - final MemorySize frameworkHeap, - final MemorySize frameworkOffHeap, - final MemorySize taskHeap, - final MemorySize taskOffHeap, - final MemorySize network, - final MemorySize managed) { - - this.frameworkHeap = checkNotNull(frameworkHeap); - this.frameworkOffHeap = checkNotNull(frameworkOffHeap); - this.taskHeap = checkNotNull(taskHeap); - this.taskOffHeap = checkNotNull(taskOffHeap); - this.network = checkNotNull(network); - this.managed = checkNotNull(managed); - } - - MemorySize getTotalFlinkMemorySize() { - return frameworkHeap.add(frameworkOffHeap).add(taskHeap).add(taskOffHeap).add(network).add(managed); - } - } - - private static class JvmMetaspaceAndOverhead { - final MemorySize metaspace; - final MemorySize overhead; - - JvmMetaspaceAndOverhead(final MemorySize jvmMetaspace, final MemorySize jvmOverhead) { - this.metaspace = checkNotNull(jvmMetaspace); - this.overhead = checkNotNull(jvmOverhead); - } - - MemorySize getTotalJvmMetaspaceAndOverheadSize() { - return metaspace.add(overhead); - } - } - public static Configuration getConfigurationMapLegacyTaskManagerHeapSizeToConfigOption( - final Configuration configuration, ConfigOption configOption) { - - if (configuration.contains(configOption)) { - return configuration; - } - - return getLegacyTaskManagerHeapMemoryIfExplicitlyConfigured(configuration).map(legacyHeapSize -> { - final Configuration copiedConfig = new Configuration(configuration); - copiedConfig.set(configOption, legacyHeapSize); - - LOG.info( - "'{}' is not specified, use the configured deprecated task manager heap value ({}) for it.", - configOption.key(), - legacyHeapSize.toHumanReadableString()); - - return copiedConfig; - }).orElse(configuration); - } - - @SuppressWarnings("deprecation") - private static Optional getLegacyTaskManagerHeapMemoryIfExplicitlyConfigured(final Configuration configuration) { - String totalProcessEnv = System.getenv("FLINK_TM_HEAP"); - if (totalProcessEnv != null) { - try { - return Optional.of(MemorySize.parse(totalProcessEnv)); - } catch (Throwable t) { - throw new IllegalConfigurationException("Cannot read total process memory size from environment variable value " - + totalProcessEnv + ".", t); - } - } - - if (configuration.contains(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY)) { - return Optional.of(getMemorySizeFromConfig(configuration, TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY)); - } - - if (configuration.contains(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY_MB)) { - final long legacyHeapMemoryMB = configuration.getInteger(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY_MB); - if (legacyHeapMemoryMB < 0) { - throw new IllegalConfigurationException("Configured total process memory size (" - + legacyHeapMemoryMB + "MB) must not be less than 0."); - } - return Optional.of(new MemorySize(legacyHeapMemoryMB << 20)); // megabytes to bytes; - } - - return Optional.empty(); + final Configuration configuration, + final ConfigOption configOption) { + return LEGACY_MEMORY_UTILS.getConfWithLegacyHeapSizeMappedToNewConfigOption(configuration, configOption); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java index 8a687287c2f66aad30e6b8ea0470f6aa52caa56e..53561ddb220a109f641f901b48b4b2549074c09c 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java @@ -384,6 +384,32 @@ public class ResourceProfile implements Serializable { ); } + @Nonnull + public ResourceProfile multiply(final int multiplier) { + checkArgument(multiplier >= 0, "multiplier must be >= 0"); + if (equals(ANY)) { + return ANY; + } + + if (this.equals(UNKNOWN)) { + return UNKNOWN; + } + + Map resultExtendedResource = new HashMap<>(extendedResources.size()); + for (Map.Entry entry : extendedResources.entrySet()) { + resultExtendedResource.put(entry.getKey(), entry.getValue().multiply(multiplier)); + } + + return new ResourceProfile( + cpuCores.multiply(multiplier), + taskHeapMemory.multiply(multiplier), + taskOffHeapMemory.multiply(multiplier), + managedMemory.multiply(multiplier), + networkMemory.multiply(multiplier), + resultExtendedResource + ); + } + @Override public String toString() { if (this.equals(UNKNOWN)) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java index 4f568aa2be1312e718cda3cc43609181acb8dbbf..7ea36e0ca2efecee5cdb4d5ca019c245d3daa899 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java @@ -32,6 +32,7 @@ import java.time.Duration; import java.util.Arrays; import java.util.Collection; import java.util.Collections; +import java.util.concurrent.Callable; import java.util.concurrent.CancellationException; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; @@ -72,6 +73,21 @@ public class FutureUtils { return COMPLETED_VOID_FUTURE; } + /** + * Fakes asynchronous execution by immediately executing the operation and returns a (exceptionally) completed + * future. + * + * @param operation to executed + * @param type of the result + */ + public static CompletableFuture runSync(Callable operation) { + try { + return CompletableFuture.completedFuture(operation.call()); + } catch (Exception e) { + return completedExceptionally(e); + } + } + // ------------------------------------------------------------------------ // retrying operations // ------------------------------------------------------------------------ @@ -1078,12 +1094,30 @@ public class FutureUtils { * @param type of the value */ public static void forward(CompletableFuture source, CompletableFuture target) { - source.whenComplete((value, throwable) -> { + source.whenComplete(forwardTo(target)); + } + + /** + * Forwards the value from the source future to the target future using the provided executor. + * + * @param source future to forward the value from + * @param target future to forward the value to + * @param executor executor to forward the source value to the target future + * @param type of the value + */ + public static void forwardAsync(CompletableFuture source, CompletableFuture target, Executor executor) { + source.whenCompleteAsync( + forwardTo(target), + executor); + } + + private static BiConsumer forwardTo(CompletableFuture target) { + return (value, throwable) -> { if (throwable != null) { target.completeExceptionally(throwable); } else { target.complete(value); } - }); + }; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java index ac5b770b9ebe0df56d5a3924930d6fe1bf98d455..cf36a128e930013a32e043930fb8deb83145115d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java @@ -35,7 +35,7 @@ import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.io.Serializable; -import java.util.Collection; +import java.util.List; /** * A task deployment descriptor contains all the information necessary to deploy a task on a task manager. @@ -134,10 +134,10 @@ public final class TaskDeploymentDescriptor implements Serializable { private final int attemptNumber; /** The list of produced intermediate result partition deployment descriptors. */ - private final Collection producedPartitions; + private final List producedPartitions; /** The list of consumed intermediate result partitions. */ - private final Collection inputGates; + private final List inputGates; /** Slot number to run the sub task in on the target machine. */ private final int targetSlotNumber; @@ -156,8 +156,8 @@ public final class TaskDeploymentDescriptor implements Serializable { int attemptNumber, int targetSlotNumber, @Nullable JobManagerTaskRestore taskRestore, - Collection resultPartitionDeploymentDescriptors, - Collection inputGateDeploymentDescriptors) { + List resultPartitionDeploymentDescriptors, + List inputGateDeploymentDescriptors) { this.jobId = Preconditions.checkNotNull(jobId); @@ -256,11 +256,11 @@ public final class TaskDeploymentDescriptor implements Serializable { return targetSlotNumber; } - public Collection getProducedPartitions() { + public List getProducedPartitions() { return producedPartitions; } - public Collection getInputGates() { + public List getInputGates() { return inputGates; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/AbstractDispatcherBootstrap.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/AbstractDispatcherBootstrap.java new file mode 100644 index 0000000000000000000000000000000000000000..9dde71e877d5d0c233fac321e8fe1ff0acb800a2 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/AbstractDispatcherBootstrap.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.dispatcher; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.jobgraph.JobGraph; + +import java.util.Collection; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A base class for a {@link DispatcherBootstrap}. + * + *

NOTE TO IMPLEMENTERS: This is meant as a bridge between the package-private + * {@link Dispatcher#runRecoveredJob(JobGraph)} method, and dispatcher bootstrap + * implementations in other packages. + */ +@Internal +public abstract class AbstractDispatcherBootstrap implements DispatcherBootstrap { + + protected void launchRecoveredJobGraphs(final Dispatcher dispatcher, final Collection recoveredJobGraphs) { + checkNotNull(dispatcher); + checkNotNull(recoveredJobGraphs); + + for (JobGraph recoveredJob : recoveredJobGraphs) { + dispatcher.runRecoveredJob(recoveredJob); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DefaultDispatcherBootstrap.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DefaultDispatcherBootstrap.java new file mode 100644 index 0000000000000000000000000000000000000000..871b5a4d0a2661f4ac66693735298d603e525e0c --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DefaultDispatcherBootstrap.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.dispatcher; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.concurrent.ScheduledExecutor; +import org.apache.flink.runtime.jobgraph.JobGraph; + +import java.util.Collection; +import java.util.HashSet; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A {@link DispatcherBootstrap} which submits the provided {@link JobGraph job graphs} + * for execution upon dispatcher initialization. + */ +@Internal +public class DefaultDispatcherBootstrap extends AbstractDispatcherBootstrap { + + private final Collection recoveredJobs; + + public DefaultDispatcherBootstrap(final Collection recoveredJobsGraphs) { + this.recoveredJobs = new HashSet<>(checkNotNull(recoveredJobsGraphs)); + } + + @Override + public void initialize(final Dispatcher dispatcher, ScheduledExecutor scheduledExecutor) { + launchRecoveredJobGraphs(dispatcher, recoveredJobs); + recoveredJobs.clear(); + } + + @Override + public void stop() throws Exception { + // do nothing + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java index 51ecfc59379a096eb63d61b0220c6a8d87c53faf..d4871228b3a3c81d84dfa4a0d34ad28aef2b92ac 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java @@ -61,6 +61,7 @@ import org.apache.flink.runtime.rest.handler.legacy.backpressure.OperatorBackPre import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.PermanentlyFencedRpcEndpoint; import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils; import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; @@ -71,6 +72,7 @@ import org.apache.flink.util.function.FunctionUtils; import org.apache.flink.util.function.FunctionWithException; import javax.annotation.Nonnull; +import javax.annotation.Nullable; import java.io.IOException; import java.util.ArrayList; @@ -87,6 +89,8 @@ import java.util.function.BiFunction; import java.util.function.Function; import java.util.stream.Collectors; +import static org.apache.flink.util.Preconditions.checkNotNull; + /** * Base class for the Dispatcher component. The Dispatcher component is responsible * for receiving job submissions, persisting them, spawning JobManagers to execute @@ -112,7 +116,7 @@ public abstract class Dispatcher extends PermanentlyFencedRpcEndpoint> jobManagerRunnerFutures; - private final Collection recoveredJobs; + private final DispatcherBootstrap dispatcherBootstrap; private final ArchivedExecutionGraphStore archivedExecutionGraphStore; @@ -122,6 +126,7 @@ public abstract class Dispatcher extends PermanentlyFencedRpcEndpoint> jobManagerTerminationFutures; @@ -130,12 +135,11 @@ public abstract class Dispatcher extends PermanentlyFencedRpcEndpoint recoveredJobs, + DispatcherBootstrap dispatcherBootstrap, DispatcherServices dispatcherServices) throws Exception { - super(rpcService, endpointId, fencingToken); - Preconditions.checkNotNull(dispatcherServices); + super(rpcService, AkkaRpcServiceUtils.createRandomName(DISPATCHER_NAME), fencingToken); + checkNotNull(dispatcherServices); this.configuration = dispatcherServices.getConfiguration(); this.highAvailabilityServices = dispatcherServices.getHighAvailabilityServices(); @@ -165,7 +169,7 @@ public abstract class Dispatcher extends PermanentlyFencedRpcEndpoint(); - this.recoveredJobs = new HashSet<>(recoveredJobs); + this.dispatcherBootstrap = checkNotNull(dispatcherBootstrap); } //------------------------------------------------------ @@ -190,7 +194,7 @@ public abstract class Dispatcher extends PermanentlyFencedRpcEndpoint handleRecoveredJobStartError(JobID jobId) { @@ -239,6 +240,8 @@ public abstract class Dispatcher extends PermanentlyFencedRpcEndpoint { + dispatcherBootstrap.stop(); + stopDispatcherServices(); log.info("Stopped dispatcher {}.", getAddress()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherBootstrap.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherBootstrap.java new file mode 100644 index 0000000000000000000000000000000000000000..e0a327bb73d0d4f3b5a3509fd1307998b071fdbd --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherBootstrap.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.dispatcher; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.concurrent.ScheduledExecutor; + +/** + * An interface containing the logic of bootstrapping the {@link Dispatcher} of a cluster. + */ +@Internal +public interface DispatcherBootstrap { + + /** + * Initializes the {@link Dispatcher} provided as an argument. + * + *

IMPORTANT: In HA settings, this method will run during + * the initialization of the **leader** dispatcher. + * + * @param dispatcher the dispatcher to be initialized. + */ + void initialize(final Dispatcher dispatcher, ScheduledExecutor scheduledExecutor) throws Exception; + + /** + * Stops and frees any resources (e.g. threads) acquired + * by the {@link #initialize(Dispatcher, ScheduledExecutor)}. + */ + void stop() throws Exception; +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherFactory.java index cd3c34daaab29cf29b5e8aebddca6e2927f20e83..71376c23b22efe0af0fd3a10b9a2f1bff2e24297 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherFactory.java @@ -18,12 +18,8 @@ package org.apache.flink.runtime.dispatcher; -import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.rpc.RpcService; -import java.util.Collection; -import java.util.UUID; - /** * {@link Dispatcher} factory interface. */ @@ -35,14 +31,6 @@ public interface DispatcherFactory { Dispatcher createDispatcher( RpcService rpcService, DispatcherId fencingToken, - Collection recoveredJobs, + DispatcherBootstrap dispatcherBootstrap, PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore) throws Exception; - - default String generateEndpointIdWithUUID() { - return getEndpointId() + UUID.randomUUID(); - } - - default String getEndpointId() { - return Dispatcher.DISPATCHER_NAME; - } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java index 0bbb10f32b8a6aeaf77b6d04cd446e7c8f75e88f..870cb5a51afb995c1bb643035bae23eaed2426d1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java @@ -28,6 +28,7 @@ import org.apache.flink.runtime.rest.RestServerEndpointConfiguration; import org.apache.flink.runtime.rest.handler.RestHandlerConfiguration; import org.apache.flink.runtime.rest.handler.RestHandlerSpecification; import org.apache.flink.runtime.rest.handler.job.JobSubmitHandler; +import org.apache.flink.runtime.rest.handler.legacy.ExecutionGraphCache; import org.apache.flink.runtime.rest.handler.legacy.metrics.MetricFetcher; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.webmonitor.WebMonitorEndpoint; @@ -42,7 +43,7 @@ import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandler; import java.io.IOException; import java.util.List; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutorService; +import java.util.concurrent.ScheduledExecutorService; /** * REST endpoint for the {@link Dispatcher} component. @@ -58,9 +59,10 @@ public class DispatcherRestEndpoint extends WebMonitorEndpoint resourceManagerRetriever, TransientBlobService transientBlobService, - ExecutorService executor, + ScheduledExecutorService executor, MetricFetcher metricFetcher, LeaderElectionService leaderElectionService, + ExecutionGraphCache executionGraphCache, FatalErrorHandler fatalErrorHandler) throws IOException { super( @@ -73,6 +75,7 @@ public class DispatcherRestEndpoint extends WebMonitorEndpoint archiveExecutionGraph(AccessExecutionGraph executionGraph); - static HistoryServerArchivist createHistoryServerArchivist(Configuration configuration, JsonArchivist jsonArchivist) { + static HistoryServerArchivist createHistoryServerArchivist(Configuration configuration, JsonArchivist jsonArchivist, Executor ioExecutor) { final String configuredArchivePath = configuration.getString(JobManagerOptions.ARCHIVE_DIR); if (configuredArchivePath != null) { final Path archivePath = WebMonitorUtils.validateAndNormalizeUri(new Path(configuredArchivePath).toUri()); - return new JsonResponseHistoryServerArchivist(jsonArchivist, archivePath); + return new JsonResponseHistoryServerArchivist(jsonArchivist, archivePath, ioExecutor); } else { return VoidHistoryServerArchivist.INSTANCE; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobDispatcherFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobDispatcherFactory.java index a8cafc8d7f86ac2988d7e31cb3884eac3455e14e..cc1717773ac58b0a085054747fb75432b558403a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobDispatcherFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobDispatcherFactory.java @@ -20,13 +20,8 @@ package org.apache.flink.runtime.dispatcher; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.entrypoint.ClusterEntrypoint; -import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.rpc.RpcService; -import org.apache.flink.shaded.guava18.com.google.common.collect.Iterables; - -import java.util.Collection; - import static org.apache.flink.runtime.entrypoint.ClusterEntrypoint.EXECUTION_MODE; /** @@ -39,20 +34,17 @@ public enum JobDispatcherFactory implements DispatcherFactory { public MiniDispatcher createDispatcher( RpcService rpcService, DispatcherId fencingToken, - Collection recoveredJobs, + DispatcherBootstrap dispatcherBootstrap, PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore) throws Exception { - final JobGraph jobGraph = Iterables.getOnlyElement(recoveredJobs); - final Configuration configuration = partialDispatcherServicesWithJobGraphStore.getConfiguration(); final String executionModeValue = configuration.getString(EXECUTION_MODE); final ClusterEntrypoint.ExecutionMode executionMode = ClusterEntrypoint.ExecutionMode.valueOf(executionModeValue); return new MiniDispatcher( rpcService, - getEndpointId(), fencingToken, DispatcherServices.from(partialDispatcherServicesWithJobGraphStore, DefaultJobManagerRunnerFactory.INSTANCE), - jobGraph, + dispatcherBootstrap, executionMode); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JsonResponseHistoryServerArchivist.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JsonResponseHistoryServerArchivist.java index be583991702f8a73c3c0e7840f1d6e0e47eabfd3..152f8f666fbb01146c4adcea77669feeb2588b78 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JsonResponseHistoryServerArchivist.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JsonResponseHistoryServerArchivist.java @@ -19,15 +19,15 @@ package org.apache.flink.runtime.dispatcher; import org.apache.flink.core.fs.Path; -import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.history.FsJobArchivist; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.webmonitor.history.JsonArchivist; import org.apache.flink.util.Preconditions; +import org.apache.flink.util.function.ThrowingRunnable; -import java.io.IOException; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; /** * Implementation which archives an {@link AccessExecutionGraph} such that it stores @@ -39,18 +39,21 @@ class JsonResponseHistoryServerArchivist implements HistoryServerArchivist { private final Path archivePath; - JsonResponseHistoryServerArchivist(JsonArchivist jsonArchivist, Path archivePath) { + private final Executor ioExecutor; + + JsonResponseHistoryServerArchivist(JsonArchivist jsonArchivist, Path archivePath, Executor ioExecutor) { this.jsonArchivist = Preconditions.checkNotNull(jsonArchivist); this.archivePath = Preconditions.checkNotNull(archivePath); + this.ioExecutor = Preconditions.checkNotNull(ioExecutor); } @Override public CompletableFuture archiveExecutionGraph(AccessExecutionGraph executionGraph) { - try { - FsJobArchivist.archiveJob(archivePath, executionGraph.getJobID(), jsonArchivist.archiveJsonWithPath(executionGraph)); - return CompletableFuture.completedFuture(Acknowledge.get()); - } catch (IOException e) { - return FutureUtils.completedExceptionally(e); - } + return CompletableFuture + .runAsync( + ThrowingRunnable.unchecked(() -> + FsJobArchivist.archiveJob(archivePath, executionGraph.getJobID(), jsonArchivist.archiveJsonWithPath(executionGraph))), + ioExecutor) + .thenApply(ignored -> Acknowledge.get()); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/MiniDispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/MiniDispatcher.java index f682cc81f73efa1aa0c2469da9c93aa5fff94198..53ec47d708000da4fa84f0c45a95bef20cb987bf 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/MiniDispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/MiniDispatcher.java @@ -33,7 +33,6 @@ import org.apache.flink.util.FlinkException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Collections; import java.util.concurrent.CompletableFuture; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -53,16 +52,14 @@ public class MiniDispatcher extends Dispatcher { public MiniDispatcher( RpcService rpcService, - String endpointId, DispatcherId fencingToken, DispatcherServices dispatcherServices, - JobGraph jobGraph, + DispatcherBootstrap dispatcherBootstrap, JobClusterEntrypoint.ExecutionMode executionMode) throws Exception { super( rpcService, - endpointId, fencingToken, - Collections.singleton(jobGraph), + dispatcherBootstrap, dispatcherServices); this.executionMode = checkNotNull(executionMode); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServices.java index 044494a915c377a7002996b421109178f6cd4ea2..89049a09b4cccc9eaff075f4749096155d4f3958 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServices.java @@ -27,6 +27,7 @@ import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; import javax.annotation.Nonnull; +import javax.annotation.Nullable; /** * Partial {@link DispatcherServices} services container which needs to @@ -61,7 +62,7 @@ public class PartialDispatcherServices { @Nonnull private final HistoryServerArchivist historyServerArchivist; - @Nonnull + @Nullable private final String metricQueryServiceAddress; public PartialDispatcherServices( @@ -74,7 +75,7 @@ public class PartialDispatcherServices { @Nonnull ArchivedExecutionGraphStore archivedExecutionGraphStore, @Nonnull FatalErrorHandler fatalErrorHandler, @Nonnull HistoryServerArchivist historyServerArchivist, - @Nonnull String metricQueryServiceAddress) { + @Nullable String metricQueryServiceAddress) { this.configuration = configuration; this.highAvailabilityServices = highAvailabilityServices; this.resourceManagerGatewayRetriever = resourceManagerGatewayRetriever; @@ -132,7 +133,7 @@ public class PartialDispatcherServices { return historyServerArchivist; } - @Nonnull + @Nullable public String getMetricQueryServiceAddress() { return metricQueryServiceAddress; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServicesWithJobGraphStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServicesWithJobGraphStore.java index bb807742cc48ab9cf2e3f7f49610b805ae7bcb4e..0eec923c710c6ff0c3dbfd67e36f9ae34b43043a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServicesWithJobGraphStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServicesWithJobGraphStore.java @@ -28,6 +28,7 @@ import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; import javax.annotation.Nonnull; +import javax.annotation.Nullable; /** * {@link DispatcherFactory} services container. @@ -47,7 +48,7 @@ public class PartialDispatcherServicesWithJobGraphStore extends PartialDispatche @Nonnull ArchivedExecutionGraphStore archivedExecutionGraphStore, @Nonnull FatalErrorHandler fatalErrorHandler, @Nonnull HistoryServerArchivist historyServerArchivist, - @Nonnull String metricQueryServiceAddress, + @Nullable String metricQueryServiceAddress, @Nonnull JobGraphWriter jobGraphWriter) { super( configuration, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SessionDispatcherFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SessionDispatcherFactory.java index fbd88a5f7fc7041047340fb2598a1bf7a04da369..cc867ca7de8349681e742b40189eff24d4e97ada 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SessionDispatcherFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SessionDispatcherFactory.java @@ -18,11 +18,8 @@ package org.apache.flink.runtime.dispatcher; -import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.rpc.RpcService; -import java.util.Collection; - /** * {@link DispatcherFactory} which creates a {@link StandaloneDispatcher}. */ @@ -33,14 +30,13 @@ public enum SessionDispatcherFactory implements DispatcherFactory { public StandaloneDispatcher createDispatcher( RpcService rpcService, DispatcherId fencingToken, - Collection recoveredJobs, + DispatcherBootstrap dispatcherBootstrap, PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore) throws Exception { // create the default dispatcher return new StandaloneDispatcher( rpcService, - getEndpointId(), fencingToken, - recoveredJobs, + dispatcherBootstrap, DispatcherServices.from(partialDispatcherServicesWithJobGraphStore, DefaultJobManagerRunnerFactory.INSTANCE)); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/StandaloneDispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/StandaloneDispatcher.java index bde5d10c83cd892e886e7383b61b72f731827608..4ba50c55169f1134d575029bf57938f20e41f626 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/StandaloneDispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/StandaloneDispatcher.java @@ -22,8 +22,6 @@ import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmaster.JobMaster; import org.apache.flink.runtime.rpc.RpcService; -import java.util.Collection; - /** * Dispatcher implementation which spawns a {@link JobMaster} for each * submitted {@link JobGraph} within in the same process. This dispatcher @@ -32,15 +30,13 @@ import java.util.Collection; public class StandaloneDispatcher extends Dispatcher { public StandaloneDispatcher( RpcService rpcService, - String endpointId, DispatcherId fencingToken, - Collection recoveredJobs, + DispatcherBootstrap dispatcherBootstrap, DispatcherServices dispatcherServices) throws Exception { super( rpcService, - endpointId, fencingToken, - recoveredJobs, + dispatcherBootstrap, dispatcherServices); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/AbstractDispatcherLeaderProcess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/AbstractDispatcherLeaderProcess.java index 948d41e71c382fb382404726157c0f6b617e6a1a..66d7d6cf6c83d025a2dafab0dee9e1b97913fbec 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/AbstractDispatcherLeaderProcess.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/AbstractDispatcherLeaderProcess.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.dispatcher.runner; +import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.clusterframework.ApplicationStatus; @@ -43,7 +44,11 @@ import java.util.concurrent.CompletableFuture; import java.util.function.Predicate; import java.util.function.Supplier; -abstract class AbstractDispatcherLeaderProcess implements DispatcherLeaderProcess { +/** + * A base {@link DispatcherLeaderProcess}. + */ +@Internal +public abstract class AbstractDispatcherLeaderProcess implements DispatcherLeaderProcess { protected final Logger log = LoggerFactory.getLogger(getClass()); @@ -227,6 +232,9 @@ abstract class AbstractDispatcherLeaderProcess implements DispatcherLeaderProces return null; } + /** + * The state of the {@link DispatcherLeaderProcess}. + */ protected enum State { CREATED, RUNNING, @@ -237,14 +245,20 @@ abstract class AbstractDispatcherLeaderProcess implements DispatcherLeaderProces // Internal classes // ------------------------------------------------------------ - interface DispatcherGatewayServiceFactory { + /** + * Factory for {@link DispatcherGatewayService}. + */ + public interface DispatcherGatewayServiceFactory { DispatcherGatewayService create( DispatcherId fencingToken, Collection recoveredJobs, JobGraphWriter jobGraphWriter); } - interface DispatcherGatewayService extends AutoCloseableAsync { + /** + * An accessor of the {@link DispatcherGateway}. + */ + public interface DispatcherGatewayService extends AutoCloseableAsync { DispatcherGateway getGateway(); CompletableFuture onRemovedJobGraph(JobID jobId); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherGatewayService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherGatewayService.java index 720c79f6ad676259f0feac279dd447520c1b7e36..cc668cc05daed0267c4b0b43399e148f3e6d2344 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherGatewayService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherGatewayService.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.dispatcher.runner; +import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.dispatcher.Dispatcher; @@ -25,7 +26,11 @@ import org.apache.flink.runtime.dispatcher.DispatcherGateway; import java.util.concurrent.CompletableFuture; -class DefaultDispatcherGatewayService implements AbstractDispatcherLeaderProcess.DispatcherGatewayService { +/** + * A facade over the {@link Dispatcher} that exposes specific functionality. + */ +@Internal +public class DefaultDispatcherGatewayService implements AbstractDispatcherLeaderProcess.DispatcherGatewayService { private final Dispatcher dispatcher; private final DispatcherGateway dispatcherGateway; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherGatewayServiceFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherGatewayServiceFactory.java index 73a0bd022d831d7ea54553f29886eff456ea35c6..072d0488ea843268dc51f2740457735bc4c85e3c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherGatewayServiceFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherGatewayServiceFactory.java @@ -18,7 +18,9 @@ package org.apache.flink.runtime.dispatcher.runner; +import org.apache.flink.runtime.dispatcher.DefaultDispatcherBootstrap; import org.apache.flink.runtime.dispatcher.Dispatcher; +import org.apache.flink.runtime.dispatcher.DispatcherBootstrap; import org.apache.flink.runtime.dispatcher.DispatcherFactory; import org.apache.flink.runtime.dispatcher.DispatcherId; import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; @@ -55,12 +57,16 @@ class DefaultDispatcherGatewayServiceFactory implements AbstractDispatcherLeader DispatcherId fencingToken, Collection recoveredJobs, JobGraphWriter jobGraphWriter) { + + final DispatcherBootstrap bootstrap = + new DefaultDispatcherBootstrap(recoveredJobs); + final Dispatcher dispatcher; try { dispatcher = dispatcherFactory.createDispatcher( rpcService, fencingToken, - recoveredJobs, + bootstrap, PartialDispatcherServicesWithJobGraphStore.from(partialDispatcherServices, jobGraphWriter)); } catch (Exception e) { throw new FlinkRuntimeException("Could not create the Dispatcher rpc endpoint.", e); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerFactory.java index 84ba3f49f57080c3b49498290adb088a037b7e0d..d08b9d48fe8dae949be6a45b36e89448c70e86f6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerFactory.java @@ -35,7 +35,7 @@ import java.util.concurrent.Executor; public class DefaultDispatcherRunnerFactory implements DispatcherRunnerFactory { private final DispatcherLeaderProcessFactoryFactory dispatcherLeaderProcessFactoryFactory; - private DefaultDispatcherRunnerFactory(DispatcherLeaderProcessFactoryFactory dispatcherLeaderProcessFactoryFactory) { + public DefaultDispatcherRunnerFactory(DispatcherLeaderProcessFactoryFactory dispatcherLeaderProcessFactoryFactory) { this.dispatcherLeaderProcessFactoryFactory = dispatcherLeaderProcessFactoryFactory; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessFactoryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessFactoryFactory.java index 8bde2df7858083e49091e06d01fa7881b0ab9a94..6cebf26379c16d2a624ab3057cbdea93cf441dc6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessFactoryFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessFactoryFactory.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.dispatcher.runner; +import org.apache.flink.annotation.Internal; import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; import org.apache.flink.runtime.jobmanager.JobGraphStoreFactory; import org.apache.flink.runtime.rpc.FatalErrorHandler; @@ -28,6 +29,7 @@ import java.util.concurrent.Executor; /** * Factory for {@link DispatcherLeaderProcessFactory}. */ +@Internal public interface DispatcherLeaderProcessFactoryFactory { DispatcherLeaderProcessFactory createFactory( diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactory.java index 6be827e92dddbe4372303c445363de51d4e251fe..9d49ebc0ba679e0e579f92f03b2839e209476478 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactory.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.dispatcher.runner; +import org.apache.flink.annotation.Internal; import org.apache.flink.runtime.jobmanager.JobGraphStoreFactory; import org.apache.flink.runtime.rpc.FatalErrorHandler; @@ -27,14 +28,15 @@ import java.util.concurrent.Executor; /** * Factory for the {@link SessionDispatcherLeaderProcess}. */ -class SessionDispatcherLeaderProcessFactory implements DispatcherLeaderProcessFactory { +@Internal +public class SessionDispatcherLeaderProcessFactory implements DispatcherLeaderProcessFactory { private final AbstractDispatcherLeaderProcess.DispatcherGatewayServiceFactory dispatcherGatewayServiceFactory; private final JobGraphStoreFactory jobGraphStoreFactory; private final Executor ioExecutor; private final FatalErrorHandler fatalErrorHandler; - SessionDispatcherLeaderProcessFactory( + public SessionDispatcherLeaderProcessFactory( AbstractDispatcherLeaderProcess.DispatcherGatewayServiceFactory dispatcherGatewayServiceFactory, JobGraphStoreFactory jobGraphStoreFactory, Executor ioExecutor, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterConfigurationParserFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterConfigurationParserFactory.java index 12115d726a43e054066d87757f8cf3771ad30e8e..069d5b1373f1f985bea800230a01900163bc5ce9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterConfigurationParserFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterConfigurationParserFactory.java @@ -36,8 +36,7 @@ import static org.apache.flink.runtime.entrypoint.parser.CommandLineOptions.DYNA */ public class ClusterConfigurationParserFactory implements ParserResultFactory { - @Override - public Options getOptions() { + public static Options options() { final Options options = new Options(); options.addOption(CONFIG_DIR_OPTION); options.addOption(DYNAMIC_PROPERTY_OPTION); @@ -45,6 +44,11 @@ public class ClusterConfigurationParserFactory implements ParserResultFactory) () -> { - runCluster(configuration); + runCluster(configuration, pluginManager); return null; }); @@ -189,9 +189,9 @@ public abstract class ClusterEntrypoint implements AutoCloseableAsync, FatalErro } } - private void configureFileSystems(Configuration configuration) { + private void configureFileSystems(Configuration configuration, PluginManager pluginManager) { LOG.info("Install default filesystem."); - FileSystem.initialize(configuration, PluginUtils.createPluginManagerFromRootFolder(configuration)); + FileSystem.initialize(configuration, pluginManager); } private SecurityContext installSecurityContext(Configuration configuration) throws Exception { @@ -202,9 +202,10 @@ public abstract class ClusterEntrypoint implements AutoCloseableAsync, FatalErro return SecurityUtils.getInstalledContext(); } - private void runCluster(Configuration configuration) throws Exception { + private void runCluster(Configuration configuration, PluginManager pluginManager) throws Exception { synchronized (lock) { - initializeServices(configuration); + + initializeServices(configuration, pluginManager); // write host information into configuration configuration.setString(JobManagerOptions.ADDRESS, commonRpcService.getAddress()); @@ -243,15 +244,17 @@ public abstract class ClusterEntrypoint implements AutoCloseableAsync, FatalErro } } - protected void initializeServices(Configuration configuration) throws Exception { + protected void initializeServices(Configuration configuration, PluginManager pluginManager) throws Exception { LOG.info("Initializing cluster services."); synchronized (lock) { - final String bindAddress = configuration.getString(JobManagerOptions.ADDRESS); - final String portRange = getRPCPortRange(configuration); - - commonRpcService = createRpcService(configuration, bindAddress, portRange); + commonRpcService = AkkaRpcServiceUtils.createRemoteRpcService( + configuration, + configuration.getString(JobManagerOptions.ADDRESS), + getRPCPortRange(configuration), + configuration.getString(JobManagerOptions.BIND_HOST), + configuration.getOptional(JobManagerOptions.RPC_BIND_PORT)); // update the configuration used to create the high availability services configuration.setString(JobManagerOptions.ADDRESS, commonRpcService.getAddress()); @@ -264,9 +267,9 @@ public abstract class ClusterEntrypoint implements AutoCloseableAsync, FatalErro blobServer = new BlobServer(configuration, haServices.createBlobStore()); blobServer.start(); heartbeatServices = createHeartbeatServices(configuration); - metricRegistry = createMetricRegistry(configuration); + metricRegistry = createMetricRegistry(configuration, pluginManager); - final RpcService metricQueryServiceRpcService = MetricUtils.startMetricsRpcService(configuration, bindAddress); + final RpcService metricQueryServiceRpcService = MetricUtils.startMetricsRpcService(configuration, commonRpcService.getAddress()); metricRegistry.startQueryService(metricQueryServiceRpcService, null); final String hostname = RpcUtils.getHostname(commonRpcService); @@ -280,11 +283,6 @@ public abstract class ClusterEntrypoint implements AutoCloseableAsync, FatalErro } } - @Nonnull - private RpcService createRpcService(Configuration configuration, String bindAddress, String portRange) throws Exception { - return AkkaRpcServiceUtils.createRpcService(bindAddress, portRange, configuration); - } - /** * Returns the port range for the common {@link RpcService}. * @@ -312,10 +310,10 @@ public abstract class ClusterEntrypoint implements AutoCloseableAsync, FatalErro return HeartbeatServices.fromConfiguration(configuration); } - protected MetricRegistryImpl createMetricRegistry(Configuration configuration) { + protected MetricRegistryImpl createMetricRegistry(Configuration configuration, PluginManager pluginManager) { return new MetricRegistryImpl( MetricRegistryConfiguration.fromConfiguration(configuration), - ReporterSetup.fromConfiguration(configuration)); + ReporterSetup.fromConfiguration(configuration, pluginManager)); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/StandaloneSessionClusterEntrypoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/StandaloneSessionClusterEntrypoint.java index e549ca9c5c181736f9a338971a69b2e0e4d1b936..06f90b1dafc3876c23f33329ddd361a77aedb3f6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/StandaloneSessionClusterEntrypoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/StandaloneSessionClusterEntrypoint.java @@ -37,7 +37,7 @@ public class StandaloneSessionClusterEntrypoint extends SessionClusterEntrypoint @Override protected DefaultDispatcherResourceManagerComponentFactory createDispatcherResourceManagerComponentFactory(Configuration configuration) { - return DefaultDispatcherResourceManagerComponentFactory.createSessionComponentFactory(StandaloneResourceManagerFactory.INSTANCE); + return DefaultDispatcherResourceManagerComponentFactory.createSessionComponentFactory(StandaloneResourceManagerFactory.getInstance()); } public static void main(String[] args) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DefaultDispatcherResourceManagerComponentFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DefaultDispatcherResourceManagerComponentFactory.java index 59fe1b01671789f29abe4bfa146d17106a35ba5a..0698e32cbeb64737fd11bb22ff6c0f5b0fa7dec0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DefaultDispatcherResourceManagerComponentFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DefaultDispatcherResourceManagerComponentFactory.java @@ -40,7 +40,6 @@ import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.jobmanager.HaServicesJobGraphStoreFactory; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.metrics.MetricRegistry; -import org.apache.flink.runtime.metrics.groups.ResourceManagerMetricGroup; import org.apache.flink.runtime.metrics.util.MetricUtils; import org.apache.flink.runtime.resourcemanager.ResourceManager; import org.apache.flink.runtime.resourcemanager.ResourceManagerFactory; @@ -71,7 +70,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; -import java.util.concurrent.ExecutorService; +import java.util.concurrent.ScheduledExecutorService; /** * Abstract class which implements the creation of the {@link DispatcherResourceManagerComponent} components. @@ -89,7 +88,7 @@ public class DefaultDispatcherResourceManagerComponentFactory implements Dispatc @Nonnull private final RestEndpointFactory restEndpointFactory; - DefaultDispatcherResourceManagerComponentFactory( + public DefaultDispatcherResourceManagerComponentFactory( @Nonnull DispatcherRunnerFactory dispatcherRunnerFactory, @Nonnull ResourceManagerFactory resourceManagerFactory, @Nonnull RestEndpointFactory restEndpointFactory) { @@ -115,7 +114,6 @@ public class DefaultDispatcherResourceManagerComponentFactory implements Dispatc LeaderRetrievalService resourceManagerRetrievalService = null; WebMonitorEndpoint webMonitorEndpoint = null; ResourceManager resourceManager = null; - ResourceManagerMetricGroup resourceManagerMetricGroup = null; DispatcherRunner dispatcherRunner = null; try { @@ -137,7 +135,7 @@ public class DefaultDispatcherResourceManagerComponentFactory implements Dispatc 10, Time.milliseconds(50L)); - final ExecutorService executor = WebMonitorEndpoint.createExecutorService( + final ScheduledExecutorService executor = WebMonitorEndpoint.createExecutorService( configuration.getInteger(RestOptions.SERVER_NUM_THREADS), configuration.getInteger(RestOptions.SERVER_THREAD_PRIORITY), "DispatcherRestEndpoint"); @@ -166,7 +164,6 @@ public class DefaultDispatcherResourceManagerComponentFactory implements Dispatc final String hostname = RpcUtils.getHostname(rpcService); - resourceManagerMetricGroup = ResourceManagerMetricGroup.create(metricRegistry, hostname); resourceManager = resourceManagerFactory.createResourceManager( configuration, ResourceID.generate(), @@ -176,9 +173,10 @@ public class DefaultDispatcherResourceManagerComponentFactory implements Dispatc fatalErrorHandler, new ClusterInformation(hostname, blobServer.getPort()), webMonitorEndpoint.getRestBaseUrl(), - resourceManagerMetricGroup); + metricRegistry, + hostname); - final HistoryServerArchivist historyServerArchivist = HistoryServerArchivist.createHistoryServerArchivist(configuration, webMonitorEndpoint); + final HistoryServerArchivist historyServerArchivist = HistoryServerArchivist.createHistoryServerArchivist(configuration, webMonitorEndpoint, ioExecutor); final PartialDispatcherServices partialDispatcherServices = new PartialDispatcherServices( configuration, @@ -254,10 +252,6 @@ public class DefaultDispatcherResourceManagerComponentFactory implements Dispatc exception = ExceptionUtils.firstOrSuppressed(e, exception); } - if (resourceManagerMetricGroup != null) { - resourceManagerMetricGroup.close(); - } - throw new FlinkException("Could not create the DispatcherResourceManagerComponent.", exception); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java index b90cbb77db04e5c383e14c3991aeabac5b6a5ebf..c29d90923606eebcca5dde402bbbac14982acd7b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java @@ -31,7 +31,7 @@ import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.network.TaskEventDispatcher; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; -import org.apache.flink.runtime.io.network.partition.consumer.InputGate; +import org.apache.flink.runtime.io.network.partition.consumer.IndexedInputGate; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider; import org.apache.flink.runtime.jobgraph.tasks.TaskOperatorEventGateway; @@ -219,9 +219,9 @@ public interface Environment { ResultPartitionWriter[] getAllWriters(); - InputGate getInputGate(int index); + IndexedInputGate getInputGate(int index); - InputGate[] getAllInputGates(); + IndexedInputGate[] getAllInputGates(); TaskEventDispatcher getTaskEventDispatcher(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index f8d79fa14a4abe7273145f74b25713d1160870ad..e348059f4c35a2d2ebbb8245d28a5ef2f11ee8a1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -41,6 +41,7 @@ import org.apache.flink.runtime.checkpoint.CheckpointStatsSnapshot; import org.apache.flink.runtime.checkpoint.CheckpointStatsTracker; import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore; import org.apache.flink.runtime.checkpoint.MasterTriggerRestoreHook; +import org.apache.flink.runtime.checkpoint.OperatorCoordinatorCheckpointContext; import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor; import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.concurrent.FutureUtils.ConjunctFuture; @@ -48,7 +49,6 @@ import org.apache.flink.runtime.concurrent.ScheduledExecutorServiceAdapter; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.execution.SuppressRestartsException; import org.apache.flink.runtime.executiongraph.failover.FailoverStrategy; -import org.apache.flink.runtime.executiongraph.failover.flip1.FailoverTopology; import org.apache.flink.runtime.executiongraph.failover.flip1.ResultPartitionAvailabilityChecker; import org.apache.flink.runtime.executiongraph.failover.flip1.partitionrelease.PartitionReleaseStrategy; import org.apache.flink.runtime.executiongraph.restart.ExecutionGraphRestartCallback; @@ -60,10 +60,12 @@ import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.jobgraph.ScheduleMode; import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration; import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup; import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider; +import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; import org.apache.flink.runtime.query.KvStateLocationRegistry; import org.apache.flink.runtime.scheduler.InternalFailuresListener; import org.apache.flink.runtime.scheduler.adapter.DefaultExecutionTopology; @@ -408,11 +410,7 @@ public class ExecutionGraph implements AccessExecutionGraph { return this.verticesInCreationOrder.size(); } - public SchedulingTopology getSchedulingTopology() { - return executionTopology; - } - - public FailoverTopology getFailoverTopology() { + public SchedulingTopology getSchedulingTopology() { return executionTopology; } @@ -457,6 +455,8 @@ public class ExecutionGraph implements AccessExecutionGraph { ExecutionVertex[] tasksToWaitFor = collectExecutionVertices(verticesToWaitFor); ExecutionVertex[] tasksToCommitTo = collectExecutionVertices(verticesToCommitTo); + final Collection operatorCoordinators = buildOpCoordinatorCheckpointContexts(); + checkpointStatsTracker = checkNotNull(statsTracker, "CheckpointStatsTracker"); CheckpointFailureManager failureManager = new CheckpointFailureManager( @@ -487,6 +487,7 @@ public class ExecutionGraph implements AccessExecutionGraph { tasksToTrigger, tasksToWaitFor, tasksToCommitTo, + operatorCoordinators, checkpointIDCounter, checkpointStore, checkpointStateBackend, @@ -566,6 +567,21 @@ public class ExecutionGraph implements AccessExecutionGraph { } } + private Collection buildOpCoordinatorCheckpointContexts() { + final ArrayList contexts = new ArrayList<>(); + for (final ExecutionJobVertex vertex : verticesInCreationOrder) { + for (final Map.Entry coordinator : vertex.getOperatorCoordinatorMap().entrySet()) { + contexts.add(new OperatorCoordinatorCheckpointContext( + coordinator.getValue(), + coordinator.getKey(), + vertex.getMaxParallelism(), + vertex.getParallelism())); + } + } + contexts.trimToSize(); + return contexts; + } + // -------------------------------------------------------------------------------------------- // Properties and Status of the Execution Graph // -------------------------------------------------------------------------------------------- @@ -1525,9 +1541,9 @@ public class ExecutionGraph implements AccessExecutionGraph { } ResultPartitionID createResultPartitionId(final IntermediateResultPartitionID resultPartitionId) { - final SchedulingResultPartition schedulingResultPartition = - getSchedulingTopology().getResultPartitionOrThrow(resultPartitionId); - final SchedulingExecutionVertex producer = schedulingResultPartition.getProducer(); + final SchedulingResultPartition schedulingResultPartition = + getSchedulingTopology().getResultPartition(resultPartitionId); + final SchedulingExecutionVertex producer = schedulingResultPartition.getProducer(); final ExecutionVertexID producerId = producer.getId(); final JobVertexID jobVertexId = producerId.getJobVertexId(); final ExecutionJobVertex jobVertex = getJobVertex(jobVertexId); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java index 4b606520fce3008d1fbc36ea0f136bc1b5ba1a91..9cca60816f9ef39b42b2c67d8041896495b1f033 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java @@ -249,10 +249,12 @@ public class ExecutionJobVertex implements AccessExecutionJobVertex, Archiveable } try { - this.operatorCoordinators = OperatorCoordinatorUtil.instantiateCoordinators( + final Map coordinators = OperatorCoordinatorUtil.instantiateCoordinators( jobVertex.getOperatorCoordinators(), graph.getUserClassLoader(), (opId) -> new ExecutionJobVertexCoordinatorContext(opId, this)); + + this.operatorCoordinators = Collections.unmodifiableMap(coordinators); } catch (IOException | ClassNotFoundException e) { throw new JobException("Cannot instantiate the coordinator for operator " + getName(), e); @@ -402,8 +404,12 @@ public class ExecutionJobVertex implements AccessExecutionJobVertex, Archiveable return operatorCoordinators.get(operatorId); } + public Map getOperatorCoordinatorMap() { + return operatorCoordinators; + } + public Collection getOperatorCoordinators() { - return Collections.unmodifiableCollection(operatorCoordinators.values()); + return operatorCoordinators.values(); } public Either, PermanentBlobKey> getTaskInformationOrBlobKey() throws IOException { @@ -441,18 +447,6 @@ public class ExecutionJobVertex implements AccessExecutionJobVertex, Archiveable return getAggregateJobVertexState(num, parallelism); } - private String generateDebugString() { - - return "ExecutionJobVertex" + - "(" + jobVertex.getName() + " | " + jobVertex.getID() + ")" + - "{" + - "parallelism=" + parallelism + - ", maxParallelism=" + getMaxParallelism() + - ", maxParallelismConfigured=" + maxParallelismConfigured + - '}'; - } - - //--------------------------------------------------------------------------------------------- public void connectToPredecessors(Map intermediateDataSets) throws JobException { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertexCoordinatorContext.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertexCoordinatorContext.java index 07fb44c54630f85c5b0118687a903984d72316ae..c9e99de1066de14bfa15e50facdcc705d6136a9e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertexCoordinatorContext.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertexCoordinatorContext.java @@ -74,6 +74,11 @@ final class ExecutionJobVertexCoordinatorContext implements OperatorCoordinator. jobVertex.getGraph().failGlobal(cause); } + @Override + public int currentParallelism() { + return jobVertex.getParallelism(); + } + private Execution getTaskExecution(int subtask) { return jobVertex.getTaskVertices()[subtask].getCurrentExecutionAttempt(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartition.java index 5cdd376b8bd4ec75d337df1ae6307c429836ba23..78bd9c44338427b0e31bdbba272301e7035aa579 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartition.java @@ -46,7 +46,7 @@ public class IntermediateResultPartition { this.producer = producer; this.partitionNumber = partitionNumber; this.consumers = new ArrayList>(0); - this.partitionId = new IntermediateResultPartitionID(); + this.partitionId = new IntermediateResultPartitionID(totalResult.getId(), partitionNumber); } public ExecutionVertex getProducer() { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/AdaptedRestartPipelinedRegionStrategyNG.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/AdaptedRestartPipelinedRegionStrategyNG.java index 21d503ee221a4665b62d7d3771f91688edb2bbe2..36dfa13fc4c351c788c0bf77ec7ef982f7054ab6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/AdaptedRestartPipelinedRegionStrategyNG.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/AdaptedRestartPipelinedRegionStrategyNG.java @@ -294,7 +294,7 @@ public class AdaptedRestartPipelinedRegionStrategyNG extends FailoverStrategy { // currently it's safe to add it here, as this method is invoked only once in production code. checkState(restartPipelinedRegionFailoverStrategy == null, "notifyNewVertices() must be called only once"); this.restartPipelinedRegionFailoverStrategy = new RestartPipelinedRegionFailoverStrategy( - executionGraph.getFailoverTopology(), + executionGraph.getSchedulingTopology(), executionGraph.getResultPartitionAvailabilityChecker()); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExecutionFailureHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExecutionFailureHandler.java index a766a3e5e82a83d8246abf92c22b952340fae218..63d5e8840dec72deb3f10afe8014e399b9f8c704 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExecutionFailureHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExecutionFailureHandler.java @@ -20,6 +20,8 @@ package org.apache.flink.runtime.executiongraph.failover.flip1; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.runtime.JobException; import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; +import org.apache.flink.runtime.scheduler.strategy.SchedulingExecutionVertex; +import org.apache.flink.runtime.scheduler.strategy.SchedulingTopology; import org.apache.flink.runtime.throwable.ThrowableClassifier; import org.apache.flink.runtime.throwable.ThrowableType; import org.apache.flink.util.IterableUtils; @@ -36,7 +38,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull; */ public class ExecutionFailureHandler { - private final FailoverTopology failoverTopology; + private final SchedulingTopology schedulingTopology; /** Strategy to judge which tasks should be restarted. */ private final FailoverStrategy failoverStrategy; @@ -50,16 +52,16 @@ public class ExecutionFailureHandler { /** * Creates the handler to deal with task failures. * - * @param failoverTopology contains the topology info for failover + * @param schedulingTopology contains the topology info for failover * @param failoverStrategy helps to decide tasks to restart on task failures * @param restartBackoffTimeStrategy helps to decide whether to restart failed tasks and the restarting delay */ public ExecutionFailureHandler( - final FailoverTopology failoverTopology, + final SchedulingTopology schedulingTopology, final FailoverStrategy failoverStrategy, final RestartBackoffTimeStrategy restartBackoffTimeStrategy) { - this.failoverTopology = checkNotNull(failoverTopology); + this.schedulingTopology = checkNotNull(schedulingTopology); this.failoverStrategy = checkNotNull(failoverStrategy); this.restartBackoffTimeStrategy = checkNotNull(restartBackoffTimeStrategy); } @@ -86,8 +88,8 @@ public class ExecutionFailureHandler { public FailureHandlingResult getGlobalFailureHandlingResult(final Throwable cause) { return handleFailure( cause, - IterableUtils.toStream(failoverTopology.getVertices()) - .map(FailoverVertex::getId) + IterableUtils.toStream(schedulingTopology.getVertices()) + .map(SchedulingExecutionVertex::getId) .collect(Collectors.toSet())); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverStrategy.java index 798d88b6220742bff5d159ff828ff218a7fdb9cb..981b198ea52b0f6dff16937fda895964dbdfe16f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverStrategy.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.executiongraph.failover.flip1; import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; +import org.apache.flink.runtime.scheduler.strategy.SchedulingTopology; import java.util.Set; @@ -52,7 +53,7 @@ public interface FailoverStrategy { * @return The instantiated failover strategy. */ FailoverStrategy create( - FailoverTopology topology, + SchedulingTopology topology, ResultPartitionAvailabilityChecker resultPartitionAvailabilityChecker); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/PipelinedRegionComputeUtil.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/PipelinedRegionComputeUtil.java index 856116b21bb12a30bb53d69ab791018898ade857..52d96f4b4ac1696f8ff733e48dbc23db52132e6d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/PipelinedRegionComputeUtil.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/PipelinedRegionComputeUtil.java @@ -19,11 +19,8 @@ package org.apache.flink.runtime.executiongraph.failover.flip1; -import org.apache.flink.runtime.executiongraph.failover.flip1.partitionrelease.PipelinedRegion; -import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; -import org.apache.flink.runtime.scheduler.strategy.SchedulingExecutionVertex; +import org.apache.flink.runtime.topology.BaseTopology; import org.apache.flink.runtime.topology.Result; -import org.apache.flink.runtime.topology.Topology; import org.apache.flink.runtime.topology.Vertex; import org.slf4j.Logger; @@ -34,8 +31,6 @@ import java.util.HashSet; import java.util.IdentityHashMap; import java.util.Map; import java.util.Set; -import java.util.function.Function; -import java.util.stream.Collectors; /** * Utility for computing pipelined regions. @@ -44,23 +39,8 @@ public final class PipelinedRegionComputeUtil { private static final Logger LOG = LoggerFactory.getLogger(PipelinedRegionComputeUtil.class); - public static Set toPipelinedRegionsSet( - final Set>> distinctRegions) { - - return distinctRegions.stream() - .map(toExecutionVertexIdSet()) - .map(PipelinedRegion::from) - .collect(Collectors.toSet()); - } - - private static Function>, Set> toExecutionVertexIdSet() { - return failoverVertices -> failoverVertices.stream() - .map(SchedulingExecutionVertex::getId) - .collect(Collectors.toSet()); - } - public static , R extends Result> Set> computePipelinedRegions( - final Topology topology) { + final BaseTopology topology) { // currently we let a job with co-location constraints fail as one region // putting co-located vertices in the same region with each other can be a future improvement @@ -72,7 +52,7 @@ public final class PipelinedRegionComputeUtil { // iterate all the vertices which are topologically sorted for (V vertex : topology.getVertices()) { - Set currentRegion = new HashSet<>(1); + Set currentRegion = new HashSet<>(); currentRegion.add(vertex); vertexToRegion.put(vertex, currentRegion); @@ -115,7 +95,7 @@ public final class PipelinedRegionComputeUtil { } private static > Map> buildOneRegionForAllVertices( - final Topology topology) { + final BaseTopology topology) { LOG.warn("Cannot decompose the topology into individual failover regions due to use of " + "Co-Location constraints (iterations). Job will fail over as one holistic unit."); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartAllFailoverStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartAllFailoverStrategy.java index f7f0d5770fb1cb29fdb4a0accea0f42f189788c9..2bd09dae133d9ad59d2bde1599ae554977a5fb7c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartAllFailoverStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartAllFailoverStrategy.java @@ -18,6 +18,8 @@ package org.apache.flink.runtime.executiongraph.failover.flip1; import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; +import org.apache.flink.runtime.scheduler.strategy.SchedulingExecutionVertex; +import org.apache.flink.runtime.scheduler.strategy.SchedulingTopology; import org.apache.flink.util.IterableUtils; import java.util.Set; @@ -30,9 +32,9 @@ import static org.apache.flink.util.Preconditions.checkNotNull; */ public class RestartAllFailoverStrategy implements FailoverStrategy { - private final FailoverTopology topology; + private final SchedulingTopology topology; - public RestartAllFailoverStrategy(final FailoverTopology topology) { + public RestartAllFailoverStrategy(final SchedulingTopology topology) { this.topology = checkNotNull(topology); } @@ -46,7 +48,7 @@ public class RestartAllFailoverStrategy implements FailoverStrategy { @Override public Set getTasksNeedingRestart(ExecutionVertexID executionVertexId, Throwable cause) { return IterableUtils.toStream(topology.getVertices()) - .map(FailoverVertex::getId) + .map(SchedulingExecutionVertex::getId) .collect(Collectors.toSet()); } @@ -57,7 +59,7 @@ public class RestartAllFailoverStrategy implements FailoverStrategy { @Override public FailoverStrategy create( - final FailoverTopology topology, + final SchedulingTopology topology, final ResultPartitionAvailabilityChecker resultPartitionAvailabilityChecker) { return new RestartAllFailoverStrategy(topology); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartPipelinedRegionFailoverStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartPipelinedRegionFailoverStrategy.java index cdc6754298e9c5712ece960f1f85b90c48804b03..eb8d06b12b15fb214f069cc72c78198fd1d1b951 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartPipelinedRegionFailoverStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartPipelinedRegionFailoverStrategy.java @@ -21,6 +21,10 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.runtime.io.network.partition.PartitionException; import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; +import org.apache.flink.runtime.scheduler.strategy.SchedulingExecutionVertex; +import org.apache.flink.runtime.scheduler.strategy.SchedulingPipelinedRegion; +import org.apache.flink.runtime.scheduler.strategy.SchedulingResultPartition; +import org.apache.flink.runtime.scheduler.strategy.SchedulingTopology; import org.apache.flink.util.ExceptionUtils; import org.slf4j.Logger; @@ -28,10 +32,8 @@ import org.slf4j.LoggerFactory; import java.util.ArrayDeque; import java.util.Collections; -import java.util.HashMap; import java.util.HashSet; import java.util.IdentityHashMap; -import java.util.Map; import java.util.Optional; import java.util.Queue; import java.util.Set; @@ -48,13 +50,7 @@ public class RestartPipelinedRegionFailoverStrategy implements FailoverStrategy private static final Logger LOG = LoggerFactory.getLogger(RestartPipelinedRegionFailoverStrategy.class); /** The topology containing info about all the vertices and result partitions. */ - private final FailoverTopology topology; - - /** All failover regions. */ - private final Set regions; - - /** Maps execution vertex id to failover region. */ - private final Map vertexToRegionMap; + private final SchedulingTopology topology; /** The checker helps to query result partition availability. */ private final RegionFailoverResultPartitionAvailabilityChecker resultPartitionAvailabilityChecker; @@ -66,7 +62,7 @@ public class RestartPipelinedRegionFailoverStrategy implements FailoverStrategy * @param topology containing info about all the vertices and result partitions */ @VisibleForTesting - public RestartPipelinedRegionFailoverStrategy(FailoverTopology topology) { + public RestartPipelinedRegionFailoverStrategy(SchedulingTopology topology) { this(topology, resultPartitionID -> true); } @@ -77,38 +73,12 @@ public class RestartPipelinedRegionFailoverStrategy implements FailoverStrategy * @param resultPartitionAvailabilityChecker helps to query result partition availability */ public RestartPipelinedRegionFailoverStrategy( - FailoverTopology topology, + SchedulingTopology topology, ResultPartitionAvailabilityChecker resultPartitionAvailabilityChecker) { this.topology = checkNotNull(topology); - this.regions = Collections.newSetFromMap(new IdentityHashMap<>()); - this.vertexToRegionMap = new HashMap<>(); this.resultPartitionAvailabilityChecker = new RegionFailoverResultPartitionAvailabilityChecker( resultPartitionAvailabilityChecker); - - // build regions based on the given topology - LOG.info("Start building failover regions."); - buildFailoverRegions(); - } - // ------------------------------------------------------------------------ - // region building - // ------------------------------------------------------------------------ - - private void buildFailoverRegions() { - final Set>> distinctRegions = - PipelinedRegionComputeUtil.computePipelinedRegions(topology); - - // creating all the failover regions and register them - for (Set> regionVertices : distinctRegions) { - LOG.debug("Creating a failover region with {} vertices.", regionVertices.size()); - final FailoverRegion failoverRegion = new FailoverRegion(regionVertices); - regions.add(failoverRegion); - for (FailoverVertex vertex : regionVertices) { - vertexToRegionMap.put(vertex.getId(), failoverRegion); - } - } - - LOG.info("Created {} failover regions.", regions.size()); } @@ -133,7 +103,7 @@ public class RestartPipelinedRegionFailoverStrategy implements FailoverStrategy public Set getTasksNeedingRestart(ExecutionVertexID executionVertexId, Throwable cause) { LOG.info("Calculating tasks to restart to recover the failed task {}.", executionVertexId); - final FailoverRegion failedRegion = vertexToRegionMap.get(executionVertexId); + final SchedulingPipelinedRegion failedRegion = topology.getPipelinedRegionOfVertex(executionVertexId); if (failedRegion == null) { // TODO: show the task name in the log throw new IllegalStateException("Can not find the failover region for task " + executionVertexId, cause); @@ -150,8 +120,8 @@ public class RestartPipelinedRegionFailoverStrategy implements FailoverStrategy // calculate the tasks to restart based on the result of regions to restart Set tasksToRestart = new HashSet<>(); - for (FailoverRegion region : getRegionsToRestart(failedRegion)) { - tasksToRestart.addAll(region.getAllExecutionVertexIDs()); + for (SchedulingPipelinedRegion region : getRegionsToRestart(failedRegion)) { + region.getVertices().forEach(vertex -> tasksToRestart.add(vertex.getId())); } // the previous failed partition will be recovered. remove its failed state from the checker @@ -172,25 +142,25 @@ public class RestartPipelinedRegionFailoverStrategy implements FailoverStrategy * the region containing the partition producer task is involved * 3. If a region is involved, all of its consumer regions are involved */ - private Set getRegionsToRestart(FailoverRegion failedRegion) { - Set regionsToRestart = Collections.newSetFromMap(new IdentityHashMap<>()); - Set visitedRegions = Collections.newSetFromMap(new IdentityHashMap<>()); + private Set getRegionsToRestart(SchedulingPipelinedRegion failedRegion) { + Set regionsToRestart = Collections.newSetFromMap(new IdentityHashMap<>()); + Set visitedRegions = Collections.newSetFromMap(new IdentityHashMap<>()); // start from the failed region to visit all involved regions - Queue regionsToVisit = new ArrayDeque<>(); + Queue regionsToVisit = new ArrayDeque<>(); visitedRegions.add(failedRegion); regionsToVisit.add(failedRegion); while (!regionsToVisit.isEmpty()) { - FailoverRegion regionToRestart = regionsToVisit.poll(); + SchedulingPipelinedRegion regionToRestart = regionsToVisit.poll(); // an involved region should be restarted regionsToRestart.add(regionToRestart); // if a needed input result partition is not available, its producer region is involved - for (FailoverVertex vertex : regionToRestart.getAllExecutionVertices()) { - for (FailoverResultPartition consumedPartition : vertex.getConsumedResults()) { + for (SchedulingExecutionVertex vertex : regionToRestart.getVertices()) { + for (SchedulingResultPartition consumedPartition : vertex.getConsumedResults()) { if (!resultPartitionAvailabilityChecker.isAvailable(consumedPartition.getId())) { - FailoverRegion producerRegion = vertexToRegionMap.get(consumedPartition.getProducer().getId()); + SchedulingPipelinedRegion producerRegion = topology.getPipelinedRegionOfVertex(consumedPartition.getProducer().getId()); if (!visitedRegions.contains(producerRegion)) { visitedRegions.add(producerRegion); regionsToVisit.add(producerRegion); @@ -200,10 +170,10 @@ public class RestartPipelinedRegionFailoverStrategy implements FailoverStrategy } // all consumer regions of an involved region should be involved - for (FailoverVertex vertex : regionToRestart.getAllExecutionVertices()) { - for (FailoverResultPartition producedPartition : vertex.getProducedResults()) { - for (FailoverVertex consumerVertex : producedPartition.getConsumers()) { - FailoverRegion consumerRegion = vertexToRegionMap.get(consumerVertex.getId()); + for (SchedulingExecutionVertex vertex : regionToRestart.getVertices()) { + for (SchedulingResultPartition producedPartition : vertex.getProducedResults()) { + for (SchedulingExecutionVertex consumerVertex : producedPartition.getConsumers()) { + SchedulingPipelinedRegion consumerRegion = topology.getPipelinedRegionOfVertex(consumerVertex.getId()); if (!visitedRegions.contains(consumerRegion)) { visitedRegions.add(consumerRegion); regionsToVisit.add(consumerRegion); @@ -226,8 +196,8 @@ public class RestartPipelinedRegionFailoverStrategy implements FailoverStrategy * @return the failover region that contains the given execution vertex */ @VisibleForTesting - public FailoverRegion getFailoverRegion(ExecutionVertexID vertexID) { - return vertexToRegionMap.get(vertexID); + public SchedulingPipelinedRegion getFailoverRegion(ExecutionVertexID vertexID) { + return topology.getPipelinedRegionOfVertex(vertexID); } /** @@ -268,7 +238,7 @@ public class RestartPipelinedRegionFailoverStrategy implements FailoverStrategy @Override public FailoverStrategy create( - final FailoverTopology topology, + final SchedulingTopology topology, final ResultPartitionAvailabilityChecker resultPartitionAvailabilityChecker) { return new RestartPipelinedRegionFailoverStrategy(topology, resultPartitionAvailabilityChecker); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/NotReleasingPartitionReleaseStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/NotReleasingPartitionReleaseStrategy.java index 019a5954472097802994655f85b41bbef9d7b170..7f2eaa89a7c5325193ab4ae60d25f3b36c51bf14 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/NotReleasingPartitionReleaseStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/NotReleasingPartitionReleaseStrategy.java @@ -47,7 +47,7 @@ public class NotReleasingPartitionReleaseStrategy implements PartitionReleaseStr public static class Factory implements PartitionReleaseStrategy.Factory { @Override - public PartitionReleaseStrategy createInstance(final SchedulingTopology schedulingStrategy) { + public PartitionReleaseStrategy createInstance(final SchedulingTopology schedulingStrategy) { return new NotReleasingPartitionReleaseStrategy(); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/PartitionReleaseStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/PartitionReleaseStrategy.java index 7002ced6e1c6ab6225167565ac8f633d2721581f..2f590c94201270613061673f6b707970e85c8c09 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/PartitionReleaseStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/PartitionReleaseStrategy.java @@ -52,6 +52,6 @@ public interface PartitionReleaseStrategy { * Factory for {@link PartitionReleaseStrategy}. */ interface Factory { - PartitionReleaseStrategy createInstance(SchedulingTopology schedulingStrategy); + PartitionReleaseStrategy createInstance(SchedulingTopology schedulingStrategy); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/PipelinedRegionExecutionView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/PipelinedRegionExecutionView.java index c92fa8b1b767da80fcf81edd3eba9f533ca4c684..493d64a96c99634c326fa72b729acc7cdf6022a7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/PipelinedRegionExecutionView.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/PipelinedRegionExecutionView.java @@ -20,28 +20,32 @@ package org.apache.flink.runtime.executiongraph.failover.flip1.partitionrelease; import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; +import org.apache.flink.runtime.scheduler.strategy.SchedulingExecutionVertex; +import org.apache.flink.runtime.scheduler.strategy.SchedulingPipelinedRegion; +import org.apache.flink.util.IterableUtils; -import java.util.HashSet; import java.util.Set; +import java.util.stream.Collectors; -import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; /** - * Provides a virtual execution state of a {@link PipelinedRegion}. + * Provides a virtual execution state of a {@link SchedulingPipelinedRegion}. * *

A pipelined region can be either finished or unfinished. It is finished iff. all its * executions have reached the finished state. */ class PipelinedRegionExecutionView { - private final PipelinedRegion pipelinedRegion; + private final SchedulingPipelinedRegion pipelinedRegion; private final Set unfinishedVertices; - PipelinedRegionExecutionView(final PipelinedRegion pipelinedRegion) { + PipelinedRegionExecutionView(final SchedulingPipelinedRegion pipelinedRegion) { this.pipelinedRegion = checkNotNull(pipelinedRegion); - this.unfinishedVertices = new HashSet<>(pipelinedRegion.getExecutionVertexIds()); + this.unfinishedVertices = IterableUtils.toStream(pipelinedRegion.getVertices()) + .map(SchedulingExecutionVertex::getId) + .collect(Collectors.toSet()); } public boolean isFinished() { @@ -49,16 +53,16 @@ class PipelinedRegionExecutionView { } public void vertexFinished(final ExecutionVertexID executionVertexId) { - checkArgument(pipelinedRegion.contains(executionVertexId)); + assertVertexInRegion(executionVertexId); unfinishedVertices.remove(executionVertexId); } public void vertexUnfinished(final ExecutionVertexID executionVertexId) { - checkArgument(pipelinedRegion.contains(executionVertexId)); + assertVertexInRegion(executionVertexId); unfinishedVertices.add(executionVertexId); } - public PipelinedRegion getPipelinedRegion() { - return pipelinedRegion; + private void assertVertexInRegion(final ExecutionVertexID executionVertexId) { + pipelinedRegion.getVertex(executionVertexId); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/RegionPartitionReleaseStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/RegionPartitionReleaseStrategy.java index a05a24a407bc297c397cf0a252651568243133a7..f561a6f4cdaf87ad2ab6a94daca6f0343a2ab90a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/RegionPartitionReleaseStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/RegionPartitionReleaseStrategy.java @@ -19,105 +19,56 @@ package org.apache.flink.runtime.executiongraph.failover.flip1.partitionrelease; -import org.apache.flink.runtime.executiongraph.failover.flip1.PipelinedRegionComputeUtil; import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; import org.apache.flink.runtime.scheduler.strategy.SchedulingExecutionVertex; +import org.apache.flink.runtime.scheduler.strategy.SchedulingPipelinedRegion; import org.apache.flink.runtime.scheduler.strategy.SchedulingResultPartition; import org.apache.flink.runtime.scheduler.strategy.SchedulingTopology; import org.apache.flink.util.IterableUtils; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; -import java.util.IdentityHashMap; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.stream.Collectors; import static org.apache.flink.util.Preconditions.checkNotNull; import static org.apache.flink.util.Preconditions.checkState; /** - * Releases blocking intermediate result partitions that are incident to a {@link PipelinedRegion}, + * Releases blocking intermediate result partitions that are incident to a {@link SchedulingPipelinedRegion}, * as soon as the region's execution vertices are finished. */ public class RegionPartitionReleaseStrategy implements PartitionReleaseStrategy { - private final SchedulingTopology schedulingTopology; - - private final Map consumedBlockingPartitionsByRegion = new IdentityHashMap<>(); + private final SchedulingTopology schedulingTopology; private final Map regionExecutionViewByVertex = new HashMap<>(); - public RegionPartitionReleaseStrategy( - final SchedulingTopology schedulingTopology, - final Set pipelinedRegions) { - + public RegionPartitionReleaseStrategy(final SchedulingTopology schedulingTopology) { this.schedulingTopology = checkNotNull(schedulingTopology); - checkNotNull(pipelinedRegions); - initConsumedBlockingPartitionsByRegion(pipelinedRegions); - initRegionExecutionViewByVertex(pipelinedRegions); - } - - private void initConsumedBlockingPartitionsByRegion(final Set pipelinedRegions) { - for (PipelinedRegion pipelinedRegion : pipelinedRegions) { - final PipelinedRegionConsumedBlockingPartitions consumedPartitions = computeConsumedPartitionsOfVertexRegion(pipelinedRegion); - consumedBlockingPartitionsByRegion.put(pipelinedRegion, consumedPartitions); - } + initRegionExecutionViewByVertex(); } - private void initRegionExecutionViewByVertex(final Set pipelinedRegions) { - for (PipelinedRegion pipelinedRegion : pipelinedRegions) { + private void initRegionExecutionViewByVertex() { + for (SchedulingPipelinedRegion pipelinedRegion : schedulingTopology.getAllPipelinedRegions()) { final PipelinedRegionExecutionView regionExecutionView = new PipelinedRegionExecutionView(pipelinedRegion); - for (ExecutionVertexID executionVertexId : pipelinedRegion) { - regionExecutionViewByVertex.put(executionVertexId, regionExecutionView); + for (SchedulingExecutionVertex executionVertexId : pipelinedRegion.getVertices()) { + regionExecutionViewByVertex.put(executionVertexId.getId(), regionExecutionView); } } } - private PipelinedRegionConsumedBlockingPartitions computeConsumedPartitionsOfVertexRegion(final PipelinedRegion pipelinedRegion) { - final Set resultPartitionsOutsideOfRegion = findResultPartitionsOutsideOfRegion(pipelinedRegion); - return new PipelinedRegionConsumedBlockingPartitions(pipelinedRegion, resultPartitionsOutsideOfRegion); - } - - private Set findResultPartitionsOutsideOfRegion(final PipelinedRegion pipelinedRegion) { - final Set> allConsumedPartitionsInRegion = pipelinedRegion - .getExecutionVertexIds() - .stream() - .map(schedulingTopology::getVertexOrThrow) - .flatMap(vertex -> IterableUtils.toStream(vertex.getConsumedResults())) - .collect(Collectors.toSet()); - - return filterResultPartitionsOutsideOfRegion(allConsumedPartitionsInRegion, pipelinedRegion); - } - - private static Set filterResultPartitionsOutsideOfRegion( - final Collection> resultPartitions, - final PipelinedRegion pipelinedRegion) { - - final Set result = new HashSet<>(); - for (final SchedulingResultPartition maybeOutsidePartition : resultPartitions) { - final SchedulingExecutionVertex producer = maybeOutsidePartition.getProducer(); - if (!pipelinedRegion.contains(producer.getId())) { - result.add(maybeOutsidePartition.getId()); - } - } - return result; - } - @Override public List vertexFinished(final ExecutionVertexID finishedVertex) { final PipelinedRegionExecutionView regionExecutionView = getPipelinedRegionExecutionViewForVertex(finishedVertex); regionExecutionView.vertexFinished(finishedVertex); if (regionExecutionView.isFinished()) { - final PipelinedRegion pipelinedRegion = getPipelinedRegionForVertex(finishedVertex); - final PipelinedRegionConsumedBlockingPartitions consumedPartitionsOfVertexRegion = getConsumedBlockingPartitionsForRegion(pipelinedRegion); - return filterReleasablePartitions(consumedPartitionsOfVertexRegion); + final SchedulingPipelinedRegion pipelinedRegion = schedulingTopology.getPipelinedRegionOfVertex(finishedVertex); + return filterReleasablePartitions(pipelinedRegion.getConsumedResults()); } return Collections.emptyList(); } @@ -135,29 +86,15 @@ public class RegionPartitionReleaseStrategy implements PartitionReleaseStrategy return pipelinedRegionExecutionView; } - private PipelinedRegion getPipelinedRegionForVertex(final ExecutionVertexID executionVertexId) { - final PipelinedRegionExecutionView pipelinedRegionExecutionView = getPipelinedRegionExecutionViewForVertex(executionVertexId); - return pipelinedRegionExecutionView.getPipelinedRegion(); - } - - private PipelinedRegionConsumedBlockingPartitions getConsumedBlockingPartitionsForRegion(final PipelinedRegion pipelinedRegion) { - final PipelinedRegionConsumedBlockingPartitions pipelinedRegionConsumedBlockingPartitions = consumedBlockingPartitionsByRegion.get(pipelinedRegion); - checkState(pipelinedRegionConsumedBlockingPartitions != null, - "Consumed partitions not found for pipelined region %s", pipelinedRegion); - checkState(pipelinedRegionConsumedBlockingPartitions.getPipelinedRegion() == pipelinedRegion); - return pipelinedRegionConsumedBlockingPartitions; - } - - private List filterReleasablePartitions(final PipelinedRegionConsumedBlockingPartitions consumedPartitionsOfVertexRegion) { - return consumedPartitionsOfVertexRegion - .getConsumedBlockingPartitions() - .stream() + private List filterReleasablePartitions(final Iterable schedulingResultPartitions) { + return IterableUtils.toStream(schedulingResultPartitions) + .map(SchedulingResultPartition::getId) .filter(this::areConsumerRegionsFinished) .collect(Collectors.toList()); } private boolean areConsumerRegionsFinished(final IntermediateResultPartitionID resultPartitionId) { - final SchedulingResultPartition resultPartition = schedulingTopology.getResultPartitionOrThrow(resultPartitionId); + final SchedulingResultPartition resultPartition = schedulingTopology.getResultPartition(resultPartitionId); return IterableUtils.toStream(resultPartition.getConsumers()) .map(SchedulingExecutionVertex::getId) .allMatch(this::isRegionOfVertexFinished); @@ -174,14 +111,8 @@ public class RegionPartitionReleaseStrategy implements PartitionReleaseStrategy public static class Factory implements PartitionReleaseStrategy.Factory { @Override - public PartitionReleaseStrategy createInstance(final SchedulingTopology schedulingStrategy) { - - final Set>> distinctRegions = - PipelinedRegionComputeUtil.computePipelinedRegions(schedulingStrategy); - - return new RegionPartitionReleaseStrategy( - schedulingStrategy, - PipelinedRegionComputeUtil.toPipelinedRegionsSet(distinctRegions)); + public PartitionReleaseStrategy createInstance(final SchedulingTopology schedulingStrategy) { + return new RegionPartitionReleaseStrategy(schedulingStrategy); } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServicesUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServicesUtils.java index 24219f409de14b1e05ef2d3fd672b3374587c97f..c9d82d2cf636da08301ec212f7fad179495cd466 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServicesUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServicesUtils.java @@ -34,7 +34,6 @@ import org.apache.flink.runtime.highavailability.nonha.standalone.StandaloneHaSe import org.apache.flink.runtime.highavailability.zookeeper.ZooKeeperClientHAServices; import org.apache.flink.runtime.highavailability.zookeeper.ZooKeeperHaServices; import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; -import org.apache.flink.runtime.jobmaster.JobMaster; import org.apache.flink.runtime.net.SSLUtils; import org.apache.flink.runtime.resourcemanager.ResourceManager; import org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils; @@ -94,22 +93,16 @@ public class HighAvailabilityServicesUtils { case NONE: final Tuple2 hostnamePort = getJobManagerAddress(configuration); - final String jobManagerRpcUrl = AkkaRpcServiceUtils.getRpcUrl( - hostnamePort.f0, - hostnamePort.f1, - JobMaster.JOB_MANAGER_NAME, - addressResolution, - configuration); final String resourceManagerRpcUrl = AkkaRpcServiceUtils.getRpcUrl( hostnamePort.f0, hostnamePort.f1, - ResourceManager.RESOURCE_MANAGER_NAME, + AkkaRpcServiceUtils.createWildcardName(ResourceManager.RESOURCE_MANAGER_NAME), addressResolution, configuration); final String dispatcherRpcUrl = AkkaRpcServiceUtils.getRpcUrl( hostnamePort.f0, hostnamePort.f1, - Dispatcher.DISPATCHER_NAME, + AkkaRpcServiceUtils.createWildcardName(Dispatcher.DISPATCHER_NAME), addressResolution, configuration); final String webMonitorAddress = getWebMonitorAddress( @@ -119,7 +112,6 @@ public class HighAvailabilityServicesUtils { return new StandaloneHaServices( resourceManagerRpcUrl, dispatcherRpcUrl, - jobManagerRpcUrl, webMonitorAddress); case ZOOKEEPER: BlobStoreService blobStoreService = BlobUtils.createBlobStoreFromConfig(configuration); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/standalone/StandaloneHaServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/standalone/StandaloneHaServices.java index 854a04a63dd1d9703374b320c59a165a5d434808..7731476d358d7d4ba5c591c7218f2a85336e7b77 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/standalone/StandaloneHaServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/standalone/StandaloneHaServices.java @@ -32,25 +32,19 @@ import static org.apache.flink.util.Preconditions.checkNotNull; * An implementation of the {@link HighAvailabilityServices} for the non-high-availability case. * This implementation can be used for testing, and for cluster setups that do not * tolerate failures of the master processes (JobManager, ResourceManager). - * + * *

This implementation has no dependencies on any external services. It returns a fix * pre-configured ResourceManager and JobManager, and stores checkpoints and metadata simply on the * heap or on a local file system and therefore in a storage without guarantees. */ public class StandaloneHaServices extends AbstractNonHaServices { - /** The constant name of the ResourceManager RPC endpoint */ - private static final String RESOURCE_MANAGER_RPC_ENDPOINT_NAME = "resource_manager"; - - /** The fix address of the ResourceManager */ + /** The fix address of the ResourceManager. */ private final String resourceManagerAddress; - /** The fix address of the Dispatcher */ + /** The fix address of the Dispatcher. */ private final String dispatcherAddress; - /** The fix address of the JobManager */ - private final String jobManagerAddress; - private final String clusterRestEndpointAddress; /** @@ -62,11 +56,9 @@ public class StandaloneHaServices extends AbstractNonHaServices { public StandaloneHaServices( String resourceManagerAddress, String dispatcherAddress, - String jobManagerAddress, String clusterRestEndpointAddress) { this.resourceManagerAddress = checkNotNull(resourceManagerAddress, "resourceManagerAddress"); this.dispatcherAddress = checkNotNull(dispatcherAddress, "dispatcherAddress"); - this.jobManagerAddress = checkNotNull(jobManagerAddress, "jobManagerAddress"); this.clusterRestEndpointAddress = checkNotNull(clusterRestEndpointAddress, clusterRestEndpointAddress); } @@ -116,7 +108,7 @@ public class StandaloneHaServices extends AbstractNonHaServices { synchronized (lock) { checkNotShutdown(); - return new StandaloneLeaderRetrievalService(jobManagerAddress, DEFAULT_LEADER_ID); + return new StandaloneLeaderRetrievalService("UNKNOWN", DEFAULT_LEADER_ID); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileChannelReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileChannelReader.java index 16178d9e5ff5fc066d34c7dd356f1a120b027ae5..3b935fee271e20f90649608addc82927c0419839 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileChannelReader.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileChannelReader.java @@ -62,10 +62,7 @@ public class BufferFileChannelReader { fileChannel.read(buffer.getNioBuffer(0, size)); buffer.setSize(size); - - if (!isBuffer) { - buffer.tagAsEvent(); - } + buffer.setDataType(isBuffer ? Buffer.DataType.DATA_BUFFER : Buffer.DataType.EVENT_BUFFER); return fileChannel.size() - fileChannel.position() == 0; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironment.java index 2d2d6f35ff12560d36597fb8e1d73a6eafd2133f..8938f5eab30151e62d6493cdae5c05fa1f1abf86 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironment.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironment.java @@ -53,6 +53,7 @@ import org.slf4j.LoggerFactory; import java.io.IOException; import java.util.Arrays; import java.util.Collection; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.concurrent.ConcurrentHashMap; @@ -183,16 +184,18 @@ public class NettyShuffleEnvironment implements ShuffleEnvironment createResultPartitionWriters( + public List createResultPartitionWriters( ShuffleIOOwnerContext ownerContext, - Collection resultPartitionDeploymentDescriptors) { + List resultPartitionDeploymentDescriptors) { synchronized (lock) { Preconditions.checkState(!isClosed, "The NettyShuffleEnvironment has already been shut down."); ResultPartition[] resultPartitions = new ResultPartition[resultPartitionDeploymentDescriptors.size()]; - int counter = 0; - for (ResultPartitionDeploymentDescriptor rpdd : resultPartitionDeploymentDescriptors) { - resultPartitions[counter++] = resultPartitionFactory.create(ownerContext.getOwnerName(), rpdd); + for (int partitionIndex = 0; partitionIndex < resultPartitions.length; partitionIndex++) { + resultPartitions[partitionIndex] = resultPartitionFactory.create( + ownerContext.getOwnerName(), + partitionIndex, + resultPartitionDeploymentDescriptors.get(partitionIndex)); } registerOutputMetrics(config.isNetworkDetailedMetrics(), ownerContext.getOutputGroup(), resultPartitions); @@ -201,10 +204,10 @@ public class NettyShuffleEnvironment implements ShuffleEnvironment createInputGates( + public List createInputGates( ShuffleIOOwnerContext ownerContext, PartitionProducerStateProvider partitionProducerStateProvider, - Collection inputGateDeploymentDescriptors) { + List inputGateDeploymentDescriptors) { synchronized (lock) { Preconditions.checkState(!isClosed, "The NettyShuffleEnvironment has already been shut down."); @@ -213,17 +216,18 @@ public class NettyShuffleEnvironment implements ShuffleEnvironment inputGatesById.remove(id)); - inputGates[counter++] = inputGate; + inputGates[gateIndex] = inputGate; } registerInputMetrics(config.isNetworkDetailedMetrics(), networkInputGroup, inputGates); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkClientHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkClientHandler.java index 7f631b4ea3d3f7663243d546e91984fe9ce0cd55..56459f988f3923a664dbfa2f58038c13b182e443 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkClientHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkClientHandler.java @@ -23,6 +23,8 @@ import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler; +import javax.annotation.Nullable; + import java.io.IOException; /** @@ -34,6 +36,9 @@ public interface NetworkClientHandler extends ChannelHandler { void removeInputChannel(RemoteInputChannel inputChannel); + @Nullable + RemoteInputChannel getInputChannel(InputChannelID inputChannelId); + void cancelRequestFor(InputChannelID inputChannelId); /** @@ -44,4 +49,11 @@ public interface NetworkClientHandler extends ChannelHandler { * @param inputChannel The input channel with unannounced credits. */ void notifyCreditAvailable(final RemoteInputChannel inputChannel); + + /** + * Resumes data consumption from the producer after an exactly once checkpoint. + * + * @param inputChannel The input channel to resume data consumption. + */ + void resumeConsumption(RemoteInputChannel inputChannel); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkSequenceViewReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkSequenceViewReader.java index 0c677601e5ff2324c4592916e42fcc71645f2a33..6283aaa595d71fd0f60c3e98682b4a6d4be61466 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkSequenceViewReader.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkSequenceViewReader.java @@ -36,7 +36,7 @@ public interface NetworkSequenceViewReader { ResultPartitionID resultPartitionId, int subPartitionIndex) throws IOException; - BufferAndAvailability getNextBuffer() throws IOException, InterruptedException; + BufferAndAvailability getNextBuffer() throws IOException; /** * The credits from consumer are added in incremental way. @@ -45,6 +45,11 @@ public interface NetworkSequenceViewReader { */ void addCredit(int creditDeltas); + /** + * Resumes data consumption after an exactly once checkpoint. + */ + void resumeConsumption(); + /** * Checks whether this reader is available or not. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/PartitionRequestClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/PartitionRequestClient.java index a215700f1cfdb134532966b6c8fbf65bde6a2b62..4afbab43d7e49dd8392c4e685fe96635ba059a88 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/PartitionRequestClient.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/PartitionRequestClient.java @@ -50,6 +50,13 @@ public interface PartitionRequestClient { */ void notifyCreditAvailable(RemoteInputChannel inputChannel); + /** + * Requests to resume data consumption from one remote input channel. + * + * @param inputChannel The remote input channel who is ready to resume data consumption. + */ + void resumeConsumption(RemoteInputChannel inputChannel); + /** * Sends a task event backwards to an intermediate result partition. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/EventSerializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/EventSerializer.java index 3eea00f7b937e62b570a74c9e5891e02a1b275e7..b0b82b226d9c8dcf782203fcba1be3669483b6da 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/EventSerializer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/EventSerializer.java @@ -208,7 +208,7 @@ public class EventSerializer { final byte[] locationBytes = checkpointOptions.getTargetLocation().isDefaultReference() ? null : checkpointOptions.getTargetLocation().getReferenceBytes(); - final ByteBuffer buf = ByteBuffer.allocate(28 + (locationBytes == null ? 0 : locationBytes.length)); + final ByteBuffer buf = ByteBuffer.allocate(30 + (locationBytes == null ? 0 : locationBytes.length)); // we do not use checkpointType.ordinal() here to make the serialization robust // against changes in the enum (such as changes in the order of the values) @@ -234,6 +234,8 @@ public class EventSerializer { buf.putInt(locationBytes.length); buf.put(locationBytes); } + buf.put((byte) (checkpointOptions.isExactlyOnceMode() ? 1 : 0)); + buf.put((byte) (checkpointOptions.isUnalignedCheckpoint() ? 1 : 0)); buf.flip(); return buf; @@ -265,8 +267,13 @@ public class EventSerializer { buffer.get(bytes); locationRef = new CheckpointStorageLocationReference(bytes); } + final boolean isExactlyOnceMode = buffer.get() == 1; + final boolean isUnalignedCheckpoint = buffer.get() == 1; - return new CheckpointBarrier(id, timestamp, new CheckpointOptions(checkpointType, locationRef)); + return new CheckpointBarrier( + id, + timestamp, + new CheckpointOptions(checkpointType, locationRef, isExactlyOnceMode, isUnalignedCheckpoint)); } // ------------------------------------------------------------------------ @@ -278,7 +285,7 @@ public class EventSerializer { MemorySegment data = MemorySegmentFactory.wrap(serializedEvent.array()); - final Buffer buffer = new NetworkBuffer(data, FreeingBufferRecycler.INSTANCE, false); + final Buffer buffer = new NetworkBuffer(data, FreeingBufferRecycler.INSTANCE, Buffer.DataType.getDataType(event)); buffer.setSize(serializedEvent.remaining()); return buffer; @@ -289,7 +296,7 @@ public class EventSerializer { MemorySegment data = MemorySegmentFactory.wrap(serializedEvent.array()); - return new BufferConsumer(data, FreeingBufferRecycler.INSTANCE, false); + return new BufferConsumer(data, FreeingBufferRecycler.INSTANCE, Buffer.DataType.getDataType(event)); } public static AbstractEvent fromBuffer(Buffer buffer, ClassLoader classLoader) throws IOException { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/RecordDeserializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/RecordDeserializer.java index 10a1b4d3a22eb62bd562a6ef26be33e25f0e757d..4f4d62163276844e23042dfdd4e24f43ad630e95 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/RecordDeserializer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/RecordDeserializer.java @@ -22,6 +22,7 @@ import org.apache.flink.core.io.IOReadableWritable; import org.apache.flink.runtime.io.network.buffer.Buffer; import java.io.IOException; +import java.util.Optional; /** * Interface for turning sequences of memory segments into records. @@ -63,4 +64,12 @@ public interface RecordDeserializer { void clear(); boolean hasUnfinishedData(); + + /** + * Gets the unconsumed buffer which needs to be persisted in unaligned checkpoint scenario. + * + *

Note that the unconsumed buffer might be null if the whole buffer was already consumed + * before and there are no partial length or data remained in the end of buffer. + */ + Optional getUnconsumedBuffer() throws IOException; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/SpillingAdaptiveSpanningRecordDeserializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/SpillingAdaptiveSpanningRecordDeserializer.java index a78cb4dd7088dd44e4fa031b28cf0bd0e83d9f6b..da30812fa0fe98ca92cac0aa4b51c81dc2e970d4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/SpillingAdaptiveSpanningRecordDeserializer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/SpillingAdaptiveSpanningRecordDeserializer.java @@ -22,8 +22,12 @@ import org.apache.flink.core.io.IOReadableWritable; import org.apache.flink.core.memory.DataInputDeserializer; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.core.memory.DataOutputSerializer; import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.core.memory.MemorySegmentFactory; import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler; +import org.apache.flink.runtime.io.network.buffer.NetworkBuffer; import org.apache.flink.util.FileUtils; import org.apache.flink.util.StringUtils; @@ -38,6 +42,7 @@ import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.nio.channels.FileChannel; import java.util.Arrays; +import java.util.Optional; import java.util.Random; /** @@ -88,6 +93,18 @@ public class SpillingAdaptiveSpanningRecordDeserializer getUnconsumedBuffer() throws IOException { + Optional target; + if (nonSpanningWrapper.remaining() > 0) { + target = nonSpanningWrapper.getUnconsumedSegment(); + } else { + target = spanningWrapper.getUnconsumedSegment(); + } + return target.map(memorySegment -> new NetworkBuffer( + memorySegment, FreeingBufferRecycler.INSTANCE, Buffer.DataType.DATA_BUFFER, memorySegment.size())); + } + @Override public DeserializationResult getNextRecord(T target) throws IOException { // always check the non-spanning wrapper first. @@ -194,6 +211,15 @@ public class SpillingAdaptiveSpanningRecordDeserializer getUnconsumedSegment() { + if (remaining() == 0) { + return Optional.empty(); + } + MemorySegment target = MemorySegmentFactory.allocateUnpooledSegment(remaining()); + segment.copyTo(position, target, 0, remaining()); + return Optional.of(target); + } + // ------------------------------------------------------------------------------------------------------------- // DataInput specific methods // ------------------------------------------------------------------------------------------------------------- @@ -557,6 +583,38 @@ public class SpillingAdaptiveSpanningRecordDeserializer getUnconsumedSegment() throws IOException { + // for the case of only partial length, no data + final int position = lengthBuffer.position(); + if (position > 0) { + MemorySegment segment = MemorySegmentFactory.allocateUnpooledSegment(position); + lengthBuffer.position(0); + segment.put(0, lengthBuffer, position); + return Optional.of(segment); + } + + // for the case of full length, partial data in buffer + if (recordLength > THRESHOLD_FOR_SPILLING) { + throw new UnsupportedOperationException("Unaligned checkpoint currently do not support spilled " + + "records."); + } else if (recordLength != -1) { + int leftOverSize = leftOverLimit - leftOverStart; + int unconsumedSize = Integer.BYTES + accumulatedRecordBytes + leftOverSize; + DataOutputSerializer serializer = new DataOutputSerializer(unconsumedSize); + serializer.writeInt(recordLength); + serializer.write(buffer, 0, accumulatedRecordBytes); + if (leftOverData != null) { + serializer.write(leftOverData, leftOverStart, leftOverSize); + } + MemorySegment segment = MemorySegmentFactory.allocateUnpooledSegment(unconsumedSize); + segment.put(0, serializer.getSharedBuffer(), 0, segment.size()); + return Optional.of(segment); + } + + // for the case of no remaining partial length or data + return Optional.empty(); + } + private void moveRemainderToNonSpanningDeserializer(NonSpanningWrapper deserializer) { deserializer.clear(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/BroadcastRecordWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/BroadcastRecordWriter.java index b4999835d7fc56c24e23ce8f3e7830663f1446e0..93bc2c4a2fb5d563db037f604142c13c1a283fa7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/BroadcastRecordWriter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/BroadcastRecordWriter.java @@ -84,7 +84,7 @@ public final class BroadcastRecordWriter extends R if (bufferBuilder != null) { for (int index = 0; index < numberOfChannels; index++) { if (index != targetChannelIndex) { - targetPartition.addBufferConsumer(bufferBuilder.createBufferConsumer(), index); + addBufferConsumer(bufferBuilder.createBufferConsumer(), index); } } } @@ -128,13 +128,13 @@ public final class BroadcastRecordWriter extends R public BufferBuilder requestNewBufferBuilder(int targetChannel) throws IOException, InterruptedException { checkState(bufferBuilder == null || bufferBuilder.isFinished()); - BufferBuilder builder = targetPartition.getBufferBuilder(); + BufferBuilder builder = getBufferBuilder(); if (randomTriggered) { - targetPartition.addBufferConsumer(builder.createBufferConsumer(), targetChannel); + addBufferConsumer(builder.createBufferConsumer(), targetChannel); } else { try (BufferConsumer bufferConsumer = builder.createBufferConsumer()) { for (int channel = 0; channel < numberOfChannels; channel++) { - targetPartition.addBufferConsumer(bufferConsumer.copy(), channel); + addBufferConsumer(bufferConsumer.copy(), channel); } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ChannelSelectorRecordWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ChannelSelectorRecordWriter.java index 5f5e5964b9889ea22463c2f217eea4e4202dbbd8..8393409660b91a727db61bf8975df2e33c89dddd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ChannelSelectorRecordWriter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ChannelSelectorRecordWriter.java @@ -100,8 +100,8 @@ public final class ChannelSelectorRecordWriter ext public BufferBuilder requestNewBufferBuilder(int targetChannel) throws IOException, InterruptedException { checkState(bufferBuilders[targetChannel] == null || bufferBuilders[targetChannel].isFinished()); - BufferBuilder bufferBuilder = targetPartition.getBufferBuilder(); - targetPartition.addBufferConsumer(bufferBuilder.createBufferConsumer(), targetChannel); + BufferBuilder bufferBuilder = getBufferBuilder(); + addBufferConsumer(bufferBuilder.createBufferConsumer(), targetChannel); bufferBuilders[targetChannel] = bufferBuilder; return bufferBuilder; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java index 6c680f5981c950d361030608ed2c9eaf640dc326..a678e5a48cb340f7f87d4a1f568f7dde6b207501 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java @@ -21,6 +21,8 @@ package org.apache.flink.runtime.io.network.api.writer; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.core.io.IOReadableWritable; import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Meter; +import org.apache.flink.metrics.MeterView; import org.apache.flink.metrics.SimpleCounter; import org.apache.flink.runtime.event.AbstractEvent; import org.apache.flink.runtime.io.AvailabilityProvider; @@ -66,7 +68,7 @@ public abstract class RecordWriter implements Avai private static final Logger LOG = LoggerFactory.getLogger(RecordWriter.class); - protected final ResultPartitionWriter targetPartition; + private final ResultPartitionWriter targetPartition; protected final int numberOfChannels; @@ -78,6 +80,8 @@ public abstract class RecordWriter implements Avai private Counter numBuffersOut = new SimpleCounter(); + protected Meter idleTimeMsPerSecond = new MeterView(new SimpleCounter()); + private final boolean flushAlways; /** The thread that periodically flushes the output, to give an upper latency bound. */ @@ -154,12 +158,16 @@ public abstract class RecordWriter implements Avai } public void broadcastEvent(AbstractEvent event) throws IOException { + broadcastEvent(event, false); + } + + public void broadcastEvent(AbstractEvent event, boolean isPriorityEvent) throws IOException { try (BufferConsumer eventBufferConsumer = EventSerializer.toBufferConsumer(event)) { for (int targetChannel = 0; targetChannel < numberOfChannels; targetChannel++) { tryFinishCurrentBufferBuilder(targetChannel); // Retain the buffer so that it can be recycled by each channel of targetPartition - targetPartition.addBufferConsumer(eventBufferConsumer.copy(), targetChannel); + targetPartition.addBufferConsumer(eventBufferConsumer.copy(), targetChannel, isPriorityEvent); } if (flushAlways) { @@ -182,6 +190,7 @@ public abstract class RecordWriter implements Avai public void setMetricGroup(TaskIOMetricGroup metrics) { numBytesOut = metrics.getNumBytesOutCounter(); numBuffersOut = metrics.getNumBuffersOutCounter(); + idleTimeMsPerSecond = metrics.getIdleTimeMsPerSecond(); } protected void finishBufferBuilder(BufferBuilder bufferBuilder) { @@ -276,6 +285,26 @@ public abstract class RecordWriter implements Avai } } + protected void addBufferConsumer(BufferConsumer consumer, int targetChannel) throws IOException { + targetPartition.addBufferConsumer(consumer, targetChannel); + } + + @VisibleForTesting + public BufferBuilder getBufferBuilder() throws IOException, InterruptedException { + BufferBuilder builder = targetPartition.tryGetBufferBuilder(); + if (builder == null) { + long start = System.currentTimeMillis(); + builder = targetPartition.getBufferBuilder(); + idleTimeMsPerSecond.markEvent(System.currentTimeMillis() - start); + } + return builder; + } + + @VisibleForTesting + public Meter getIdleTimeMsPerSecond() { + return idleTimeMsPerSecond; + } + // ------------------------------------------------------------------------ /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ResultPartitionWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ResultPartitionWriter.java index 75cd5fb9d3605ca14cb4f5bae8d6ea288a823963..efb35dcc6fdece5ef224656caaa00b7939ddf84b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ResultPartitionWriter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ResultPartitionWriter.java @@ -18,10 +18,12 @@ package org.apache.flink.runtime.io.network.api.writer; +import org.apache.flink.runtime.checkpoint.channel.ChannelStateReader; import org.apache.flink.runtime.io.AvailabilityProvider; import org.apache.flink.runtime.io.network.buffer.BufferBuilder; import org.apache.flink.runtime.io.network.buffer.BufferConsumer; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.io.network.partition.ResultSubpartition; import javax.annotation.Nullable; @@ -42,6 +44,12 @@ public interface ResultPartitionWriter extends AutoCloseable, AvailabilityProvid */ void setup() throws IOException; + /** + * Loads the previous output states with the given reader for unaligned checkpoint. + * It should be done before task processing the inputs. + */ + void initializeState(ChannelStateReader stateReader) throws IOException, InterruptedException; + ResultPartitionID getPartitionId(); int getNumberOfSubpartitions(); @@ -53,6 +61,14 @@ public interface ResultPartitionWriter extends AutoCloseable, AvailabilityProvid */ BufferBuilder getBufferBuilder() throws IOException, InterruptedException; + + /** + * Try to request a {@link BufferBuilder} from this partition for writing data. + * + *

Returns null if no buffer is available or the buffer provider has been destroyed. + */ + BufferBuilder tryGetBufferBuilder() throws IOException; + /** * Adds the bufferConsumer to the subpartition with the given index. * @@ -64,7 +80,27 @@ public interface ResultPartitionWriter extends AutoCloseable, AvailabilityProvid * * @return true if operation succeeded and bufferConsumer was enqueued for consumption. */ - boolean addBufferConsumer(BufferConsumer bufferConsumer, int subpartitionIndex) throws IOException; + boolean addBufferConsumer(BufferConsumer bufferConsumer, int subpartitionIndex, boolean isPriorityEvent) throws IOException; + + /** + * Adds the bufferConsumer to the subpartition with the given index. + * + *

This method takes the ownership of the passed {@code bufferConsumer} and thus is responsible for releasing + * it's resources. + * + *

To avoid problems with data re-ordering, before adding new {@link BufferConsumer} the previously added one + * the given {@code subpartitionIndex} must be marked as {@link BufferConsumer#isFinished()}. + * + * @return true if operation succeeded and bufferConsumer was enqueued for consumption. + */ + default boolean addBufferConsumer(BufferConsumer bufferConsumer, int subpartitionIndex) throws IOException { + return addBufferConsumer(bufferConsumer, subpartitionIndex, false); + } + + /** + * Returns the subpartition with the given index. + */ + ResultSubpartition getSubpartition(int subpartitionIndex); /** * Manually trigger consumption from enqueued {@link BufferConsumer BufferConsumers} in all subpartitions. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/Buffer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/Buffer.java index fd6fdb903ab6c81e382cd1d2529b2946c0a83078..d9f423ff613c7230683e7dbff9e7b777b062acb3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/Buffer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/Buffer.java @@ -19,6 +19,8 @@ package org.apache.flink.runtime.io.network.buffer; import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.runtime.event.AbstractEvent; +import org.apache.flink.runtime.io.network.api.CheckpointBarrier; import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf; import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBufAllocator; @@ -54,11 +56,6 @@ public interface Buffer { */ boolean isBuffer(); - /** - * Tags this buffer to represent an event. - */ - void tagAsEvent(); - /** * Returns the underlying memory segment. This method is dangerous since it ignores read only protections and omits * slices. Use it only along the {@link #getMemorySegmentOffset()}. @@ -227,4 +224,61 @@ public interface Buffer { * Tags the buffer as compressed or uncompressed. */ void setCompressed(boolean isCompressed); + + /** + * Gets the type of data this buffer represents. + */ + DataType getDataType(); + + /** + * Sets the type of data this buffer represents. + */ + void setDataType(DataType dataType); + + /** + * Used to identify the type of data contained in the {@link Buffer} so that we can get + * the information without deserializing the serialized data. + */ + enum DataType { + /** + * DATA_BUFFER indicates that this buffer represents a non-event data buffer. + */ + DATA_BUFFER(true), + + /** + * EVENT_BUFFER indicates that this buffer represents serialized data of an event. + * Note that this type can be further divided into more fine-grained event types + * like {@link #ALIGNED_EXACTLY_ONCE_CHECKPOINT_BARRIER} and etc. + */ + EVENT_BUFFER(false), + + /** + * ALIGNED_EXACTLY_ONCE_CHECKPOINT_BARRIER indicates that this buffer represents a + * serialized checkpoint barrier of aligned exactly-once checkpoint mode. + */ + ALIGNED_EXACTLY_ONCE_CHECKPOINT_BARRIER(false); + + private final boolean isBuffer; + + DataType(boolean isBuffer) { + this.isBuffer = isBuffer; + } + + public boolean isBuffer() { + return isBuffer; + } + + public static boolean isAlignedExactlyOnceCheckpointBarrier(Buffer buffer) { + return buffer.getDataType() == ALIGNED_EXACTLY_ONCE_CHECKPOINT_BARRIER; + } + + public static DataType getDataType(AbstractEvent event) { + if (event instanceof CheckpointBarrier && + ((CheckpointBarrier) event).getCheckpointOptions().isExactlyOnceMode() && + !((CheckpointBarrier) event).getCheckpointOptions().isUnalignedCheckpoint()) { + return ALIGNED_EXACTLY_ONCE_CHECKPOINT_BARRIER; + } + return EVENT_BUFFER; + } + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferBuilder.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferBuilder.java index bcd42d23b1ba9c89e088dbd313ce37cedcec4b01..77d3526dbb6db49eecd2350e0870ccbe48be9766 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferBuilder.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferBuilder.java @@ -117,6 +117,11 @@ public class BufferBuilder { return positionMarker.getCached() == getMaxCapacity(); } + public int getWritableBytes() { + checkState(positionMarker.getCached() <= getMaxCapacity()); + return getMaxCapacity() - positionMarker.getCached(); + } + public int getMaxCapacity() { return memorySegment.size(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferConsumer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferConsumer.java index ecebc6734495f420bd8d44b18ef232eee5687951..863b231201ba520646527760036d4cc3e5e0b896 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferConsumer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferConsumer.java @@ -53,7 +53,7 @@ public class BufferConsumer implements Closeable { PositionMarker currentWriterPosition, int currentReaderPosition) { this( - new NetworkBuffer(checkNotNull(memorySegment), checkNotNull(recycler), true), + new NetworkBuffer(checkNotNull(memorySegment), checkNotNull(recycler)), currentWriterPosition, currentReaderPosition); } @@ -61,15 +61,15 @@ public class BufferConsumer implements Closeable { /** * Constructs {@link BufferConsumer} instance with static content. */ - public BufferConsumer(MemorySegment memorySegment, BufferRecycler recycler, boolean isBuffer) { - this(memorySegment, recycler, memorySegment.size(), isBuffer); + public BufferConsumer(MemorySegment memorySegment, BufferRecycler recycler, Buffer.DataType dataType) { + this(memorySegment, recycler, memorySegment.size(), dataType); } /** * Constructs {@link BufferConsumer} instance with static content of a certain size. */ - public BufferConsumer(MemorySegment memorySegment, BufferRecycler recycler, int size, boolean isBuffer) { - this(new NetworkBuffer(checkNotNull(memorySegment), checkNotNull(recycler), isBuffer), + public BufferConsumer(MemorySegment memorySegment, BufferRecycler recycler, int size, Buffer.DataType dataType) { + this(new NetworkBuffer(checkNotNull(memorySegment), checkNotNull(recycler), dataType), () -> -size, 0); checkState(memorySegment.size() > 0); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferProvider.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferProvider.java index 5e17b3087fab97f7935144be2d246bbfeed7a265..05518044a3e9a0d92acb0fa6efb9875e3fea1864 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferProvider.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferProvider.java @@ -37,6 +37,13 @@ public interface BufferProvider extends AvailabilityProvider { */ Buffer requestBuffer() throws IOException; + /** + * Returns a {@link BufferBuilder} instance from the buffer provider. + * + *

Returns null if no buffer is available or the buffer provider has been destroyed. + */ + BufferBuilder requestBufferBuilder() throws IOException; + /** * Returns a {@link BufferBuilder} instance from the buffer provider. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferReceivedListener.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferReceivedListener.java new file mode 100644 index 0000000000000000000000000000000000000000..efcf0306c709e48cef8cbdc219e5a1fd6b1dd061 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferReceivedListener.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.io.network.buffer; + +import org.apache.flink.runtime.checkpoint.channel.InputChannelInfo; +import org.apache.flink.runtime.io.network.api.CheckpointBarrier; + +import java.io.IOException; + +/** + * Interface of listener for notifying the received data buffer or checkpoint barrier + * from network channel. + */ +public interface BufferReceivedListener { + + /** + * Called whenever an {@link org.apache.flink.runtime.io.network.partition.consumer.InputChannel} receives a + * non-barrier buffer. + * + *

The listener is responsible for copying the buffer if it needs to outlive the invocation. It is guaranteed + * that no parallel processing of the buffer happens until this callback returns. + */ + void notifyBufferReceived(Buffer buffer, InputChannelInfo channelInfo) throws IOException; + + /** + * Invoked when an {@link org.apache.flink.runtime.io.network.partition.consumer.InputChannel} receives a + * buffer with a {@link CheckpointBarrier}. It is guaranteed that no parallel processing of the buffer happens + * until this callback returns. + */ + void notifyBarrierReceived(CheckpointBarrier barrier, InputChannelInfo channelInfo) throws IOException; +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java index d7aef4c30c3231e89d98cdd65ab0d47ab9e7c323..a6af47176b0fb03a8ea86ac2c9308d0804c13477 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java @@ -204,6 +204,11 @@ class LocalBufferPool implements BufferPool { return toBuffer(requestMemorySegment()); } + @Override + public BufferBuilder requestBufferBuilder() throws IOException { + return toBufferBuilder(requestMemorySegment()); + } + @Override public BufferBuilder requestBufferBuilderBlocking() throws IOException, InterruptedException { return toBufferBuilder(requestMemorySegmentBlocking()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBuffer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBuffer.java index 93ad6634fcdd04425654938c5ded9faa67991d54..02f98920f909348cdca8675ceaf7913c7e0bdd24 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBuffer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBuffer.java @@ -52,8 +52,8 @@ public class NetworkBuffer extends AbstractReferenceCountedByteBuf implements Bu /** The recycler for the backing {@link MemorySegment}. */ private final BufferRecycler recycler; - /** Whether this buffer represents a buffer or an event. */ - private boolean isBuffer; + /** The {@link DataType} this buffer represents. */ + private DataType dataType; /** Allocator for further byte buffers (needed by netty). */ private ByteBufAllocator allocator; @@ -77,7 +77,7 @@ public class NetworkBuffer extends AbstractReferenceCountedByteBuf implements Bu * will be called to recycle this buffer once the reference count is 0 */ public NetworkBuffer(MemorySegment memorySegment, BufferRecycler recycler) { - this(memorySegment, recycler, true); + this(memorySegment, recycler, DataType.DATA_BUFFER); } /** @@ -88,11 +88,11 @@ public class NetworkBuffer extends AbstractReferenceCountedByteBuf implements Bu * backing memory segment (defines {@link #maxCapacity}) * @param recycler * will be called to recycle this buffer once the reference count is 0 - * @param isBuffer - * whether this buffer represents a buffer (true) or an event (false) + * @param dataType + * the {@link DataType} this buffer represents */ - public NetworkBuffer(MemorySegment memorySegment, BufferRecycler recycler, boolean isBuffer) { - this(memorySegment, recycler, isBuffer, 0); + public NetworkBuffer(MemorySegment memorySegment, BufferRecycler recycler, DataType dataType) { + this(memorySegment, recycler, dataType, 0); } /** @@ -103,13 +103,13 @@ public class NetworkBuffer extends AbstractReferenceCountedByteBuf implements Bu * backing memory segment (defines {@link #maxCapacity}) * @param recycler * will be called to recycle this buffer once the reference count is 0 - * @param isBuffer - * whether this buffer represents a buffer (true) or an event (false) + * @param dataType + * the {@link DataType} this buffer represents * @param size * current size of data in the buffer, i.e. the writer index to set */ - public NetworkBuffer(MemorySegment memorySegment, BufferRecycler recycler, boolean isBuffer, int size) { - this(memorySegment, recycler, isBuffer, false, size); + public NetworkBuffer(MemorySegment memorySegment, BufferRecycler recycler, DataType dataType, int size) { + this(memorySegment, recycler, dataType, false, size); } /** @@ -120,18 +120,18 @@ public class NetworkBuffer extends AbstractReferenceCountedByteBuf implements Bu * backing memory segment (defines {@link #maxCapacity}) * @param recycler * will be called to recycle this buffer once the reference count is 0 - * @param isBuffer - * whether this buffer represents a buffer (true) or an event (false) + * @param dataType + * the {@link DataType} this buffer represents * @param size * current size of data in the buffer, i.e. the writer index to set * @param isCompressed * whether the buffer is compressed or not */ - public NetworkBuffer(MemorySegment memorySegment, BufferRecycler recycler, boolean isBuffer, boolean isCompressed, int size) { + public NetworkBuffer(MemorySegment memorySegment, BufferRecycler recycler, DataType dataType, boolean isCompressed, int size) { super(memorySegment.size()); this.memorySegment = checkNotNull(memorySegment); this.recycler = checkNotNull(recycler); - this.isBuffer = isBuffer; + this.dataType = dataType; this.isCompressed = isCompressed; this.currentSize = memorySegment.size(); setSize(size); @@ -139,14 +139,7 @@ public class NetworkBuffer extends AbstractReferenceCountedByteBuf implements Bu @Override public boolean isBuffer() { - return isBuffer; - } - - @Override - public void tagAsEvent() { - ensureAccessible(); - - isBuffer = false; + return dataType.isBuffer(); } @Override @@ -639,4 +632,16 @@ public class NetworkBuffer extends AbstractReferenceCountedByteBuf implements Bu public void setCompressed(boolean isCompressed) { this.isCompressed = isCompressed; } + + @Override + public DataType getDataType() { + return dataType; + } + + @Override + public void setDataType(DataType dataType) { + ensureAccessible(); + + this.dataType = dataType; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/ReadOnlySlicedNetworkBuffer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/ReadOnlySlicedNetworkBuffer.java index 05b21b4b8e6ecc25026f6c7377604b0d59e3ff77..b0546d990bb16478e9e23d335f0664a2ffc304e1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/ReadOnlySlicedNetworkBuffer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/ReadOnlySlicedNetworkBuffer.java @@ -89,11 +89,6 @@ public final class ReadOnlySlicedNetworkBuffer extends ReadOnlyByteBuf implement return getBuffer().isBuffer(); } - @Override - public void tagAsEvent() { - throw new ReadOnlyBufferException(); - } - /** * Returns the underlying memory segment. * @@ -223,6 +218,16 @@ public final class ReadOnlySlicedNetworkBuffer extends ReadOnlyByteBuf implement this.isCompressed = isCompressed; } + @Override + public DataType getDataType() { + return getBuffer().getDataType(); + } + + @Override + public void setDataType(DataType dataType) { + throw new ReadOnlyBufferException(); + } + private Buffer getBuffer() { return ((Buffer) unwrap().unwrap()); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/BufferResponseDecoder.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/BufferResponseDecoder.java new file mode 100644 index 0000000000000000000000000000000000000000..508f469e9643d768e8f85eb7b3f0d506f91c7a41 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/BufferResponseDecoder.java @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.io.network.netty; + +import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf; +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext; + +import javax.annotation.Nullable; + +import static org.apache.flink.runtime.io.network.netty.NettyMessage.BufferResponse; +import static org.apache.flink.runtime.io.network.netty.NettyMessage.BufferResponse.MESSAGE_HEADER_LENGTH; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * The decoder for {@link BufferResponse}. + */ +class BufferResponseDecoder extends NettyMessageDecoder { + + /** The Buffer allocator. */ + private final NetworkBufferAllocator allocator; + + /** The accumulation buffer of message header. */ + private ByteBuf messageHeaderBuffer; + + /** + * The BufferResponse message that has its message header decoded, but still + * not received all the bytes of the buffer part. + */ + @Nullable + private BufferResponse bufferResponse; + + /** How many bytes have been received or discarded for the data buffer part. */ + private int decodedDataBufferSize; + + BufferResponseDecoder(NetworkBufferAllocator allocator) { + this.allocator = checkNotNull(allocator); + } + + @Override + public void onChannelActive(ChannelHandlerContext ctx) { + messageHeaderBuffer = ctx.alloc().directBuffer(MESSAGE_HEADER_LENGTH); + } + + @Override + public DecodingResult onChannelRead(ByteBuf data) throws Exception { + if (bufferResponse == null) { + decodeMessageHeader(data); + } + + if (bufferResponse != null) { + int remainingBufferSize = bufferResponse.bufferSize - decodedDataBufferSize; + int actualBytesToDecode = Math.min(data.readableBytes(), remainingBufferSize); + + // For the case of data buffer really exists in BufferResponse now. + if (actualBytesToDecode > 0) { + // For the case of released input channel, the respective data buffer part would be + // discarded from the received buffer. + if (bufferResponse.getBuffer() == null) { + data.readerIndex(data.readerIndex() + actualBytesToDecode); + } else { + bufferResponse.getBuffer().asByteBuf().writeBytes(data, actualBytesToDecode); + } + + decodedDataBufferSize += actualBytesToDecode; + } + + if (decodedDataBufferSize == bufferResponse.bufferSize) { + BufferResponse result = bufferResponse; + clearState(); + return DecodingResult.fullMessage(result); + } + } + + return DecodingResult.NOT_FINISHED; + } + + private void decodeMessageHeader(ByteBuf data) { + ByteBuf fullFrameHeaderBuf = ByteBufUtils.accumulate( + messageHeaderBuffer, + data, + MESSAGE_HEADER_LENGTH, + messageHeaderBuffer.readableBytes()); + if (fullFrameHeaderBuf != null) { + bufferResponse = BufferResponse.readFrom(fullFrameHeaderBuf, allocator); + } + } + + private void clearState() { + bufferResponse = null; + decodedDataBufferSize = 0; + + messageHeaderBuffer.clear(); + } + + @Override + public void close() { + if (bufferResponse != null) { + bufferResponse.releaseBuffer(); + } + + messageHeaderBuffer.release(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/ByteBufUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/ByteBufUtils.java new file mode 100644 index 0000000000000000000000000000000000000000..0f10e1e7b4cd4108c0888d94d58806f3e7e82675 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/ByteBufUtils.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.io.network.netty; + +import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf; + +import javax.annotation.Nullable; + +/** + * Utility routines to process {@link ByteBuf}. + */ +public class ByteBufUtils { + + /** + * Accumulates data from source to target. If no data has been + * accumulated yet and source has enough data, source will be + * returned directly. Otherwise, data will be copied into target. If the + * size of data copied after this operation has reached targetAccumulationSize, + * target will be returned, otherwise null will be returned to indicate + * more data is required. + * + * @param target The target buffer. + * @param source The source buffer. + * @param targetAccumulationSize The target size of data to accumulate. + * @param accumulatedSize The size of data accumulated so far. + * + * @return The ByteBuf containing accumulated data. If not enough data has been accumulated, + * null will be returned. + */ + @Nullable + public static ByteBuf accumulate(ByteBuf target, ByteBuf source, int targetAccumulationSize, int accumulatedSize) { + if (accumulatedSize == 0 && source.readableBytes() >= targetAccumulationSize) { + return source; + } + + int copyLength = Math.min(source.readableBytes(), targetAccumulationSize - accumulatedSize); + if (copyLength > 0) { + target.writeBytes(source, copyLength); + } + + if (accumulatedSize + copyLength == targetAccumulationSize) { + return target; + } + + return null; + } + +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedPartitionRequestClientHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedPartitionRequestClientHandler.java index e261ff78530596f016c458def45229f833455917..4f3d872a692b3021f278c20d3bb2b31b86859e47 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedPartitionRequestClientHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedPartitionRequestClientHandler.java @@ -18,21 +18,16 @@ package org.apache.flink.runtime.io.network.netty; -import org.apache.flink.core.memory.MemorySegment; -import org.apache.flink.core.memory.MemorySegmentFactory; import org.apache.flink.runtime.io.network.NetworkClientHandler; -import org.apache.flink.runtime.io.network.buffer.Buffer; -import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler; -import org.apache.flink.runtime.io.network.buffer.NetworkBuffer; import org.apache.flink.runtime.io.network.netty.exception.LocalTransportException; import org.apache.flink.runtime.io.network.netty.exception.RemoteTransportException; import org.apache.flink.runtime.io.network.netty.exception.TransportException; import org.apache.flink.runtime.io.network.netty.NettyMessage.AddCredit; +import org.apache.flink.runtime.io.network.netty.NettyMessage.ResumeConsumption; import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException; import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID; import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel; -import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf; import org.apache.flink.shaded.netty4.io.netty.channel.Channel; import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFuture; import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFutureListener; @@ -49,6 +44,8 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicReference; +import static org.apache.flink.util.Preconditions.checkNotNull; + /** * Channel handler to read the messages of buffer response or error response from the * producer, to write and flush the unannounced credits for the producer. @@ -62,8 +59,8 @@ class CreditBasedPartitionRequestClientHandler extends ChannelInboundHandlerAdap /** Channels, which already requested partitions from the producers. */ private final ConcurrentMap inputChannels = new ConcurrentHashMap<>(); - /** Channels, which will notify the producers about unannounced credit. */ - private final ArrayDeque inputChannelsWithCredit = new ArrayDeque<>(); + /** Messages to be sent to the producers (credit announcement or resume consumption request). */ + private final ArrayDeque clientOutboundMessages = new ArrayDeque<>(); private final AtomicReference channelError = new AtomicReference<>(); @@ -97,6 +94,11 @@ class CreditBasedPartitionRequestClientHandler extends ChannelInboundHandlerAdap inputChannels.remove(listener.getInputChannelId()); } + @Override + public RemoteInputChannel getInputChannel(InputChannelID inputChannelId) { + return inputChannels.get(inputChannelId); + } + @Override public void cancelRequestFor(InputChannelID inputChannelId) { if (inputChannelId == null || ctx == null) { @@ -110,7 +112,12 @@ class CreditBasedPartitionRequestClientHandler extends ChannelInboundHandlerAdap @Override public void notifyCreditAvailable(final RemoteInputChannel inputChannel) { - ctx.executor().execute(() -> ctx.pipeline().fireUserEventTriggered(inputChannel)); + ctx.executor().execute(() -> ctx.pipeline().fireUserEventTriggered(new AddCreditMessage(inputChannel))); + } + + @Override + public void resumeConsumption(RemoteInputChannel inputChannel) { + ctx.executor().execute(() -> ctx.pipeline().fireUserEventTriggered(new ResumeConsumptionMessage(inputChannel))); } // ------------------------------------------------------------------------ @@ -186,10 +193,10 @@ class CreditBasedPartitionRequestClientHandler extends ChannelInboundHandlerAdap */ @Override public void userEventTriggered(ChannelHandlerContext ctx, Object msg) throws Exception { - if (msg instanceof RemoteInputChannel) { - boolean triggerWrite = inputChannelsWithCredit.isEmpty(); + if (msg instanceof ClientOutboundMessage) { + boolean triggerWrite = clientOutboundMessages.isEmpty(); - inputChannelsWithCredit.add((RemoteInputChannel) msg); + clientOutboundMessages.add((ClientOutboundMessage) msg); if (triggerWrite) { writeAndFlushNextMessageIfPossible(ctx.channel()); @@ -215,7 +222,7 @@ class CreditBasedPartitionRequestClientHandler extends ChannelInboundHandlerAdap LOG.warn("An Exception was thrown during error notification of a remote input channel.", t); } finally { inputChannels.clear(); - inputChannelsWithCredit.clear(); + clientOutboundMessages.clear(); if (ctx != null) { ctx.close(); @@ -249,7 +256,7 @@ class CreditBasedPartitionRequestClientHandler extends ChannelInboundHandlerAdap NettyMessage.BufferResponse bufferOrEvent = (NettyMessage.BufferResponse) msg; RemoteInputChannel inputChannel = inputChannels.get(bufferOrEvent.receiverId); - if (inputChannel == null) { + if (inputChannel == null || inputChannel.isReleased()) { bufferOrEvent.releaseBuffer(); cancelRequestFor(bufferOrEvent.receiverId); @@ -290,43 +297,12 @@ class CreditBasedPartitionRequestClientHandler extends ChannelInboundHandlerAdap } private void decodeBufferOrEvent(RemoteInputChannel inputChannel, NettyMessage.BufferResponse bufferOrEvent) throws Throwable { - try { - ByteBuf nettyBuffer = bufferOrEvent.getNettyBuffer(); - final int receivedSize = nettyBuffer.readableBytes(); - if (bufferOrEvent.isBuffer()) { - // ---- Buffer ------------------------------------------------ - - // Early return for empty buffers. Otherwise Netty's readBytes() throws an - // IndexOutOfBoundsException. - if (receivedSize == 0) { - inputChannel.onEmptyBuffer(bufferOrEvent.sequenceNumber, bufferOrEvent.backlog); - return; - } - - Buffer buffer = inputChannel.requestBuffer(); - if (buffer != null) { - nettyBuffer.readBytes(buffer.asByteBuf(), receivedSize); - buffer.setCompressed(bufferOrEvent.isCompressed); - - inputChannel.onBuffer(buffer, bufferOrEvent.sequenceNumber, bufferOrEvent.backlog); - } else if (inputChannel.isReleased()) { - cancelRequestFor(bufferOrEvent.receiverId); - } else { - throw new IllegalStateException("No buffer available in credit-based input channel."); - } - } else { - // ---- Event ------------------------------------------------- - // TODO We can just keep the serialized data in the Netty buffer and release it later at the reader - byte[] byteArray = new byte[receivedSize]; - nettyBuffer.readBytes(byteArray); - - MemorySegment memSeg = MemorySegmentFactory.wrap(byteArray); - Buffer buffer = new NetworkBuffer(memSeg, FreeingBufferRecycler.INSTANCE, false, receivedSize); - - inputChannel.onBuffer(buffer, bufferOrEvent.sequenceNumber, bufferOrEvent.backlog); - } - } finally { - bufferOrEvent.releaseBuffer(); + if (bufferOrEvent.isBuffer() && bufferOrEvent.bufferSize == 0) { + inputChannel.onEmptyBuffer(bufferOrEvent.sequenceNumber, bufferOrEvent.backlog); + } else if (bufferOrEvent.getBuffer() != null) { + inputChannel.onBuffer(bufferOrEvent.getBuffer(), bufferOrEvent.sequenceNumber, bufferOrEvent.backlog); + } else { + throw new IllegalStateException("The read buffer is null in credit-based input channel."); } } @@ -342,19 +318,17 @@ class CreditBasedPartitionRequestClientHandler extends ChannelInboundHandlerAdap } while (true) { - RemoteInputChannel inputChannel = inputChannelsWithCredit.poll(); + ClientOutboundMessage outboundMessage = clientOutboundMessages.poll(); // The input channel may be null because of the write callbacks // that are executed after each write. - if (inputChannel == null) { + if (outboundMessage == null) { return; } - //It is no need to notify credit for the released channel. - if (!inputChannel.isReleased()) { - AddCredit msg = new AddCredit( - inputChannel.getAndResetUnannouncedCredit(), - inputChannel.getInputChannelId()); + //It is no need to notify credit or resume data consumption for the released channel. + if (!outboundMessage.inputChannel.isReleased()) { + Object msg = outboundMessage.buildMessage(); // Write and flush and wait until this is done before // trying to continue with the next input channel. @@ -382,4 +356,38 @@ class CreditBasedPartitionRequestClientHandler extends ChannelInboundHandlerAdap } } } + + private static abstract class ClientOutboundMessage { + protected final RemoteInputChannel inputChannel; + + ClientOutboundMessage(RemoteInputChannel inputChannel) { + this.inputChannel = inputChannel; + } + + abstract Object buildMessage(); + } + + private static class AddCreditMessage extends ClientOutboundMessage { + + AddCreditMessage(RemoteInputChannel inputChannel) { + super(checkNotNull(inputChannel)); + } + + @Override + public Object buildMessage() { + return new AddCredit(inputChannel.getAndResetUnannouncedCredit(), inputChannel.getInputChannelId()); + } + } + + private static class ResumeConsumptionMessage extends ClientOutboundMessage { + + ResumeConsumptionMessage(RemoteInputChannel inputChannel) { + super(checkNotNull(inputChannel)); + } + + @Override + Object buildMessage() { + return new ResumeConsumption(inputChannel.getInputChannelId()); + } + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java index ff7eba9224e58766a70884834ee6c21185fbe38c..a814268eb416fd4c1b32e85dbf3fbceb784431fa 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java @@ -98,6 +98,11 @@ class CreditBasedSequenceNumberingViewReader implements BufferAvailabilityListen numCreditsAvailable += creditDeltas; } + @Override + public void resumeConsumption() { + subpartitionView.resumeConsumption(); + } + @Override public void setRegisteredAsAvailable(boolean isRegisteredAvailable) { this.isRegisteredAsAvailable = isRegisteredAvailable; @@ -115,12 +120,7 @@ class CreditBasedSequenceNumberingViewReader implements BufferAvailabilityListen @Override public boolean isAvailable() { // BEWARE: this must be in sync with #isAvailable(BufferAndBacklog)! - if (numCreditsAvailable > 0) { - return subpartitionView.isAvailable(); - } - else { - return subpartitionView.nextBufferIsEvent(); - } + return subpartitionView.isAvailable(numCreditsAvailable); } /** @@ -136,10 +136,10 @@ class CreditBasedSequenceNumberingViewReader implements BufferAvailabilityListen private boolean isAvailable(BufferAndBacklog bufferAndBacklog) { // BEWARE: this must be in sync with #isAvailable()! if (numCreditsAvailable > 0) { - return bufferAndBacklog.isMoreAvailable(); + return bufferAndBacklog.isDataAvailable(); } else { - return bufferAndBacklog.nextBufferIsEvent(); + return bufferAndBacklog.isEventAvailable(); } } @@ -160,11 +160,11 @@ class CreditBasedSequenceNumberingViewReader implements BufferAvailabilityListen @VisibleForTesting boolean hasBuffersAvailable() { - return subpartitionView.isAvailable(); + return subpartitionView.isAvailable(Integer.MAX_VALUE); } @Override - public BufferAndAvailability getNextBuffer() throws IOException, InterruptedException { + public BufferAndAvailability getNextBuffer() throws IOException { BufferAndBacklog next = subpartitionView.getNextBuffer(); if (next != null) { sequenceNumber++; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyBufferPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyBufferPool.java index 6d2a6c88287dca8a99054d6ade4eb849c99ec3c3..87bf7f6e12842862bcd90e9bb1c2a9512839e047 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyBufferPool.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyBufferPool.java @@ -53,7 +53,7 @@ public class NettyBufferPool extends PooledByteBufAllocator { /** * Arenas allocate chunks of pageSize << maxOrder bytes. With these defaults, this results in - * chunks of 16 MB. + * chunks of 4 MB. * * @see #MAX_ORDER */ @@ -61,11 +61,13 @@ public class NettyBufferPool extends PooledByteBufAllocator { /** * Arenas allocate chunks of pageSize << maxOrder bytes. With these defaults, this results in - * chunks of 16 MB. + * chunks of 4 MB, which is smaller than the previous default (16 MB) to further reduce the + * netty memory overhead. According to the test result, after introducing client side zero-copy + * in FLINK-10742, 4 MB is enough to support large-scale netty shuffle. * * @see #PAGE_SIZE */ - private static final int MAX_ORDER = 11; + private static final int MAX_ORDER = 9; /** * Creates Netty's buffer pool with the specified number of direct arenas. @@ -78,8 +80,8 @@ public class NettyBufferPool extends PooledByteBufAllocator { PREFER_DIRECT, // No heap arenas, please. 0, - // Number of direct arenas. Each arena allocates a chunk of 16 MB, i.e. - // we allocate numDirectArenas * 16 MB of direct memory. This can grow + // Number of direct arenas. Each arena allocates a chunk of 4 MB, i.e. + // we allocate numDirectArenas * 4 MB of direct memory. This can grow // to multiple chunks per arena during runtime, but this should only // happen with a large amount of connections per task manager. We // control the memory allocations with low/high watermarks when writing @@ -92,7 +94,7 @@ public class NettyBufferPool extends PooledByteBufAllocator { this.numberOfArenas = numberOfArenas; // Arenas allocate chunks of pageSize << maxOrder bytes. With these - // defaults, this results in chunks of 16 MB. + // defaults, this results in chunks of 4 MB. this.chunkSize = PAGE_SIZE << MAX_ORDER; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessage.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessage.java index ea41b42ddbfa9e51bde628d505d02c83f92cfee6..5f840d26ce64af130d8246c81ecab5d3eecb6485 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessage.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessage.java @@ -84,7 +84,7 @@ public abstract class NettyMessage { * {@link NettyMessage} subclass ID * * @return a newly allocated direct buffer with header data written for {@link - * NettyMessageDecoder} + * NettyMessageEncoder} */ private static ByteBuf allocateBuffer(ByteBufAllocator allocator, byte id) { return allocateBuffer(allocator, id, -1); @@ -105,7 +105,7 @@ public abstract class NettyMessage { * content length (or -1 if unknown) * * @return a newly allocated direct buffer with header data written for {@link - * NettyMessageDecoder} + * NettyMessageEncoder} */ private static ByteBuf allocateBuffer(ByteBufAllocator allocator, byte id, int contentLength) { return allocateBuffer(allocator, id, 0, contentLength, true); @@ -131,7 +131,7 @@ public abstract class NettyMessage { * only return a buffer with the header information (false) * * @return a newly allocated direct buffer with header data written for {@link - * NettyMessageDecoder} + * NettyMessageEncoder} */ private static ByteBuf allocateBuffer( ByteBufAllocator allocator, @@ -228,18 +228,12 @@ public abstract class NettyMessage { final NettyMessage decodedMsg; switch (msgId) { - case BufferResponse.ID: - decodedMsg = BufferResponse.readFrom(msg); - break; case PartitionRequest.ID: decodedMsg = PartitionRequest.readFrom(msg); break; case TaskEventRequest.ID: decodedMsg = TaskEventRequest.readFrom(msg, getClass().getClassLoader()); break; - case ErrorResponse.ID: - decodedMsg = ErrorResponse.readFrom(msg); - break; case CancelPartitionRequest.ID: decodedMsg = CancelPartitionRequest.readFrom(msg); break; @@ -249,6 +243,9 @@ public abstract class NettyMessage { case AddCredit.ID: decodedMsg = AddCredit.readFrom(msg); break; + case ResumeConsumption.ID: + decodedMsg = ResumeConsumption.readFrom(msg); + break; default: throw new ProtocolException( "Received unknown message from producer: " + msg); @@ -269,9 +266,12 @@ public abstract class NettyMessage { static class BufferResponse extends NettyMessage { - private static final byte ID = 0; + static final byte ID = 0; - final ByteBuf buffer; + // receiver ID (16), sequence number (4), backlog (4), isBuffer (1), isCompressed (1), buffer size (4) + static final int MESSAGE_HEADER_LENGTH = 16 + 4 + 4 + 1 + 1 + 4; + + final Buffer buffer; final InputChannelID receiverId; @@ -283,19 +283,23 @@ public abstract class NettyMessage { final boolean isCompressed; + final int bufferSize; + private BufferResponse( - ByteBuf buffer, + @Nullable Buffer buffer, boolean isBuffer, boolean isCompressed, int sequenceNumber, InputChannelID receiverId, - int backlog) { - this.buffer = checkNotNull(buffer); + int backlog, + int bufferSize) { + this.buffer = buffer; this.isBuffer = isBuffer; this.isCompressed = isCompressed; this.sequenceNumber = sequenceNumber; this.receiverId = checkNotNull(receiverId); this.backlog = backlog; + this.bufferSize = bufferSize; } BufferResponse( @@ -303,24 +307,28 @@ public abstract class NettyMessage { int sequenceNumber, InputChannelID receiverId, int backlog) { - this.buffer = checkNotNull(buffer).asByteBuf(); + this.buffer = checkNotNull(buffer); this.isBuffer = buffer.isBuffer(); this.isCompressed = buffer.isCompressed(); this.sequenceNumber = sequenceNumber; this.receiverId = checkNotNull(receiverId); this.backlog = backlog; + this.bufferSize = buffer.getSize(); } boolean isBuffer() { return isBuffer; } - ByteBuf getNettyBuffer() { + @Nullable + public Buffer getBuffer() { return buffer; } void releaseBuffer() { - buffer.release(); + if (buffer != null) { + buffer.recycleBuffer(); + } } // -------------------------------------------------------------------- @@ -329,18 +337,13 @@ public abstract class NettyMessage { @Override ByteBuf write(ByteBufAllocator allocator) throws IOException { - // receiver ID (16), sequence number (4), backlog (4), isBuffer (1), isCompressed (1), buffer size (4) - final int messageHeaderLength = 16 + 4 + 4 + 1 + 1 + 4; - ByteBuf headerBuf = null; try { - if (buffer instanceof Buffer) { - // in order to forward the buffer to netty, it needs an allocator set - ((Buffer) buffer).setAllocator(allocator); - } + // in order to forward the buffer to netty, it needs an allocator set + buffer.setAllocator(allocator); // only allocate header buffer - we will combine it with the data buffer below - headerBuf = allocateBuffer(allocator, ID, messageHeaderLength, buffer.readableBytes(), false); + headerBuf = allocateBuffer(allocator, ID, MESSAGE_HEADER_LENGTH, bufferSize, false); receiverId.writeTo(headerBuf); headerBuf.writeInt(sequenceNumber); @@ -351,38 +354,67 @@ public abstract class NettyMessage { CompositeByteBuf composityBuf = allocator.compositeDirectBuffer(); composityBuf.addComponent(headerBuf); - composityBuf.addComponent(buffer); + composityBuf.addComponent(buffer.asByteBuf()); // update writer index since we have data written to the components: - composityBuf.writerIndex(headerBuf.writerIndex() + buffer.writerIndex()); + composityBuf.writerIndex(headerBuf.writerIndex() + buffer.asByteBuf().writerIndex()); return composityBuf; } catch (Throwable t) { if (headerBuf != null) { headerBuf.release(); } - buffer.release(); + buffer.recycleBuffer(); ExceptionUtils.rethrowIOException(t); return null; // silence the compiler } } - static BufferResponse readFrom(ByteBuf buffer) { - InputChannelID receiverId = InputChannelID.fromByteBuf(buffer); - int sequenceNumber = buffer.readInt(); - int backlog = buffer.readInt(); - boolean isBuffer = buffer.readBoolean(); - boolean isCompressed = buffer.readBoolean(); - int size = buffer.readInt(); + /** + * Parses the message header part and composes a new BufferResponse with an empty data buffer. The + * data buffer will be filled in later. + * + * @param messageHeader the serialized message header. + * @param bufferAllocator the allocator for network buffer. + * @return a BufferResponse object with the header parsed and the data buffer to fill in later. The + * data buffer will be null if the target channel has been released or the buffer size is 0. + */ + static BufferResponse readFrom(ByteBuf messageHeader, NetworkBufferAllocator bufferAllocator) { + InputChannelID receiverId = InputChannelID.fromByteBuf(messageHeader); + int sequenceNumber = messageHeader.readInt(); + int backlog = messageHeader.readInt(); + boolean isBuffer = messageHeader.readBoolean(); + boolean isCompressed = messageHeader.readBoolean(); + int size = messageHeader.readInt(); + + Buffer dataBuffer = null; + + if (size != 0) { + if (isBuffer) { + dataBuffer = bufferAllocator.allocatePooledNetworkBuffer(receiverId); + } else { + dataBuffer = bufferAllocator.allocateUnPooledNetworkBuffer(size); + } + } - ByteBuf retainedSlice = buffer.readSlice(size).retain(); - return new BufferResponse(retainedSlice, isBuffer, isCompressed, sequenceNumber, receiverId, backlog); + if (dataBuffer != null) { + dataBuffer.setCompressed(isCompressed); + } + + return new BufferResponse( + dataBuffer, + isBuffer, + isCompressed, + sequenceNumber, + receiverId, + backlog, + size); } } static class ErrorResponse extends NettyMessage { - private static final byte ID = 1; + static final byte ID = 1; final Throwable cause; @@ -479,7 +511,7 @@ public abstract class NettyMessage { ByteBuf result = null; try { - result = allocateBuffer(allocator, ID, 16 + 16 + 4 + 16 + 4); + result = allocateBuffer(allocator, ID, 20 + 16 + 4 + 16 + 4); partitionId.getPartitionId().writeTo(result); partitionId.getProducerId().writeTo(result); @@ -540,7 +572,7 @@ public abstract class NettyMessage { // TODO Directly serialize to Netty's buffer ByteBuffer serializedEvent = EventSerializer.toSerializedEvent(event); - result = allocateBuffer(allocator, ID, 4 + serializedEvent.remaining() + 16 + 16 + 16); + result = allocateBuffer(allocator, ID, 4 + serializedEvent.remaining() + 20 + 16 + 16); result.writeInt(serializedEvent.remaining()); result.writeBytes(serializedEvent); @@ -653,7 +685,6 @@ public abstract class NettyMessage { AddCredit(int credit, InputChannelID receiverId) { checkArgument(credit > 0, "The announced credit should be greater than 0"); - this.credit = credit; this.receiverId = receiverId; } @@ -690,4 +721,46 @@ public abstract class NettyMessage { return String.format("AddCredit(%s : %d)", receiverId, credit); } } + + /** + * Message to notify the producer to unblock from checkpoint. + */ + static class ResumeConsumption extends NettyMessage { + + private static final byte ID = 7; + + final InputChannelID receiverId; + + ResumeConsumption(InputChannelID receiverId) { + this.receiverId = receiverId; + } + + @Override + ByteBuf write(ByteBufAllocator allocator) throws IOException { + ByteBuf result = null; + + try { + result = allocateBuffer(allocator, ID, 16); + receiverId.writeTo(result); + + return result; + } + catch (Throwable t) { + if (result != null) { + result.release(); + } + + throw new IOException(t); + } + } + + static ResumeConsumption readFrom(ByteBuf buffer) { + return new ResumeConsumption(InputChannelID.fromByteBuf(buffer)); + } + + @Override + public String toString() { + return String.format("ResumeConsumption(%s)", receiverId); + } + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientDecoderDelegate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientDecoderDelegate.java new file mode 100644 index 0000000000000000000000000000000000000000..d3fe208c87140e59616c3ff65ef3b5db586acbb1 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientDecoderDelegate.java @@ -0,0 +1,153 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.io.network.netty; + +import org.apache.flink.runtime.io.network.NetworkClientHandler; +import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf; +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext; +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter; +import org.apache.flink.util.IOUtils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.flink.runtime.io.network.netty.NettyMessage.FRAME_HEADER_LENGTH; +import static org.apache.flink.runtime.io.network.netty.NettyMessage.MAGIC_NUMBER; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; + +/** + * Decodes messages from the received netty buffers. This decoder assumes the + * messages have the following format: + * +-----------------------------------+--------------------------------+ + * | FRAME_HEADER || MESSAGE_HEADER | DATA BUFFER (Optional) | + * +-----------------------------------+--------------------------------+ + * + *

This decoder decodes the frame header and delegates the following work to the + * corresponding message decoders according to the message type. During this process + * The frame header and message header are only accumulated if they span received + * multiple netty buffers, and the data buffer is copied directly to the buffer + * of corresponding input channel to avoid more copying. + * + *

The format of the frame header is + * +------------------+------------------+--------+ + * | FRAME LENGTH (4) | MAGIC NUMBER (4) | ID (1) | + * +------------------+------------------+--------+ + */ +public class NettyMessageClientDecoderDelegate extends ChannelInboundHandlerAdapter { + private final Logger LOG = LoggerFactory.getLogger(NettyMessageClientDecoderDelegate.class); + + /** The decoder for BufferResponse. */ + private final NettyMessageDecoder bufferResponseDecoder; + + /** The decoder for messages other than BufferResponse. */ + private final NettyMessageDecoder nonBufferResponseDecoder; + + /** The accumulation buffer for the frame header. */ + private ByteBuf frameHeaderBuffer; + + /** The decoder for the current message. It is null if we are decoding the frame header. */ + private NettyMessageDecoder currentDecoder; + + NettyMessageClientDecoderDelegate(NetworkClientHandler networkClientHandler) { + this.bufferResponseDecoder = new BufferResponseDecoder( + new NetworkBufferAllocator( + checkNotNull(networkClientHandler))); + this.nonBufferResponseDecoder = new NonBufferResponseDecoder(); + } + + @Override + public void channelActive(ChannelHandlerContext ctx) throws Exception { + bufferResponseDecoder.onChannelActive(ctx); + nonBufferResponseDecoder.onChannelActive(ctx); + + frameHeaderBuffer = ctx.alloc().directBuffer(FRAME_HEADER_LENGTH); + + super.channelActive(ctx); + } + + /** + * Releases resources when the channel is closed. When exceptions are thrown during + * processing received netty buffers, {@link CreditBasedPartitionRequestClientHandler} + * is expected to catch the exception and close the channel and trigger this notification. + * + * @param ctx The context of the channel close notification. + */ + @Override + public void channelInactive(ChannelHandlerContext ctx) throws Exception { + IOUtils.cleanup(LOG, bufferResponseDecoder, nonBufferResponseDecoder); + frameHeaderBuffer.release(); + + super.channelInactive(ctx); + } + + @Override + public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { + if (!(msg instanceof ByteBuf)) { + ctx.fireChannelRead(msg); + return; + } + + ByteBuf data = (ByteBuf) msg; + try { + while (data.isReadable()) { + if (currentDecoder != null) { + NettyMessageDecoder.DecodingResult result = currentDecoder.onChannelRead(data); + if (!result.isFinished()) { + break; + } + ctx.fireChannelRead(result.getMessage()); + + currentDecoder = null; + frameHeaderBuffer.clear(); + } + + decodeFrameHeader(data); + } + checkState(!data.isReadable(), "Not all data of the received buffer consumed."); + } finally { + data.release(); + } + } + + private void decodeFrameHeader(ByteBuf data) { + ByteBuf fullFrameHeaderBuf = ByteBufUtils.accumulate( + frameHeaderBuffer, + data, + FRAME_HEADER_LENGTH, + frameHeaderBuffer.readableBytes()); + + if (fullFrameHeaderBuf != null) { + int messageAndFrameLength = fullFrameHeaderBuf.readInt(); + checkState(messageAndFrameLength >= 0, "The length field of current message must be non-negative"); + + int magicNumber = fullFrameHeaderBuf.readInt(); + checkState(magicNumber == MAGIC_NUMBER, "Network stream corrupted: received incorrect magic number."); + + int msgId = fullFrameHeaderBuf.readByte(); + if (msgId == NettyMessage.BufferResponse.ID) { + currentDecoder = bufferResponseDecoder; + } else { + currentDecoder = nonBufferResponseDecoder; + } + + currentDecoder.onNewMessageReceived(msgId, messageAndFrameLength - FRAME_HEADER_LENGTH); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessageDecoder.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessageDecoder.java new file mode 100644 index 0000000000000000000000000000000000000000..b514eb569f20adda54a33b1c7783b49dccc42645 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessageDecoder.java @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.io.network.netty; + +import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf; +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext; + +import javax.annotation.Nullable; + +/** + * Base class of decoders for specified netty messages. + */ +abstract class NettyMessageDecoder implements AutoCloseable { + + /** ID of the message under decoding. */ + protected int msgId; + + /** Length of the message under decoding. */ + protected int messageLength; + + /** + * The result of decoding one {@link ByteBuf}. + */ + static class DecodingResult { + final static DecodingResult NOT_FINISHED = new DecodingResult(false, null); + + private final boolean finished; + + @Nullable + private final NettyMessage message; + + private DecodingResult(boolean finished, @Nullable NettyMessage message) { + this.finished = finished; + this.message = message; + } + + public boolean isFinished() { + return finished; + } + + @Nullable + public NettyMessage getMessage() { + return message; + } + + static DecodingResult fullMessage(NettyMessage message) { + return new DecodingResult(true, message); + } + } + + /** + * Notifies that the underlying channel becomes active. + * + * @param ctx The context for the callback. + */ + abstract void onChannelActive(ChannelHandlerContext ctx); + + /** + * Notifies that a new message is to be decoded. + * + * @param msgId The type of the message to be decoded. + * @param messageLength The length of the message to be decoded. + */ + void onNewMessageReceived(int msgId, int messageLength) { + this.msgId = msgId; + this.messageLength = messageLength; + } + + /** + * Notifies that more data is received to continue decoding. + * + * @param data The data received. + * @return The result of decoding received data. + */ + abstract DecodingResult onChannelRead(ByteBuf data) throws Exception; +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyPartitionRequestClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyPartitionRequestClient.java index 4d42a3ca7fdbe50b121c609d316d5716411db5f3..35e15723674f5a90680615ad65e46a69d791ab4e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyPartitionRequestClient.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyPartitionRequestClient.java @@ -175,6 +175,11 @@ public class NettyPartitionRequestClient implements PartitionRequestClient { clientHandler.notifyCreditAvailable(inputChannel); } + @Override + public void resumeConsumption(RemoteInputChannel inputChannel) { + clientHandler.resumeConsumption(inputChannel); + } + @Override public void close(RemoteInputChannel inputChannel) throws IOException { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyProtocol.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyProtocol.java index 4cf98e0351afcc7306f110f8e3bfb368a598e98d..497d40d19dedf8ea8f88abeb0d2cd2b93b0aa6ea 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyProtocol.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyProtocol.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.io.network.netty; +import org.apache.flink.runtime.io.network.NetworkClientHandler; import org.apache.flink.runtime.io.network.TaskEventPublisher; import org.apache.flink.runtime.io.network.partition.ResultPartitionProvider; @@ -120,10 +121,12 @@ public class NettyProtocol { * @return channel handlers */ public ChannelHandler[] getClientChannelHandlers() { - return new ChannelHandler[] { + NetworkClientHandler networkClientHandler = new CreditBasedPartitionRequestClientHandler(); + + return new ChannelHandler[]{ messageEncoder, - new NettyMessage.NettyMessageDecoder(), - new CreditBasedPartitionRequestClientHandler()}; + new NettyMessageClientDecoderDelegate(networkClientHandler), + networkClientHandler}; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NetworkBufferAllocator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NetworkBufferAllocator.java new file mode 100644 index 0000000000000000000000000000000000000000..1687632831f90a4f6f4e083de404fb8875298c9a --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NetworkBufferAllocator.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.io.network.netty; + +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.core.memory.MemorySegmentFactory; +import org.apache.flink.runtime.io.network.NetworkClientHandler; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler; +import org.apache.flink.runtime.io.network.buffer.NetworkBuffer; +import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID; +import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel; + +import javax.annotation.Nullable; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * An allocator used for requesting buffers in the client side netty handlers. + */ +class NetworkBufferAllocator { + private final NetworkClientHandler networkClientHandler; + + NetworkBufferAllocator(NetworkClientHandler networkClientHandler) { + this.networkClientHandler = checkNotNull(networkClientHandler); + } + + /** + * Allocates a pooled network buffer for the specific input channel. + * + * @param receiverId The id of the requested input channel. + * @return The pooled network buffer. + */ + @Nullable + Buffer allocatePooledNetworkBuffer(InputChannelID receiverId) { + Buffer buffer = null; + + RemoteInputChannel inputChannel = networkClientHandler.getInputChannel(receiverId); + + // If the input channel has been released, we cannot allocate buffer and the received message + // will be discarded. + if (inputChannel != null) { + buffer = inputChannel.requestBuffer(); + } + + return buffer; + } + + /** + * Allocates an un-pooled network buffer with the specific size. + * + * @param size The requested buffer size. + * @return The un-pooled network buffer. + */ + Buffer allocateUnPooledNetworkBuffer(int size) { + byte[] byteArray = new byte[size]; + MemorySegment memSeg = MemorySegmentFactory.wrap(byteArray); + + return new NetworkBuffer(memSeg, FreeingBufferRecycler.INSTANCE, Buffer.DataType.EVENT_BUFFER); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NonBufferResponseDecoder.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NonBufferResponseDecoder.java new file mode 100644 index 0000000000000000000000000000000000000000..7380e92bba989c4ba6a409b926c23add8cc224a3 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NonBufferResponseDecoder.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.io.network.netty; + +import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf; +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext; + +import java.net.ProtocolException; + +import static org.apache.flink.runtime.io.network.netty.NettyMessage.BufferResponse; +import static org.apache.flink.runtime.io.network.netty.NettyMessage.ErrorResponse; + +/** + * The decoder for messages other than {@link BufferResponse}. + */ +class NonBufferResponseDecoder extends NettyMessageDecoder { + + /** The initial size of the message header accumulation buffer. */ + private static final int INITIAL_MESSAGE_HEADER_BUFFER_LENGTH = 128; + + /** The accumulation buffer of the message. */ + private ByteBuf messageBuffer; + + @Override + public void onChannelActive(ChannelHandlerContext ctx) { + messageBuffer = ctx.alloc().directBuffer(INITIAL_MESSAGE_HEADER_BUFFER_LENGTH); + } + + @Override + void onNewMessageReceived(int msgId, int messageLength) { + super.onNewMessageReceived(msgId, messageLength); + messageBuffer.clear(); + ensureBufferCapacity(); + } + + @Override + public DecodingResult onChannelRead(ByteBuf data) throws Exception { + ByteBuf fullFrameHeaderBuf = ByteBufUtils.accumulate( + messageBuffer, + data, + messageLength, + messageBuffer.readableBytes()); + if (fullFrameHeaderBuf == null) { + return DecodingResult.NOT_FINISHED; + } + + switch (msgId) { + case ErrorResponse.ID: + return DecodingResult.fullMessage(ErrorResponse.readFrom(fullFrameHeaderBuf)); + default: + throw new ProtocolException("Received unknown message from producer: " + msgId); + } + } + + /** + * Ensures the message header accumulation buffer has enough capacity for + * the current message. + */ + private void ensureBufferCapacity() { + if (messageBuffer.capacity() < messageLength) { + messageBuffer.capacity(messageLength); + } + } + + @Override + public void close() { + messageBuffer.release(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueue.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueue.java index e01e86e0cdb58d9b5c4909166335d2ee1088fb94..593c77f82a0bad6631e25f2a865336d2b746a384 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueue.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueue.java @@ -41,6 +41,7 @@ import java.io.IOException; import java.util.ArrayDeque; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.function.Consumer; import static org.apache.flink.runtime.io.network.netty.NettyMessage.BufferResponse; @@ -140,20 +141,22 @@ class PartitionRequestQueue extends ChannelInboundHandlerAdapter { } /** - * Adds unannounced credits from the consumer and enqueues the corresponding reader for this - * consumer (if not enqueued yet). + * Adds unannounced credits from the consumer or resumes data consumption after an exactly-once + * checkpoint and enqueues the corresponding reader for this consumer (if not enqueued yet). * * @param receiverId The input channel id to identify the consumer. - * @param credit The unannounced credits of the consumer. + * @param operation The operation to be performed (add credit or resume data consumption). */ - void addCredit(InputChannelID receiverId, int credit) throws Exception { + void addCreditOrResumeConsumption( + InputChannelID receiverId, + Consumer operation) throws Exception { if (fatalError) { return; } NetworkSequenceViewReader reader = allReaders.get(receiverId); if (reader != null) { - reader.addCredit(credit); + operation.accept(reader); enqueueAvailableReader(reader); } else { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestServerHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestServerHandler.java index b7d8ecf50d0406cec3a6db44445ceba27f1913cf..4b836ba617df7f5347fe99e61b44148c88c8f703 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestServerHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestServerHandler.java @@ -23,6 +23,7 @@ import org.apache.flink.runtime.io.network.TaskEventPublisher; import org.apache.flink.runtime.io.network.netty.NettyMessage.AddCredit; import org.apache.flink.runtime.io.network.netty.NettyMessage.CancelPartitionRequest; import org.apache.flink.runtime.io.network.netty.NettyMessage.CloseRequest; +import org.apache.flink.runtime.io.network.netty.NettyMessage.ResumeConsumption; import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException; import org.apache.flink.runtime.io.network.partition.ResultPartitionProvider; import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID; @@ -117,7 +118,11 @@ class PartitionRequestServerHandler extends SimpleChannelInboundHandler reader.addCredit(request.credit)); + } else if (msgClazz == ResumeConsumption.class) { + ResumeConsumption request = (ResumeConsumption) msg; + + outboundQueue.addCreditOrResumeConsumption(request.receiverId, NetworkSequenceViewReader::resumeConsumption); } else { LOG.warn("Received unexpected client request: {}", msg); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartition.java index 626ba3b8f874ae8a1e9fdcdc6ec074e37f700a59..abb9bc813869255e88ff307635246c3021e5f37c 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartition.java @@ -30,6 +30,7 @@ import javax.annotation.concurrent.GuardedBy; import java.io.File; import java.io.IOException; import java.util.HashSet; +import java.util.List; import java.util.Set; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -115,7 +116,7 @@ final class BoundedBlockingSubpartition extends ResultSubpartition { } @Override - public boolean add(BufferConsumer bufferConsumer) throws IOException { + public boolean add(BufferConsumer bufferConsumer, boolean isPriorityEvent) throws IOException { if (isFinished()) { bufferConsumer.close(); return false; @@ -145,6 +146,11 @@ final class BoundedBlockingSubpartition extends ResultSubpartition { } } + @Override + public List requestInflightBufferSnapshot() { + throw new UnsupportedOperationException("The batch job does not support unaligned checkpoint."); + } + private void writeAndCloseBufferConsumer(BufferConsumer bufferConsumer) throws IOException { try { final Buffer buffer = bufferConsumer.build(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionReader.java index 83858e9436d55d857bad0ebf19a6654698d3f397..bb4cf35ea4497fedd652002625d0f277ea9fb62e 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionReader.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionReader.java @@ -146,13 +146,17 @@ final class BoundedBlockingSubpartitionReader implements ResultSubpartitionView } @Override - public boolean nextBufferIsEvent() { - return nextBuffer != null && !nextBuffer.isBuffer(); + public void resumeConsumption() { + throw new UnsupportedOperationException("Method should never be called."); } @Override - public boolean isAvailable() { - return nextBuffer != null; + public boolean isAvailable(int numCreditsAvailable) { + if (numCreditsAvailable > 0) { + return nextBuffer != null; + } + + return nextBuffer != null && !nextBuffer.isBuffer(); } @Override @@ -170,6 +174,6 @@ final class BoundedBlockingSubpartitionReader implements ResultSubpartitionView public String toString() { return String.format("Blocking Subpartition Reader: ID=%s, index=%d", parent.parent.getPartitionId(), - parent.index); + parent.getSubPartitionIndex()); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BufferAvailabilityListener.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BufferAvailabilityListener.java index e78f99afee80c03d8a595ce4e9ca619bb839b84d..6686558a246a2c5289bbc02c4014a79697490431 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BufferAvailabilityListener.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BufferAvailabilityListener.java @@ -18,6 +18,10 @@ package org.apache.flink.runtime.io.network.partition; +import org.apache.flink.runtime.io.network.buffer.BufferConsumer; + +import java.io.IOException; + /** * Listener interface implemented by consumers of {@link ResultSubpartitionView} * that want to be notified of availability of further buffers. @@ -28,4 +32,13 @@ public interface BufferAvailabilityListener { * Called whenever there might be new data available. */ void notifyDataAvailable(); + + /** + * Allows the listener to react to a priority event before it is added to the outgoing buffer queue. + * + * @return true if the event has been fully processed and should not be added to the buffer queue. + */ + default boolean notifyPriorityEvent(BufferConsumer eventBufferConsumer) throws IOException { + return false; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BufferReaderWriterUtil.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BufferReaderWriterUtil.java index 1a69e2203a55dd27fcd3af142ca678724d0a88ed..e6c9e87d41763aa748f7d660722ba5f63dd04491 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BufferReaderWriterUtil.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BufferReaderWriterUtil.java @@ -92,7 +92,8 @@ final class BufferReaderWriterUtil { MemorySegment memorySegment = MemorySegmentFactory.wrapOffHeapMemory(buf); - return new NetworkBuffer(memorySegment, FreeingBufferRecycler.INSTANCE, !isEvent, isCompressed, size); + Buffer.DataType dataType = isEvent ? Buffer.DataType.EVENT_BUFFER : Buffer.DataType.DATA_BUFFER; + return new NetworkBuffer(memorySegment, FreeingBufferRecycler.INSTANCE, dataType, isCompressed, size); } // ------------------------------------------------------------------------ @@ -172,7 +173,8 @@ final class BufferReaderWriterUtil { readByteBufferFully(channel, targetBuf); - return new NetworkBuffer(memorySegment, bufferRecycler, !isEvent, isCompressed, size); + Buffer.DataType dataType = isEvent ? Buffer.DataType.EVENT_BUFFER : Buffer.DataType.DATA_BUFFER; + return new NetworkBuffer(memorySegment, bufferRecycler, dataType, isCompressed, size); } static ByteBuffer allocatedHeaderBuffer() { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ClusterPartitionManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ClusterPartitionManager.java new file mode 100644 index 0000000000000000000000000000000000000000..6290b4f192d09e3053dda1cf4e30433c351b7716 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ClusterPartitionManager.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.io.network.partition; + +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; + +import java.util.Map; +import java.util.concurrent.CompletableFuture; + +/** + * Interface for components that manage cluster partitions. + */ +public interface ClusterPartitionManager { + + /** + * Returns all datasets for which partitions are being tracked. + * + * @return tracked datasets + */ + CompletableFuture> listDataSets(); + + /** + * Releases all partitions associated with the given dataset. + * + * @param dataSetToRelease dataset for which all associated partitions should be released + * @return future that is completed once all partitions have been released + */ + CompletableFuture releaseClusterPartitions(IntermediateDataSetID dataSetToRelease); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/DataSetMetaInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/DataSetMetaInfo.java new file mode 100644 index 0000000000000000000000000000000000000000..0da50eb5e366dfd2367d08b87b910b3e5b763da4 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/DataSetMetaInfo.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.io.network.partition; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.util.Preconditions; + +import java.util.Optional; + +/** + * Container for meta-data of a data set. + */ +public final class DataSetMetaInfo { + private static final int UNKNOWN = -1; + + private final int numRegisteredPartitions; + private final int numTotalPartitions; + + private DataSetMetaInfo(int numRegisteredPartitions, int numTotalPartitions) { + this.numRegisteredPartitions = numRegisteredPartitions; + this.numTotalPartitions = numTotalPartitions; + } + + public Optional getNumRegisteredPartitions() { + return numRegisteredPartitions == UNKNOWN + ? Optional.empty() + : Optional.of(numRegisteredPartitions); + } + + public int getNumTotalPartitions() { + return numTotalPartitions; + } + + static DataSetMetaInfo withoutNumRegisteredPartitions(int numTotalPartitions) { + return new DataSetMetaInfo(UNKNOWN, numTotalPartitions); + } + + @VisibleForTesting + public static DataSetMetaInfo withNumRegisteredPartitions(int numRegisteredPartitions, int numTotalPartitions) { + Preconditions.checkArgument(numRegisteredPartitions > 0); + return new DataSetMetaInfo(numRegisteredPartitions, numTotalPartitions); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/NoOpResultSubpartitionView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/NoOpResultSubpartitionView.java index 2b649c41375894c8622f78d8b4b15bef85c72e86..f8022a13c66277facad1ae07d179cfcbd76f8f21 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/NoOpResultSubpartitionView.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/NoOpResultSubpartitionView.java @@ -44,17 +44,16 @@ public class NoOpResultSubpartitionView implements ResultSubpartitionView { } @Override - public Throwable getFailureCause() { - return null; + public void resumeConsumption() { } @Override - public boolean nextBufferIsEvent() { - return false; + public Throwable getFailureCause() { + return null; } @Override - public boolean isAvailable() { + public boolean isAvailable(int numCreditsAvailable) { return false; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java index ecf695696aabb116e9cef101b017bba1c2f1de92..95f1168a1422ace395f537408127f3b748078af0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java @@ -19,9 +19,12 @@ package org.apache.flink.runtime.io.network.partition; import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.runtime.checkpoint.channel.ChannelStateReader; +import org.apache.flink.runtime.checkpoint.channel.ChannelStateReader.ReadResult; import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent; import org.apache.flink.runtime.io.network.api.serialization.EventSerializer; import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.BufferBuilder; import org.apache.flink.runtime.io.network.buffer.BufferConsumer; import org.slf4j.Logger; @@ -32,6 +35,8 @@ import javax.annotation.concurrent.GuardedBy; import java.io.IOException; import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.List; import static org.apache.flink.util.Preconditions.checkNotNull; import static org.apache.flink.util.Preconditions.checkState; @@ -39,10 +44,10 @@ import static org.apache.flink.util.Preconditions.checkState; /** * A pipelined in-memory only subpartition, which can be consumed once. * - *

Whenever {@link #add(BufferConsumer)} adds a finished {@link BufferConsumer} or a second + *

Whenever {@link ResultSubpartition#add(BufferConsumer, boolean)} adds a finished {@link BufferConsumer} or a second * {@link BufferConsumer} (in which case we will assume the first one finished), we will * {@link PipelinedSubpartitionView#notifyDataAvailable() notify} a read view created via - * {@link #createReadView(BufferAvailabilityListener)} of new data availability. Except by calling + * {@link ResultSubpartition#createReadView(BufferAvailabilityListener)} of new data availability. Except by calling * {@link #flush()} explicitly, we always only notify when the first finished buffer turns up and * then, the reader has to drain the buffers via {@link #pollBuffer()} until its return value shows * no more buffers being available. This results in a buffer queue which is either empty or has an @@ -52,7 +57,7 @@ import static org.apache.flink.util.Preconditions.checkState; * {@link PipelinedSubpartitionView#notifyDataAvailable() notification} for any * {@link BufferConsumer} present in the queue. */ -class PipelinedSubpartition extends ResultSubpartition { +public class PipelinedSubpartition extends ResultSubpartition { private static final Logger LOG = LoggerFactory.getLogger(PipelinedSubpartition.class); @@ -83,6 +88,13 @@ class PipelinedSubpartition extends ResultSubpartition { /** The total number of bytes (both data and event buffers). */ private long totalNumberOfBytes; + /** The collection of buffers which are spanned over by checkpoint barrier and needs to be persisted for snapshot. */ + private final List inflightBufferSnapshot = new ArrayList<>(); + + /** Whether this subpartition is blocked by exactly once checkpoint and is waiting for resumption. */ + @GuardedBy("buffers") + private boolean isBlockedByCheckpoint = false; + // ------------------------------------------------------------------------ PipelinedSubpartition(int index, ResultPartition parent) { @@ -90,17 +102,41 @@ class PipelinedSubpartition extends ResultSubpartition { } @Override - public boolean add(BufferConsumer bufferConsumer) { - return add(bufferConsumer, false); + public void initializeState(ChannelStateReader stateReader) throws IOException, InterruptedException { + for (ReadResult readResult = ReadResult.HAS_MORE_DATA; readResult == ReadResult.HAS_MORE_DATA;) { + BufferBuilder bufferBuilder = parent.getBufferPool().requestBufferBuilderBlocking(); + BufferConsumer bufferConsumer = bufferBuilder.createBufferConsumer(); + readResult = stateReader.readOutputData(subpartitionInfo, bufferBuilder); + + // check whether there are some states data filled in this time + if (bufferConsumer.isDataAvailable()) { + add(bufferConsumer, false, false); + bufferBuilder.finish(); + } else { + bufferConsumer.close(); + } + } + } + + @Override + public boolean add(BufferConsumer bufferConsumer, boolean isPriorityEvent) throws IOException { + if (isPriorityEvent) { + if (readView != null && readView.notifyPriorityEvent(bufferConsumer)) { + bufferConsumer.close(); + return true; + } + return add(bufferConsumer, false, true); + } + return add(bufferConsumer, false, false); } @Override public void finish() throws IOException { - add(EventSerializer.toBufferConsumer(EndOfPartitionEvent.INSTANCE), true); + add(EventSerializer.toBufferConsumer(EndOfPartitionEvent.INSTANCE), true, false); LOG.debug("{}: Finished {}.", parent.getOwningTaskName(), this); } - private boolean add(BufferConsumer bufferConsumer, boolean finish) { + private boolean add(BufferConsumer bufferConsumer, boolean finish, boolean insertAsHead) { checkNotNull(bufferConsumer); final boolean notifyDataAvailable; @@ -111,10 +147,10 @@ class PipelinedSubpartition extends ResultSubpartition { } // Add the bufferConsumer and update the stats - buffers.add(bufferConsumer); + handleAddingBarrier(bufferConsumer, insertAsHead); updateStatistics(bufferConsumer); increaseBuffersInBacklog(bufferConsumer); - notifyDataAvailable = shouldNotifyDataAvailable() || finish; + notifyDataAvailable = insertAsHead || finish || shouldNotifyDataAvailable(); isFinished |= finish; } @@ -126,6 +162,32 @@ class PipelinedSubpartition extends ResultSubpartition { return true; } + private void handleAddingBarrier(BufferConsumer bufferConsumer, boolean insertAsHead) { + assert Thread.holdsLock(buffers); + if (insertAsHead) { + checkState(inflightBufferSnapshot.isEmpty(), "Supporting only one concurrent checkpoint in unaligned " + + "checkpoints"); + + // Meanwhile prepare the collection of in-flight buffers which would be fetched in the next step later. + for (BufferConsumer buffer : buffers) { + try (BufferConsumer bc = buffer.copy()) { + inflightBufferSnapshot.add(bc.build()); + } + } + + buffers.addFirst(bufferConsumer); + } else { + buffers.add(bufferConsumer); + } + } + + @Override + public List requestInflightBufferSnapshot() { + List snapshot = new ArrayList<>(inflightBufferSnapshot); + inflightBufferSnapshot.clear(); + return snapshot; + } + @Override public void release() { // view reference accessible outside the lock, but assigned inside the locked scope @@ -159,6 +221,10 @@ class PipelinedSubpartition extends ResultSubpartition { @Nullable BufferAndBacklog pollBuffer() { synchronized (buffers) { + if (isBlockedByCheckpoint) { + return null; + } + Buffer buffer = null; if (buffers.isEmpty()) { @@ -197,28 +263,28 @@ class PipelinedSubpartition extends ResultSubpartition { return null; } + if (Buffer.DataType.isAlignedExactlyOnceCheckpointBarrier(buffer)) { + isBlockedByCheckpoint = true; + } + updateStatistics(buffer); // Do not report last remaining buffer on buffers as available to read (assuming it's unfinished). // It will be reported for reading either on flush or when the number of buffers in the queue // will be 2 or more. return new BufferAndBacklog( buffer, - isAvailableUnsafe(), + isDataAvailableUnsafe(), getBuffersInBacklog(), - nextBufferIsEventUnsafe()); + isEventAvailableUnsafe()); } } - boolean nextBufferIsEvent() { + void resumeConsumption() { synchronized (buffers) { - return nextBufferIsEventUnsafe(); - } - } + checkState(isBlockedByCheckpoint, "Should be blocked by checkpoint."); - private boolean nextBufferIsEventUnsafe() { - assert Thread.holdsLock(buffers); - - return !buffers.isEmpty() && !buffers.peekFirst().isBuffer(); + isBlockedByCheckpoint = false; + } } @Override @@ -239,11 +305,13 @@ class PipelinedSubpartition extends ResultSubpartition { synchronized (buffers) { checkState(!isReleased); checkState(readView == null, - "Subpartition %s of is being (or already has been) consumed, " + - "but pipelined subpartitions can only be consumed once.", index, parent.getPartitionId()); + "Subpartition %s of is being (or already has been) consumed, " + + "but pipelined subpartitions can only be consumed once.", + getSubPartitionIndex(), + parent.getPartitionId()); LOG.debug("{}: Creating read view for subpartition {} of partition {}.", - parent.getOwningTaskName(), index, parent.getPartitionId()); + parent.getOwningTaskName(), getSubPartitionIndex(), parent.getPartitionId()); readView = new PipelinedSubpartitionView(this, availabilityListener); notifyDataAvailable = !buffers.isEmpty(); @@ -255,14 +323,26 @@ class PipelinedSubpartition extends ResultSubpartition { return readView; } - public boolean isAvailable() { + public boolean isAvailable(int numCreditsAvailable) { synchronized (buffers) { - return isAvailableUnsafe(); + if (numCreditsAvailable > 0) { + return isDataAvailableUnsafe(); + } + + return isEventAvailableUnsafe(); } } - private boolean isAvailableUnsafe() { - return flushRequested || getNumberOfFinishedBuffers() > 0; + private boolean isDataAvailableUnsafe() { + assert Thread.holdsLock(buffers); + + return !isBlockedByCheckpoint && (flushRequested || getNumberOfFinishedBuffers() > 0); + } + + private boolean isEventAvailableUnsafe() { + assert Thread.holdsLock(buffers); + + return !isBlockedByCheckpoint && !buffers.isEmpty() && !buffers.peekFirst().isBuffer(); } // ------------------------------------------------------------------------ @@ -289,7 +369,7 @@ class PipelinedSubpartition extends ResultSubpartition { return String.format( "PipelinedSubpartition#%d [number of buffers: %d (%d bytes), number of buffers in backlog: %d, finished? %s, read view? %s]", - index, numBuffers, numBytes, getBuffersInBacklog(), finished, hasReadView); + getSubPartitionIndex(), numBuffers, numBytes, getBuffersInBacklog(), finished, hasReadView); } @Override @@ -302,13 +382,13 @@ class PipelinedSubpartition extends ResultSubpartition { public void flush() { final boolean notifyDataAvailable; synchronized (buffers) { - if (buffers.isEmpty()) { + if (buffers.isEmpty() || flushRequested) { return; } // if there is more then 1 buffer, we already notified the reader // (at the latest when adding the second buffer) - notifyDataAvailable = !flushRequested && buffers.size() == 1 && buffers.peek().isDataAvailable(); - flushRequested = flushRequested || buffers.size() > 1 || notifyDataAvailable; + notifyDataAvailable = !isBlockedByCheckpoint && buffers.size() == 1 && buffers.peek().isDataAvailable(); + flushRequested = buffers.size() > 1 || notifyDataAvailable; } if (notifyDataAvailable) { notifyDataAvailable(); @@ -376,7 +456,7 @@ class PipelinedSubpartition extends ResultSubpartition { private boolean shouldNotifyDataAvailable() { // Notify only when we added first finished buffer. - return readView != null && !flushRequested && getNumberOfFinishedBuffers() == 1; + return readView != null && !flushRequested && !isBlockedByCheckpoint && getNumberOfFinishedBuffers() == 1; } private void notifyDataAvailable() { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionView.java index febbfbd0c0b5072075801e7350f0be1cddc69934..875d1246b38a3e07b520d267aca7370a86b46fa8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionView.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionView.java @@ -18,10 +18,12 @@ package org.apache.flink.runtime.io.network.partition; +import org.apache.flink.runtime.io.network.buffer.BufferConsumer; import org.apache.flink.runtime.io.network.partition.ResultSubpartition.BufferAndBacklog; import javax.annotation.Nullable; +import java.io.IOException; import java.util.concurrent.atomic.AtomicBoolean; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -29,7 +31,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull; /** * View over a pipelined in-memory only subpartition. */ -class PipelinedSubpartitionView implements ResultSubpartitionView { +public class PipelinedSubpartitionView implements ResultSubpartitionView { /** The subpartition this view belongs to. */ private final PipelinedSubpartition parent; @@ -39,7 +41,7 @@ class PipelinedSubpartitionView implements ResultSubpartitionView { /** Flag indicating whether this view has been released. */ private final AtomicBoolean isReleased; - PipelinedSubpartitionView(PipelinedSubpartition parent, BufferAvailabilityListener listener) { + public PipelinedSubpartitionView(PipelinedSubpartition parent, BufferAvailabilityListener listener) { this.parent = checkNotNull(parent); this.availabilityListener = checkNotNull(listener); this.isReleased = new AtomicBoolean(); @@ -71,13 +73,13 @@ class PipelinedSubpartitionView implements ResultSubpartitionView { } @Override - public boolean nextBufferIsEvent() { - return parent.nextBufferIsEvent(); + public void resumeConsumption() { + parent.resumeConsumption(); } @Override - public boolean isAvailable() { - return parent.isAvailable(); + public boolean isAvailable(int numCreditsAvailable) { + return parent.isAvailable(numCreditsAvailable); } @Override @@ -93,7 +95,11 @@ class PipelinedSubpartitionView implements ResultSubpartitionView { @Override public String toString() { return String.format("PipelinedSubpartitionView(index: %d) of ResultPartition %s", - parent.index, + parent.getSubPartitionIndex(), parent.parent.getPartitionId()); } + + public boolean notifyPriorityEvent(BufferConsumer eventBufferConsumer) throws IOException { + return availabilityListener.notifyPriorityEvent(eventBufferConsumer); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ReleaseOnConsumptionResultPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ReleaseOnConsumptionResultPartition.java index d32bfa6141d985e4c96f24323001730c9858c3f7..4c6d9a4b6ac4758b3d85be2006e4703ae9ac6bd0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ReleaseOnConsumptionResultPartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ReleaseOnConsumptionResultPartition.java @@ -48,6 +48,7 @@ public class ReleaseOnConsumptionResultPartition extends ResultPartition { ReleaseOnConsumptionResultPartition( String owningTaskName, + int partitionIndex, ResultPartitionID partitionId, ResultPartitionType partitionType, ResultSubpartition[] subpartitions, @@ -57,6 +58,7 @@ public class ReleaseOnConsumptionResultPartition extends ResultPartition { FunctionWithException bufferPoolFactory) { super( owningTaskName, + partitionIndex, partitionId, partitionType, subpartitions, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResourceManagerPartitionTracker.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResourceManagerPartitionTracker.java new file mode 100644 index 0000000000000000000000000000000000000000..82f8afd6d3302729329452905710cbdc047625d9 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResourceManagerPartitionTracker.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.io.network.partition; + +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; +import org.apache.flink.runtime.taskexecutor.partition.ClusterPartitionReport; + +import java.util.Map; +import java.util.concurrent.CompletableFuture; + +/** + * Utility for tracking and releasing partitions on the ResourceManager. + */ +public interface ResourceManagerPartitionTracker { + + /** + * Processes {@link ClusterPartitionReport} of a task executor. Updates the tracking information for the respective + * task executor. Any partition no longer being hosted on the task executor is considered lost, corrupting the + * corresponding data set. + * For any such data set this method issues partition release calls to all task executors that are hosting + * partitions of this data set. + * + * @param taskExecutorId origin of the report + * @param clusterPartitionReport partition report + */ + void processTaskExecutorClusterPartitionReport(ResourceID taskExecutorId, ClusterPartitionReport clusterPartitionReport); + + /** + * Processes the shutdown of task executor. Removes all tracking information for the given executor, determines + * datasets that may be corrupted by the shutdown (and implied loss of partitions). + * For any such data set this method issues partition release calls to all task executors that are hosting + * partitions of this data set, and issues release calls. + * + * @param taskExecutorId task executor that shut down + */ + void processTaskExecutorShutdown(ResourceID taskExecutorId); + + /** + * Issues a release calls to all task executors that are hosting partitions of the given data set. + * + * @param dataSetId data set to release + */ + CompletableFuture releaseClusterPartitions(IntermediateDataSetID dataSetId); + + /** + * Returns all data sets for which partitions are being tracked. + * + * @return tracked datasets + */ + Map listDataSets(); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResourceManagerPartitionTrackerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResourceManagerPartitionTrackerFactory.java new file mode 100644 index 0000000000000000000000000000000000000000..215e3db4554c2134d8f0e5fabe5da3e067c5c7f3 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResourceManagerPartitionTrackerFactory.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.io.network.partition; + +/** + * Factory for {@link ResourceManagerPartitionTracker}. + */ +@FunctionalInterface +public interface ResourceManagerPartitionTrackerFactory { + ResourceManagerPartitionTracker get(TaskExecutorClusterPartitionReleaser taskExecutorClusterPartitionReleaser); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResourceManagerPartitionTrackerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResourceManagerPartitionTrackerImpl.java new file mode 100644 index 0000000000000000000000000000000000000000..2c77a6e0a895037441d57c0815dd0989a1a61967 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResourceManagerPartitionTrackerImpl.java @@ -0,0 +1,246 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.io.network.partition; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; +import org.apache.flink.runtime.taskexecutor.partition.ClusterPartitionReport; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; + +/** + * Default {@link ResourceManagerPartitionTracker} implementation. + * + *

Internal tracking info must only be updated upon reception of a {@link ClusterPartitionReport}, as the task + * executor state is the source of truth. + */ +public class ResourceManagerPartitionTrackerImpl implements ResourceManagerPartitionTracker { + + private static final Logger LOG = LoggerFactory.getLogger(ResourceManagerPartitionTrackerImpl.class); + + private final Map> taskExecutorToDataSets = new HashMap<>(); + private final Map>> dataSetToTaskExecutors = new HashMap<>(); + private final Map dataSetMetaInfo = new HashMap<>(); + private final Map> partitionReleaseCompletionFutures = new HashMap<>(); + + private final TaskExecutorClusterPartitionReleaser taskExecutorClusterPartitionReleaser; + + public ResourceManagerPartitionTrackerImpl(TaskExecutorClusterPartitionReleaser taskExecutorClusterPartitionReleaser) { + this.taskExecutorClusterPartitionReleaser = taskExecutorClusterPartitionReleaser; + } + + @Override + public void processTaskExecutorClusterPartitionReport(ResourceID taskExecutorId, ClusterPartitionReport clusterPartitionReport) { + Preconditions.checkNotNull(taskExecutorId); + Preconditions.checkNotNull(clusterPartitionReport); + LOG.debug("Processing cluster partition report from task executor {}: {}.", taskExecutorId, clusterPartitionReport); + + internalProcessClusterPartitionReport(taskExecutorId, clusterPartitionReport); + } + + @Override + public void processTaskExecutorShutdown(ResourceID taskExecutorId) { + Preconditions.checkNotNull(taskExecutorId); + LOG.debug("Processing shutdown of task executor {}.", taskExecutorId); + + internalProcessClusterPartitionReport(taskExecutorId, new ClusterPartitionReport(Collections.emptyList())); + } + + @Override + public CompletableFuture releaseClusterPartitions(IntermediateDataSetID dataSetId) { + Preconditions.checkNotNull(dataSetId); + if (!dataSetMetaInfo.containsKey(dataSetId)) { + LOG.debug("Attempted released of unknown data set {}.", dataSetId); + return CompletableFuture.completedFuture(null); + } + LOG.debug("Releasing cluster partitions for data set {}.", dataSetId); + + CompletableFuture partitionReleaseCompletionFuture = partitionReleaseCompletionFutures.computeIfAbsent(dataSetId, ignored -> new CompletableFuture<>()); + internalReleasePartitions(Collections.singleton(dataSetId)); + return partitionReleaseCompletionFuture; + } + + private void internalProcessClusterPartitionReport(ResourceID taskExecutorId, ClusterPartitionReport clusterPartitionReport) { + final Set dataSetsWithLostPartitions = clusterPartitionReport.getEntries().isEmpty() + ? processEmptyReport(taskExecutorId) + : setHostedDataSetsAndCheckCorruption(taskExecutorId, clusterPartitionReport.getEntries()); + + updateDataSetMetaData(clusterPartitionReport); + + checkForFullyLostDatasets(dataSetsWithLostPartitions); + + internalReleasePartitions(dataSetsWithLostPartitions); + } + + private void internalReleasePartitions(Set dataSetsToRelease) { + Map> releaseCalls = prepareReleaseCalls(dataSetsToRelease); + releaseCalls.forEach(taskExecutorClusterPartitionReleaser::releaseClusterPartitions); + } + + private Set processEmptyReport(ResourceID taskExecutorId) { + Set previouslyHostedDatasets = taskExecutorToDataSets.remove(taskExecutorId); + if (previouslyHostedDatasets == null) { + // default path for task executors that never have any cluster partitions + previouslyHostedDatasets = Collections.emptySet(); + } else { + previouslyHostedDatasets.forEach(dataSetId -> removeInnerKey(dataSetId, taskExecutorId, dataSetToTaskExecutors)); + } + return previouslyHostedDatasets; + } + + /** + * Updates the data sets for which the given task executor is hosting partitions and returns data sets that were + * corrupted due to a loss of partitions. + * + * @param taskExecutorId ID of the hosting TaskExecutor + * @param reportEntries IDs of data sets for which partitions are hosted + * @return corrupted data sets + */ + private Set setHostedDataSetsAndCheckCorruption(ResourceID taskExecutorId, Collection reportEntries) { + final Set currentlyHostedDatasets = reportEntries + .stream() + .map(ClusterPartitionReport.ClusterPartitionReportEntry::getDataSetId) + .collect(Collectors.toSet()); + + final Set previouslyHostedDataSets = taskExecutorToDataSets.put( + taskExecutorId, + currentlyHostedDatasets); + + // previously tracked data sets may be corrupted since we may be tracking less partitions than before + final Set potentiallyCorruptedDataSets = Optional + .ofNullable(previouslyHostedDataSets) + .orElse(new HashSet<>(0)); + + // update data set -> task executor mapping and find datasets for which lost a partition + reportEntries.forEach(hostedPartition -> { + final Map> taskExecutorHosts = dataSetToTaskExecutors.computeIfAbsent(hostedPartition.getDataSetId(), ignored -> new HashMap<>()); + final Set previouslyHostedPartitions = taskExecutorHosts.put(taskExecutorId, hostedPartition.getHostedPartitions()); + + final boolean noPartitionLost = previouslyHostedPartitions == null || hostedPartition.getHostedPartitions().containsAll(previouslyHostedPartitions); + if (noPartitionLost) { + potentiallyCorruptedDataSets.remove(hostedPartition.getDataSetId()); + } + }); + + // now only contains data sets for which a partition is no longer tracked + return potentiallyCorruptedDataSets; + } + + private void updateDataSetMetaData(ClusterPartitionReport clusterPartitionReport) { + // add meta info for new data sets + clusterPartitionReport.getEntries().forEach(entry -> + dataSetMetaInfo.compute(entry.getDataSetId(), (dataSetID, dataSetMetaInfo) -> { + if (dataSetMetaInfo == null) { + return DataSetMetaInfo.withoutNumRegisteredPartitions(entry.getNumTotalPartitions()); + } else { + // double check that the meta data is consistent + Preconditions.checkState(dataSetMetaInfo.getNumTotalPartitions() == entry.getNumTotalPartitions()); + return dataSetMetaInfo; + } + })); + } + + private void checkForFullyLostDatasets(Set dataSetsWithLostPartitions) { + dataSetsWithLostPartitions.forEach(dataSetId -> { + if (getHostingTaskExecutors(dataSetId).isEmpty()) { + LOG.debug("There are no longer partitions being tracked for dataset {}.", dataSetId); + dataSetMetaInfo.remove(dataSetId); + Optional.ofNullable(partitionReleaseCompletionFutures.remove(dataSetId)).map(future -> future.complete(null)); + } + }); + } + + private Map> prepareReleaseCalls(Set dataSetsToRelease) { + final Map> releaseCalls = new HashMap<>(); + dataSetsToRelease.forEach(dataSetToRelease -> { + final Set hostingTaskExecutors = getHostingTaskExecutors(dataSetToRelease); + hostingTaskExecutors.forEach(hostingTaskExecutor -> insert(hostingTaskExecutor, dataSetToRelease, releaseCalls)); + }); + return releaseCalls; + } + + private Set getHostingTaskExecutors(IntermediateDataSetID dataSetId) { + Preconditions.checkNotNull(dataSetId); + + Map> trackedPartitions = dataSetToTaskExecutors.get(dataSetId); + if (trackedPartitions == null) { + return Collections.emptySet(); + } else { + return trackedPartitions.keySet(); + } + } + + @Override + public Map listDataSets() { + return dataSetMetaInfo.entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getKey, entry -> { + final Map> taskExecutorToPartitions = dataSetToTaskExecutors.get(entry.getKey()); + Preconditions.checkState(taskExecutorToPartitions != null, "Have metadata entry for dataset %s, but no partition is tracked.", entry.getKey()); + + int numTrackedPartitions = 0; + for (Set hostedPartitions : taskExecutorToPartitions.values()) { + numTrackedPartitions += hostedPartitions.size(); + } + + return DataSetMetaInfo.withNumRegisteredPartitions(numTrackedPartitions, entry.getValue().getNumTotalPartitions()); + })); + } + + /** + * Returns whether all maps are empty; used for checking for resource leaks in case entries aren't properly removed. + * + * @return whether all contained maps are empty + */ + @VisibleForTesting + boolean areAllMapsEmpty() { + return taskExecutorToDataSets.isEmpty() && dataSetToTaskExecutors.isEmpty() && dataSetMetaInfo.isEmpty() && partitionReleaseCompletionFutures.isEmpty(); + } + + private static void insert(K key1, V value, Map> collection) { + collection.compute(key1, (key, values) -> { + if (values == null) { + values = new HashSet<>(); + } + values.add(value); + return values; + }); + } + + private static void removeInnerKey(K1 key1, K2 value, Map> collection) { + collection.computeIfPresent(key1, (key, values) -> { + values.remove(value); + if (values.isEmpty()) { + return null; + } + return values; + }); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java index e814191fb7f017881f02339e49b08bbb478320f4..a89b56453ca5320b18a7332a56e67291a3e57082 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.io.network.partition; +import org.apache.flink.runtime.checkpoint.channel.ChannelStateReader; import org.apache.flink.runtime.executiongraph.IntermediateResultPartition; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; import org.apache.flink.runtime.io.network.buffer.Buffer; @@ -30,6 +31,7 @@ import org.apache.flink.runtime.io.network.partition.consumer.LocalInputChannel; import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel; import org.apache.flink.runtime.jobgraph.DistributionPattern; import org.apache.flink.runtime.taskexecutor.TaskExecutor; +import org.apache.flink.util.Preconditions; import org.apache.flink.util.function.FunctionWithException; import org.slf4j.Logger; @@ -76,6 +78,8 @@ public class ResultPartition implements ResultPartitionWriter, BufferPoolOwner { private final String owningTaskName; + private final int partitionIndex; + protected final ResultPartitionID partitionId; /** Type of this partition. Defines the concrete subpartition implementation to use. */ @@ -106,6 +110,7 @@ public class ResultPartition implements ResultPartitionWriter, BufferPoolOwner { public ResultPartition( String owningTaskName, + int partitionIndex, ResultPartitionID partitionId, ResultPartitionType partitionType, ResultSubpartition[] subpartitions, @@ -115,6 +120,8 @@ public class ResultPartition implements ResultPartitionWriter, BufferPoolOwner { FunctionWithException bufferPoolFactory) { this.owningTaskName = checkNotNull(owningTaskName); + Preconditions.checkArgument(0 <= partitionIndex, "The partition index must be positive."); + this.partitionIndex = partitionIndex; this.partitionId = checkNotNull(partitionId); this.partitionType = checkNotNull(partitionType); this.subpartitions = checkNotNull(subpartitions); @@ -144,6 +151,13 @@ public class ResultPartition implements ResultPartitionWriter, BufferPoolOwner { partitionManager.registerResultPartition(this); } + @Override + public void initializeState(ChannelStateReader stateReader) throws IOException, InterruptedException { + for (ResultSubpartition subpartition : subpartitions) { + subpartition.initializeState(stateReader); + } + } + public String getOwningTaskName() { return owningTaskName; } @@ -152,6 +166,15 @@ public class ResultPartition implements ResultPartitionWriter, BufferPoolOwner { return partitionId; } + public int getPartitionIndex() { + return partitionIndex; + } + + @Override + public ResultSubpartition getSubpartition(int subpartitionIndex) { + return subpartitions[subpartitionIndex]; + } + @Override public int getNumberOfSubpartitions() { return subpartitions.length; @@ -190,7 +213,16 @@ public class ResultPartition implements ResultPartitionWriter, BufferPoolOwner { } @Override - public boolean addBufferConsumer(BufferConsumer bufferConsumer, int subpartitionIndex) throws IOException { + public BufferBuilder tryGetBufferBuilder() throws IOException { + BufferBuilder bufferBuilder = bufferPool.requestBufferBuilder(); + return bufferBuilder; + } + + @Override + public boolean addBufferConsumer( + BufferConsumer bufferConsumer, + int subpartitionIndex, + boolean isPriorityEvent) throws IOException { checkNotNull(bufferConsumer); ResultSubpartition subpartition; @@ -203,7 +235,7 @@ public class ResultPartition implements ResultPartitionWriter, BufferPoolOwner { throw ex; } - return subpartition.add(bufferConsumer); + return subpartition.add(bufferConsumer, isPriorityEvent); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactory.java index e924bd0bbb35aa3b075c7738f4223ed0f952d912..f52ffed1333950922fd3f1c9c17012b8eaea59d7 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactory.java @@ -90,9 +90,11 @@ public class ResultPartitionFactory { public ResultPartition create( String taskNameWithSubtaskAndId, + int partitionIndex, ResultPartitionDeploymentDescriptor desc) { return create( taskNameWithSubtaskAndId, + partitionIndex, desc.getShuffleDescriptor().getResultPartitionID(), desc.getPartitionType(), desc.getNumberOfSubpartitions(), @@ -103,6 +105,7 @@ public class ResultPartitionFactory { @VisibleForTesting public ResultPartition create( String taskNameWithSubtaskAndId, + int partitionIndex, ResultPartitionID id, ResultPartitionType type, int numberOfSubpartitions, @@ -117,6 +120,7 @@ public class ResultPartitionFactory { ResultPartition partition = forcePartitionReleaseOnConsumption || !type.isBlocking() ? new ReleaseOnConsumptionResultPartition( taskNameWithSubtaskAndId, + partitionIndex, id, type, subpartitions, @@ -126,6 +130,7 @@ public class ResultPartitionFactory { bufferPoolFactory) : new ResultPartition( taskNameWithSubtaskAndId, + partitionIndex, id, type, subpartitions, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartition.java index 31b184ac99c6f96c57db8eba6b4c88c1cb66f4f9..5dc25eb31d817cf5851f6b57c1fe06645d283fac 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartition.java @@ -19,10 +19,13 @@ package org.apache.flink.runtime.io.network.partition; import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.runtime.checkpoint.channel.ChannelStateReader; +import org.apache.flink.runtime.checkpoint.channel.ResultSubpartitionInfo; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.buffer.BufferConsumer; import java.io.IOException; +import java.util.List; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -31,8 +34,8 @@ import static org.apache.flink.util.Preconditions.checkNotNull; */ public abstract class ResultSubpartition { - /** The index of the subpartition at the parent partition. */ - protected final int index; + /** The info of the subpartition to identify it globally within a task. */ + protected final ResultSubpartitionInfo subpartitionInfo; /** The parent partition this subpartition belongs to. */ protected final ResultPartition parent; @@ -40,8 +43,8 @@ public abstract class ResultSubpartition { // - Statistics ---------------------------------------------------------- public ResultSubpartition(int index, ResultPartition parent) { - this.index = index; this.parent = parent; + this.subpartitionInfo = new ResultSubpartitionInfo(parent.getPartitionIndex(), index); } /** @@ -52,6 +55,10 @@ public abstract class ResultSubpartition { return parent.bufferCompressor != null && buffer.isBuffer() && buffer.readableBytes() > 0; } + public ResultSubpartitionInfo getSubpartitionInfo() { + return subpartitionInfo; + } + /** * Gets the total numbers of buffers (data buffers plus events). */ @@ -59,13 +66,38 @@ public abstract class ResultSubpartition { protected abstract long getTotalNumberOfBytes(); + public int getSubPartitionIndex() { + return subpartitionInfo.getSubPartitionIdx(); + } + /** * Notifies the parent partition about a consumed {@link ResultSubpartitionView}. */ protected void onConsumedSubpartition() { - parent.onConsumedSubpartition(index); + parent.onConsumedSubpartition(getSubPartitionIndex()); + } + + public void initializeState(ChannelStateReader stateReader) throws IOException, InterruptedException { } + /** + * Adds the given buffer. + * + *

The request may be executed synchronously, or asynchronously, depending on the + * implementation. + * + *

IMPORTANT: Before adding new {@link BufferConsumer} previously added must be in finished + * state. Because of the performance reasons, this is only enforced during the data reading. + * + * @param bufferConsumer + * the buffer to add (transferring ownership to this writer) + * @param isPriorityEvent + * @return true if operation succeeded and bufferConsumer was enqueued for consumption. + * @throws IOException + * thrown in case of errors while adding the buffer + */ + public abstract boolean add(BufferConsumer bufferConsumer, boolean isPriorityEvent) throws IOException; + /** * Adds the given buffer. * @@ -81,7 +113,11 @@ public abstract class ResultSubpartition { * @throws IOException * thrown in case of errors while adding the buffer */ - public abstract boolean add(BufferConsumer bufferConsumer) throws IOException; + public boolean add(BufferConsumer bufferConsumer) throws IOException { + return add(bufferConsumer, false); + } + + public abstract List requestInflightBufferSnapshot(); public abstract void flush(); @@ -120,31 +156,31 @@ public abstract class ResultSubpartition { public static final class BufferAndBacklog { private final Buffer buffer; - private final boolean isMoreAvailable; + private final boolean isDataAvailable; private final int buffersInBacklog; - private final boolean nextBufferIsEvent; + private final boolean isEventAvailable; - public BufferAndBacklog(Buffer buffer, boolean isMoreAvailable, int buffersInBacklog, boolean nextBufferIsEvent) { + public BufferAndBacklog(Buffer buffer, boolean isDataAvailable, int buffersInBacklog, boolean isEventAvailable) { this.buffer = checkNotNull(buffer); this.buffersInBacklog = buffersInBacklog; - this.isMoreAvailable = isMoreAvailable; - this.nextBufferIsEvent = nextBufferIsEvent; + this.isDataAvailable = isDataAvailable; + this.isEventAvailable = isEventAvailable; } public Buffer buffer() { return buffer; } - public boolean isMoreAvailable() { - return isMoreAvailable; + public boolean isDataAvailable() { + return isDataAvailable; } public int buffersInBacklog() { return buffersInBacklog; } - public boolean nextBufferIsEvent() { - return nextBufferIsEvent; + public boolean isEventAvailable() { + return isEventAvailable; } public static BufferAndBacklog fromBufferAndLookahead(Buffer current, Buffer lookahead, int backlog) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartitionView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartitionView.java index f648a8cc31bf2399f3e336d39a4a295ebb807dcb..581fcc51c1b070c5a0c20c88b10230ac126b1966 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartitionView.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartitionView.java @@ -42,7 +42,7 @@ public interface ResultSubpartitionView { * after it has been consumed. */ @Nullable - BufferAndBacklog getNextBuffer() throws IOException, InterruptedException; + BufferAndBacklog getNextBuffer() throws IOException; void notifyDataAvailable(); @@ -50,14 +50,11 @@ public interface ResultSubpartitionView { boolean isReleased(); - Throwable getFailureCause(); + void resumeConsumption(); - /** - * Returns whether the next buffer is an event or not. - */ - boolean nextBufferIsEvent(); + Throwable getFailureCause(); - boolean isAvailable(); + boolean isAvailable(int numCreditsAvailable); int unsynchronizedGetNumberOfQueuedBuffers(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorClusterPartitionReleaser.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorClusterPartitionReleaser.java new file mode 100644 index 0000000000000000000000000000000000000000..2f9dd0f44aef5446f90727a48b08a7850813532a --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorClusterPartitionReleaser.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.io.network.partition; + +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; + +import java.util.Set; + +/** + * Interface for releasing cluster partitions on a task executor. + */ +@FunctionalInterface +public interface TaskExecutorClusterPartitionReleaser { + void releaseClusterPartitions(ResourceID taskExecutorId, Set dataSetsToRelease); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTracker.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTracker.java index a586d81f365fe623204ff9e028c924392bb40ad9..18c60cfd6a5ab5819760dd7bcd62f4deb9664b8a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTracker.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTracker.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.io.network.partition; import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.taskexecutor.partition.ClusterPartitionReport; import java.util.Collection; @@ -50,6 +51,13 @@ public interface TaskExecutorPartitionTracker extends PartitionTracker partitionsToPromote); + /** + * Releases partitions associated with the given datasets and stops tracking of partitions that were released. + * + * @param dataSetsToRelease data sets to release + */ + void stopTrackingAndReleaseClusterPartitions(Collection dataSetsToRelease); + /** * Releases and stops tracking all partitions. */ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTrackerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTrackerImpl.java index 2b6fc6920d74744be6cf278863b08ff959b2b5fa..d8780cf1720abcb737248edf13d4022ca454439c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTrackerImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTrackerImpl.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.io.network.partition; import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.shuffle.ShuffleEnvironment; import org.apache.flink.runtime.taskexecutor.partition.ClusterPartitionReport; import org.apache.flink.util.CollectionUtil; @@ -25,6 +26,7 @@ import org.apache.flink.util.Preconditions; import java.util.Collection; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -35,7 +37,7 @@ import java.util.stream.Collectors; */ public class TaskExecutorPartitionTrackerImpl extends AbstractPartitionTracker implements TaskExecutorPartitionTracker { - private final Map> clusterPartitions = new HashMap<>(); + private final Map clusterPartitions = new HashMap<>(); private final ShuffleEnvironment shuffleEnvironment; public TaskExecutorPartitionTrackerImpl(ShuffleEnvironment shuffleEnvironment) { @@ -52,6 +54,10 @@ public class TaskExecutorPartitionTrackerImpl extends AbstractPartitionTracker partitionsToRelease) { + if (partitionsToRelease.isEmpty()) { + return; + } + stopTrackingPartitions(partitionsToRelease); shuffleEnvironment.releasePartitionsLocally(partitionsToRelease); } @@ -66,41 +72,67 @@ public class TaskExecutorPartitionTrackerImpl extends AbstractPartitionTracker partitionsToPromote) { + if (partitionsToPromote.isEmpty()) { + return; + } + final Collection> partitionTrackerEntries = stopTrackingPartitions(partitionsToPromote); - final Map> newClusterPartitions = partitionTrackerEntries.stream() - .collect(Collectors.groupingBy( - PartitionTrackerEntry::getMetaInfo, - Collectors.mapping(PartitionTrackerEntry::getResultPartitionId, Collectors.toSet()))); - - newClusterPartitions.forEach( - (dataSetMetaInfo, newPartitionEntries) -> clusterPartitions.compute(dataSetMetaInfo, (ignored, existingPartitions) -> { - if (existingPartitions == null) { - return newPartitionEntries; - } else { - existingPartitions.addAll(newPartitionEntries); - return existingPartitions; - } - })); + for (PartitionTrackerEntry partitionTrackerEntry : partitionTrackerEntries) { + final TaskExecutorPartitionInfo dataSetMetaInfo = partitionTrackerEntry.getMetaInfo(); + final DataSetEntry dataSetEntry = clusterPartitions.computeIfAbsent( + dataSetMetaInfo.getIntermediateDataSetId(), + ignored -> new DataSetEntry(dataSetMetaInfo.getNumberOfPartitions())); + dataSetEntry.addPartition(partitionTrackerEntry.getResultPartitionId()); + } + } + + @Override + public void stopTrackingAndReleaseClusterPartitions(Collection dataSetsToRelease) { + for (IntermediateDataSetID dataSetID : dataSetsToRelease) { + final DataSetEntry dataSetEntry = clusterPartitions.remove(dataSetID); + final Set partitionIds = dataSetEntry.getPartitionIds(); + shuffleEnvironment.releasePartitionsLocally(partitionIds); + } } @Override public void stopTrackingAndReleaseAllClusterPartitions() { - clusterPartitions.values().forEach(shuffleEnvironment::releasePartitionsLocally); + clusterPartitions.values().stream().map(DataSetEntry::getPartitionIds).forEach(shuffleEnvironment::releasePartitionsLocally); clusterPartitions.clear(); } @Override public ClusterPartitionReport createClusterPartitionReport() { - List collect = clusterPartitions.entrySet().stream().map(entry -> { - TaskExecutorPartitionInfo dataSetMetaInfo = entry.getKey(); - Set partitionsIds = entry.getValue(); - return new ClusterPartitionReport.ClusterPartitionReportEntry( - dataSetMetaInfo.getIntermediateDataSetId(), - partitionsIds, - dataSetMetaInfo.getNumberOfPartitions()); - }).collect(Collectors.toList()); + List reportEntries = clusterPartitions.entrySet().stream().map(entry -> + new ClusterPartitionReport.ClusterPartitionReportEntry( + entry.getKey(), + entry.getValue().getPartitionIds(), + entry.getValue().getTotalNumberOfPartitions())) + .collect(Collectors.toList()); + + return new ClusterPartitionReport(reportEntries); + } + + private static class DataSetEntry { + + private final Set partitionIds = new HashSet<>(); + private final int totalNumberOfPartitions; + + private DataSetEntry(int totalNumberOfPartitions) { + this.totalNumberOfPartitions = totalNumberOfPartitions; + } + + void addPartition(ResultPartitionID resultPartitionId) { + partitionIds.add(resultPartitionId); + } + + public Set getPartitionIds() { + return partitionIds; + } - return new ClusterPartitionReport(collect); + public int getTotalNumberOfPartitions() { + return totalNumberOfPartitions; + } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/BufferOrEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/BufferOrEvent.java index f3ba122c5a610135d009fd1a21583906fa4f4a74..1ec864dfee5519c8a071fa71814d94e7353415c8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/BufferOrEvent.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/BufferOrEvent.java @@ -97,7 +97,7 @@ public class BufferOrEvent { this.channelIndex = channelIndex; } - boolean moreAvailable() { + public boolean moreAvailable() { return moreAvailable; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/IndexedInputGate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/IndexedInputGate.java new file mode 100644 index 0000000000000000000000000000000000000000..85c081c9a729363c84d21234aea01b9695f05489 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/IndexedInputGate.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.io.network.partition.consumer; + +/** + * An {@link InputGate} with a specific index. + */ +public abstract class IndexedInputGate extends InputGate { + /** + * Returns the index of this input gate. Only supported on + */ + public abstract int getGateIndex(); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java index 61703f856ddb6cf1bb205b79c95a69163a48d39d..c02ee70a2933c9a8c3fc2f7cf16367639a1886ff 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java @@ -19,14 +19,22 @@ package org.apache.flink.runtime.io.network.partition.consumer; import org.apache.flink.metrics.Counter; +import org.apache.flink.runtime.checkpoint.channel.InputChannelInfo; +import org.apache.flink.runtime.event.AbstractEvent; import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.execution.CancelTaskException; +import org.apache.flink.runtime.io.network.api.CheckpointBarrier; +import org.apache.flink.runtime.io.network.api.serialization.EventSerializer; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.partition.PartitionException; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.ResultSubpartitionView; +import javax.annotation.Nullable; + import java.io.IOException; +import java.util.Collections; +import java.util.List; import java.util.Optional; import java.util.concurrent.atomic.AtomicReference; @@ -45,7 +53,8 @@ import static org.apache.flink.util.Preconditions.checkNotNull; */ public abstract class InputChannel { - protected final int channelIndex; + /** The info of the input channel to identify it globally within a task. */ + protected final InputChannelInfo channelInfo; protected final ResultPartitionID partitionId; @@ -87,7 +96,7 @@ public abstract class InputChannel { checkArgument(initial >= 0 && initial <= max); this.inputGate = checkNotNull(inputGate); - this.channelIndex = channelIndex; + this.channelInfo = new InputChannelInfo(inputGate.getGateIndex(), channelIndex); this.partitionId = checkNotNull(partitionId); this.initialBackoff = initial; @@ -102,14 +111,31 @@ public abstract class InputChannel { // Properties // ------------------------------------------------------------------------ - int getChannelIndex() { - return channelIndex; + /** + * Returns the index of this channel within its {@link SingleInputGate}. + */ + public int getChannelIndex() { + return channelInfo.getInputChannelIdx(); + } + + /** + * Returns the info of this channel, which uniquely identifies the channel in respect to its operator instance. + */ + public InputChannelInfo getChannelInfo() { + return channelInfo; } public ResultPartitionID getPartitionId() { return partitionId; } + /** + * After sending a {@link org.apache.flink.runtime.io.network.api.CheckpointBarrier} of + * exactly-once mode, the upstream will be blocked and become unavailable. This method + * tries to unblock the corresponding upstream and resume data consumption. + */ + public abstract void resumeConsumption(); + /** * Notifies the owning {@link SingleInputGate} that this channel became non-empty. * @@ -126,6 +152,10 @@ public abstract class InputChannel { inputGate.notifyChannelNonEmpty(this); } + public List requestInflightBuffers(long checkpointId) throws IOException { + return Collections.emptyList(); + } + // ------------------------------------------------------------------------ // Consume // ------------------------------------------------------------------------ @@ -256,6 +286,23 @@ public abstract class InputChannel { // ------------------------------------------------------------------------ + /** + * Parses the buffer as an event and returns the {@link CheckpointBarrier} if the event is indeed a barrier or + * returns null in all other cases. + */ + @Nullable + protected CheckpointBarrier parseCheckpointBarrierOrNull(Buffer buffer) throws IOException { + if (buffer.isBuffer()) { + return null; + } + + AbstractEvent event = EventSerializer.fromBuffer(buffer, getClass().getClassLoader()); + // reset the buffer because it would be deserialized again in SingleInputGate while getting next buffer. + // we can further improve to avoid double deserialization in the future. + buffer.setReaderIndex(0); + return event.getClass() == CheckpointBarrier.class ? (CheckpointBarrier) event : null; + } + /** * A combination of a {@link Buffer} and a flag indicating availability of further buffers, * and the backlog length indicating how many non-event buffers are available in the diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputGate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputGate.java index 467dffe9d519d137db73de90377bb5bbb118cb45..15acfaceeb3e6394acfffb74099a2aea9d6ea392 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputGate.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputGate.java @@ -20,6 +20,7 @@ package org.apache.flink.runtime.io.network.partition.consumer; import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.io.PullingAsyncDataInput; +import org.apache.flink.runtime.io.network.buffer.BufferReceivedListener; import java.io.IOException; import java.util.Optional; @@ -107,6 +108,13 @@ public abstract class InputGate implements PullingAsyncDataInput, return availabilityHelper.getAvailableFuture(); } + public abstract void resumeConsumption(int channelIndex); + + /** + * Returns the channel of this gate. + */ + public abstract InputChannel getChannel(int channelIndex); + /** * Simple pojo for INPUT, DATA and moreAvailable. */ @@ -126,4 +134,6 @@ public abstract class InputGate implements PullingAsyncDataInput, * Setup gate, potentially heavy-weight, blocking operation comparing to just creation. */ public abstract void setup() throws IOException, InterruptedException; + + public abstract void registerBufferReceivedListener(BufferReceivedListener listener); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannel.java index 62a8c7c1c114db69da77f30d1fa4c0757836c8ff..78dc5b092b1803ada037ac73cc8f8fa22d686b16 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannel.java @@ -21,6 +21,9 @@ package org.apache.flink.runtime.io.network.partition.consumer; import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.execution.CancelTaskException; import org.apache.flink.runtime.io.network.TaskEventPublisher; +import org.apache.flink.runtime.io.network.api.CheckpointBarrier; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.BufferConsumer; import org.apache.flink.runtime.io.network.metrics.InputChannelMetrics; import org.apache.flink.runtime.io.network.partition.BufferAvailabilityListener; import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException; @@ -95,7 +98,7 @@ public class LocalInputChannel extends InputChannel implements BufferAvailabilit // ------------------------------------------------------------------------ @Override - protected void requestSubpartition(int subpartitionIndex) throws IOException, InterruptedException { + protected void requestSubpartition(int subpartitionIndex) throws IOException { boolean retriggerRequest = false; @@ -195,7 +198,7 @@ public class LocalInputChannel extends InputChannel implements BufferAvailabilit numBytesIn.inc(next.buffer().getSize()); numBuffersIn.inc(); - return Optional.of(new BufferAndAvailability(next.buffer(), next.isMoreAvailable(), next.buffersInBacklog())); + return Optional.of(new BufferAndAvailability(next.buffer(), next.isDataAvailable(), next.buffersInBacklog())); } @Override @@ -214,6 +217,17 @@ public class LocalInputChannel extends InputChannel implements BufferAvailabilit } } + @Override + public void resumeConsumption() { + checkState(!isReleased, "Channel released."); + + subpartitionView.resumeConsumption(); + + if (subpartitionView.isAvailable(Integer.MAX_VALUE)) { + notifyChannelNonEmpty(); + } + } + // ------------------------------------------------------------------------ // Task events // ------------------------------------------------------------------------ @@ -268,4 +282,25 @@ public class LocalInputChannel extends InputChannel implements BufferAvailabilit public String toString() { return "LocalInputChannel [" + partitionId + "]"; } + + @Override + public boolean notifyPriorityEvent(BufferConsumer eventBufferConsumer) throws IOException { + if (inputGate.getBufferReceivedListener() == null) { + // in rare cases and very low checkpointing intervals, we may receive the first barrier, before setting + // up CheckpointedInputGate + return false; + } + Buffer buffer = eventBufferConsumer.build(); + try { + CheckpointBarrier event = parseCheckpointBarrierOrNull(buffer); + if (event == null) { + throw new IllegalStateException("Currently only checkpoint barriers are known priority events"); + } + inputGate.getBufferReceivedListener().notifyBarrierReceived(event, channelInfo); + } finally { + buffer.recycleBuffer(); + } + // already processed + return true; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java index 6ee3ffb771bfa04fcac1ff41f648bd3963434e0e..a2ab2a970db2d7adb9ff285cdc0058460e37bdd2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java @@ -25,9 +25,11 @@ import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.io.network.ConnectionID; import org.apache.flink.runtime.io.network.ConnectionManager; import org.apache.flink.runtime.io.network.PartitionRequestClient; +import org.apache.flink.runtime.io.network.api.CheckpointBarrier; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.buffer.BufferListener; import org.apache.flink.runtime.io.network.buffer.BufferProvider; +import org.apache.flink.runtime.io.network.buffer.BufferReceivedListener; import org.apache.flink.runtime.io.network.buffer.BufferRecycler; import org.apache.flink.runtime.io.network.buffer.NetworkBuffer; import org.apache.flink.runtime.io.network.metrics.InputChannelMetrics; @@ -109,6 +111,13 @@ public class RemoteInputChannel extends InputChannel implements BufferRecycler, @Nonnull private final MemorySegmentProvider memorySegmentProvider; + /** The latest already triggered checkpoint id which would be updated during {@link #requestInflightBuffers(long)}.*/ + @GuardedBy("receivedBuffers") + private long lastRequestedCheckpointId = -1; + + /** The current received checkpoint id from the network. */ + private long receivedCheckpointId = -1; + public RemoteInputChannel( SingleInputGate inputGate, int channelIndex, @@ -206,6 +215,28 @@ public class RemoteInputChannel extends InputChannel implements BufferRecycler, return Optional.of(new BufferAndAvailability(next, moreAvailable, getSenderBacklog())); } + @Override + public List requestInflightBuffers(long checkpointId) throws IOException { + synchronized (receivedBuffers) { + checkState(checkpointId > lastRequestedCheckpointId, "Need to request the next checkpointId"); + + final List inflightBuffers = new ArrayList<>(receivedBuffers.size()); + for (Buffer buffer : receivedBuffers) { + CheckpointBarrier checkpointBarrier = parseCheckpointBarrierOrNull(buffer); + if (checkpointBarrier != null && checkpointBarrier.getId() >= checkpointId) { + break; + } + if (buffer.isBuffer()) { + inflightBuffers.add(buffer.retainBuffer()); + } + } + + lastRequestedCheckpointId = checkpointId; + + return inflightBuffers; + } + } + // ------------------------------------------------------------------------ // Task events // ------------------------------------------------------------------------ @@ -397,6 +428,16 @@ public class RemoteInputChannel extends InputChannel implements BufferRecycler, // Nothing to do actually. } + @Override + public void resumeConsumption() { + checkState(!isReleased.get(), "Channel released."); + checkState(partitionRequestClient != null, "Trying to send event to producer before requesting a queue."); + + // notifies the producer that this channel is ready to + // unblock from checkpoint and resume data consumption + partitionRequestClient.resumeConsumption(this); + } + // ------------------------------------------------------------------------ // Network I/O notifications (called by network I/O thread) // ------------------------------------------------------------------------ @@ -513,8 +554,15 @@ public class RemoteInputChannel extends InputChannel implements BufferRecycler, boolean recycleBuffer = true; try { + if (expectedSequenceNumber != sequenceNumber) { + onError(new BufferReorderingException(expectedSequenceNumber, sequenceNumber)); + return; + } final boolean wasEmpty; + final CheckpointBarrier notifyReceivedBarrier; + final Buffer notifyReceivedBuffer; + final BufferReceivedListener listener = inputGate.getBufferReceivedListener(); synchronized (receivedBuffers) { // Similar to notifyBufferAvailable(), make sure that we never add a buffer // after releaseAllResources() released all buffers from receivedBuffers @@ -523,15 +571,17 @@ public class RemoteInputChannel extends InputChannel implements BufferRecycler, return; } - if (expectedSequenceNumber != sequenceNumber) { - onError(new BufferReorderingException(expectedSequenceNumber, sequenceNumber)); - return; - } - wasEmpty = receivedBuffers.isEmpty(); receivedBuffers.add(buffer); - recycleBuffer = false; + + if (listener != null && buffer.isBuffer() && receivedCheckpointId < lastRequestedCheckpointId) { + notifyReceivedBuffer = buffer.retainBuffer(); + } else { + notifyReceivedBuffer = null; + } + notifyReceivedBarrier = listener != null ? parseCheckpointBarrierOrNull(buffer) : null; } + recycleBuffer = false; ++expectedSequenceNumber; @@ -542,6 +592,13 @@ public class RemoteInputChannel extends InputChannel implements BufferRecycler, if (backlog >= 0) { onSenderBacklog(backlog); } + + if (notifyReceivedBarrier != null) { + receivedCheckpointId = notifyReceivedBarrier.getId(); + listener.notifyBarrierReceived(notifyReceivedBarrier, channelInfo); + } else if (notifyReceivedBuffer != null) { + listener.notifyBufferReceived(notifyReceivedBuffer, channelInfo); + } } finally { if (recycleBuffer) { buffer.recycleBuffer(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java index f03e347078e131ded9ed77a994bc20f0d337a330..1f74f09d435f0c80ebe7cc441ff23784becd7008 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java @@ -29,6 +29,7 @@ import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.buffer.BufferDecompressor; import org.apache.flink.runtime.io.network.buffer.BufferPool; import org.apache.flink.runtime.io.network.buffer.BufferProvider; +import org.apache.flink.runtime.io.network.buffer.BufferReceivedListener; import org.apache.flink.runtime.io.network.partition.PartitionProducerStateProvider; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; @@ -37,12 +38,14 @@ import org.apache.flink.runtime.jobgraph.DistributionPattern; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; import org.apache.flink.runtime.shuffle.NettyShuffleDescriptor; +import org.apache.flink.util.Preconditions; import org.apache.flink.util.function.SupplierWithException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import javax.annotation.Nullable; +import javax.annotation.concurrent.GuardedBy; import java.io.IOException; import java.util.ArrayDeque; @@ -99,7 +102,7 @@ import static org.apache.flink.util.Preconditions.checkState; * in two partitions (Partition 1 and 2). Each of these partitions is further partitioned into two * subpartitions -- one for each parallel reduce subtask. */ -public class SingleInputGate extends InputGate { +public class SingleInputGate extends IndexedInputGate { private static final Logger LOG = LoggerFactory.getLogger(SingleInputGate.class); @@ -109,6 +112,8 @@ public class SingleInputGate extends InputGate { /** The name of the owning task, for logging purposes. */ private final String owningTaskName; + private final int gateIndex; + /** * The ID of the consumed intermediate result. Each input gate consumes partitions of the * intermediate result specified by this ID. This ID also identifies the input gate at the @@ -134,6 +139,9 @@ public class SingleInputGate extends InputGate { */ private final Map inputChannels; + @GuardedBy("requestLock") + private final InputChannel[] channels; + /** Channels, which notified this input gate about available data. */ private final ArrayDeque inputChannelsWithData = new ArrayDeque<>(); @@ -170,11 +178,15 @@ public class SingleInputGate extends InputGate { private final CompletableFuture closeFuture; + @Nullable + private volatile BufferReceivedListener bufferReceivedListener; + @Nullable private final BufferDecompressor bufferDecompressor; public SingleInputGate( String owningTaskName, + int gateIndex, IntermediateDataSetID consumedResultId, final ResultPartitionType consumedPartitionType, int consumedSubpartitionIndex, @@ -184,6 +196,8 @@ public class SingleInputGate extends InputGate { @Nullable BufferDecompressor bufferDecompressor) { this.owningTaskName = checkNotNull(owningTaskName); + Preconditions.checkArgument(0 <= gateIndex, "The gate index must be positive."); + this.gateIndex = gateIndex; this.consumedResultId = checkNotNull(consumedResultId); this.consumedPartitionType = checkNotNull(consumedPartitionType); @@ -196,6 +210,7 @@ public class SingleInputGate extends InputGate { this.numberOfInputChannels = numberOfInputChannels; this.inputChannels = new HashMap<>(numberOfInputChannels); + this.channels = new InputChannel[numberOfInputChannels]; this.channelsWithEndOfPartitionEvents = new BitSet(numberOfInputChannels); this.enqueuedInputChannelsWithData = new BitSet(numberOfInputChannels); @@ -245,6 +260,12 @@ public class SingleInputGate extends InputGate { } } + @Override + public void registerBufferReceivedListener(BufferReceivedListener bufferReceivedListener) { + checkState(this.bufferReceivedListener == null, "Trying to overwrite the buffer received listener"); + this.bufferReceivedListener = checkNotNull(bufferReceivedListener); + } + // ------------------------------------------------------------------------ // Properties // ------------------------------------------------------------------------ @@ -254,6 +275,16 @@ public class SingleInputGate extends InputGate { return numberOfInputChannels; } + @Override + public int getGateIndex() { + return gateIndex; + } + + @Nullable + BufferReceivedListener getBufferReceivedListener() { + return bufferReceivedListener; + } + /** * Returns the type of this input channel's consumed result partition. * @@ -293,6 +324,11 @@ public class SingleInputGate extends InputGate { return closeFuture; } + @Override + public InputChannel getChannel(int channelIndex) { + return channels[channelIndex]; + } + // ------------------------------------------------------------------------ // Setup/Life-cycle // ------------------------------------------------------------------------ @@ -318,12 +354,20 @@ public class SingleInputGate extends InputGate { } } - public void setInputChannel(IntermediateResultPartitionID partitionId, InputChannel inputChannel) { + public void setInputChannels(InputChannel... channels) { + if (channels.length != numberOfInputChannels) { + throw new IllegalArgumentException("Expected " + numberOfInputChannels + " channels, " + + "but got " + channels.length); + } synchronized (requestLock) { - if (inputChannels.put(checkNotNull(partitionId), checkNotNull(inputChannel)) == null + System.arraycopy(channels, 0, this.channels, 0, numberOfInputChannels); + for (InputChannel inputChannel : channels) { + IntermediateResultPartitionID partitionId = inputChannel.getPartitionId().getPartitionId(); + if (inputChannels.put(partitionId, inputChannel) == null && inputChannel instanceof UnknownInputChannel) { - numberOfUninitializedChannels++; + numberOfUninitializedChannels++; + } } } } @@ -356,6 +400,7 @@ public class SingleInputGate extends InputGate { LOG.debug("{}: Updated unknown input channel to {}.", owningTaskName, newChannel); inputChannels.put(partitionId, newChannel); + channels[current.getChannelIndex()] = newChannel; if (requestedPartitionsFlag) { newChannel.requestSubpartition(consumedSubpartitionIndex); @@ -604,6 +649,15 @@ public class SingleInputGate extends InputGate { } } + @Override + public void resumeConsumption(int channelIndex) { + // BEWARE: consumption resumption only happens for streaming jobs in which all slots + // are allocated together so there should be no UnknownInputChannel. As a result, it + // is safe to not synchronize the requestLock here. We will refactor the code to not + // rely on this assumption in the future. + channels[channelIndex].resumeConsumption(); + } + // ------------------------------------------------------------------------ // Channel notifications // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateFactory.java index dca505d727d8b7222637329d94553db21c5b248f..1428bbf7739bb6170b201bad6d97b9f76842976b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateFactory.java @@ -108,6 +108,7 @@ public class SingleInputGateFactory { */ public SingleInputGate create( @Nonnull String owningTaskName, + int gateIndex, @Nonnull InputGateDeploymentDescriptor igdd, @Nonnull PartitionProducerStateProvider partitionProducerStateProvider, @Nonnull InputChannelMetrics metrics) { @@ -125,6 +126,7 @@ public class SingleInputGateFactory { SingleInputGate inputGate = new SingleInputGate( owningTaskName, + gateIndex, igdd.getConsumedResultId(), igdd.getConsumedPartitionType(), igdd.getConsumedSubpartitionIndex(), @@ -156,9 +158,8 @@ public class SingleInputGateFactory { shuffleDescriptors[i], channelStatistics, metrics); - ResultPartitionID resultPartitionID = inputChannels[i].getPartitionId(); - inputGate.setInputChannel(resultPartitionID.getPartitionId(), inputChannels[i]); } + inputGate.setInputChannels(inputChannels); LOG.debug("{}: Created {} input channels ({}).", owningTaskName, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java index 02a32e8bb6472ce0eaf8b1b40d411f1aa8299d58..115511215a709047fea665f02c4f25aa325f4e05 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java @@ -20,17 +20,19 @@ package org.apache.flink.runtime.io.network.partition.consumer; import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent; +import org.apache.flink.runtime.io.network.buffer.BufferReceivedListener; -import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; import org.apache.flink.shaded.guava18.com.google.common.collect.Sets; import java.io.IOException; +import java.util.Arrays; import java.util.Iterator; import java.util.LinkedHashSet; -import java.util.Map; import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; +import java.util.stream.IntStream; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -69,45 +71,49 @@ public class UnionInputGate extends InputGate { /** The input gates to union. */ private final InputGate[] inputGates; - private final Set inputGatesWithRemainingData; + private final Set inputGatesWithRemainingData; /** * Gates, which notified this input gate about available data. We are using it as a FIFO * queue of {@link InputGate}s to avoid starvation and provide some basic fairness. */ - private final LinkedHashSet inputGatesWithData = new LinkedHashSet<>(); + private final LinkedHashSet inputGatesWithData = new LinkedHashSet<>(); - /** The total number of input channels across all unioned input gates. */ - private final int totalNumberOfInputChannels; + /** Input channels across all unioned input gates. */ + private final InputChannel[] inputChannels; /** - * A mapping from input gate to (logical) channel index offset. Valid channel indexes go from 0 + * A mapping from input gate index to (logical) channel index offset. Valid channel indexes go from 0 * (inclusive) to the total number of input channels (exclusive). */ - private final Map inputGateToIndexOffsetMap; + private final int[] inputGateChannelIndexOffsets; - public UnionInputGate(InputGate... inputGates) { + public UnionInputGate(IndexedInputGate... inputGates) { this.inputGates = checkNotNull(inputGates); checkArgument(inputGates.length > 1, "Union input gate should union at least two input gates."); - this.inputGateToIndexOffsetMap = Maps.newHashMapWithExpectedSize(inputGates.length); + if (Arrays.stream(inputGates).map(IndexedInputGate::getGateIndex).distinct().count() != inputGates.length) { + throw new IllegalArgumentException("Union of two input gates with the same gate index. Given indices: " + + Arrays.stream(inputGates).map(IndexedInputGate::getGateIndex).collect(Collectors.toList())); + } + this.inputGatesWithRemainingData = Sets.newHashSetWithExpectedSize(inputGates.length); + final int maxGateIndex = Arrays.stream(inputGates).mapToInt(IndexedInputGate::getGateIndex).max().orElse(0); + inputGateChannelIndexOffsets = new int[maxGateIndex + 1]; int currentNumberOfInputChannels = 0; + for (final IndexedInputGate inputGate : inputGates) { + inputGateChannelIndexOffsets[inputGate.getGateIndex()] = currentNumberOfInputChannels; + currentNumberOfInputChannels += inputGate.getNumberOfInputChannels(); + } + inputChannels = Arrays.stream(inputGates) + .flatMap(gate -> IntStream.range(0, gate.getNumberOfInputChannels()).mapToObj(gate::getChannel)) + .toArray(InputChannel[]::new); synchronized (inputGatesWithData) { - for (InputGate inputGate : inputGates) { - if (inputGate instanceof UnionInputGate) { - // if we want to add support for this, we need to implement pollNext() - throw new UnsupportedOperationException("Cannot union a union of input gates."); - } - - // The offset to use for buffer or event instances received from this input gate. - inputGateToIndexOffsetMap.put(checkNotNull(inputGate), currentNumberOfInputChannels); + for (IndexedInputGate inputGate : inputGates) { inputGatesWithRemainingData.add(inputGate); - currentNumberOfInputChannels += inputGate.getNumberOfInputChannels(); - CompletableFuture available = inputGate.getAvailableFuture(); if (available.isDone()) { @@ -121,8 +127,6 @@ public class UnionInputGate extends InputGate { availabilityHelper.resetAvailable(); } } - - this.totalNumberOfInputChannels = currentNumberOfInputChannels; } /** @@ -130,7 +134,12 @@ public class UnionInputGate extends InputGate { */ @Override public int getNumberOfInputChannels() { - return totalNumberOfInputChannels; + return inputChannels.length; + } + + @Override + public InputChannel getChannel(int channelIndex) { + return inputChannels[channelIndex]; } @Override @@ -153,12 +162,12 @@ public class UnionInputGate extends InputGate { return Optional.empty(); } - Optional> next = waitAndGetNextData(blocking); + Optional> next = waitAndGetNextData(blocking); if (!next.isPresent()) { return Optional.empty(); } - InputWithData inputWithData = next.get(); + InputWithData inputWithData = next.get(); handleEndOfPartitionEvent(inputWithData.data, inputWithData.input); return Optional.of(adjustForUnionInputGate( @@ -167,10 +176,10 @@ public class UnionInputGate extends InputGate { inputWithData.moreAvailable)); } - private Optional> waitAndGetNextData(boolean blocking) + private Optional> waitAndGetNextData(boolean blocking) throws IOException, InterruptedException { while (true) { - Optional inputGate = getInputGate(blocking); + Optional inputGate = getInputGate(blocking); if (!inputGate.isPresent()) { return Optional.empty(); } @@ -204,10 +213,10 @@ public class UnionInputGate extends InputGate { private BufferOrEvent adjustForUnionInputGate( BufferOrEvent bufferOrEvent, - InputGate inputGate, + IndexedInputGate inputGate, boolean moreInputGatesAvailable) { // Set the channel index to identify the input channel (across all unioned input gates) - final int channelIndexOffset = inputGateToIndexOffsetMap.get(inputGate); + final int channelIndexOffset = inputGateChannelIndexOffsets[inputGate.getGateIndex()]; bufferOrEvent.setChannelIndex(channelIndexOffset + bufferOrEvent.getChannelIndex()); bufferOrEvent.setMoreAvailable(bufferOrEvent.moreAvailable() || moreInputGatesAvailable); @@ -246,6 +255,14 @@ public class UnionInputGate extends InputGate { } } + @Override + public void resumeConsumption(int channelIndex) { + // BEWARE: consumption resumption only happens for streaming jobs in which all + // slots are allocated together so there should be no UnknownInputChannel. We + // will refactor the code to not rely on this assumption in the future. + inputChannels[channelIndex].resumeConsumption(); + } + @Override public void setup() { } @@ -254,7 +271,7 @@ public class UnionInputGate extends InputGate { public void close() throws IOException { } - private void queueInputGate(InputGate inputGate) { + private void queueInputGate(IndexedInputGate inputGate) { checkNotNull(inputGate); CompletableFuture toNotify = null; @@ -279,7 +296,7 @@ public class UnionInputGate extends InputGate { } } - private Optional getInputGate(boolean blocking) throws InterruptedException { + private Optional getInputGate(boolean blocking) throws InterruptedException { synchronized (inputGatesWithData) { while (inputGatesWithData.size() == 0) { if (blocking) { @@ -290,11 +307,18 @@ public class UnionInputGate extends InputGate { } } - Iterator inputGateIterator = inputGatesWithData.iterator(); - InputGate inputGate = inputGateIterator.next(); + Iterator inputGateIterator = inputGatesWithData.iterator(); + IndexedInputGate inputGate = inputGateIterator.next(); inputGateIterator.remove(); return Optional.of(inputGate); } } + + @Override + public void registerBufferReceivedListener(BufferReceivedListener listener) { + for (InputGate inputGate : inputGates) { + inputGate.registerBufferReceivedListener(listener); + } + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnknownInputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnknownInputChannel.java index 22d7176ea9a1432ef97dfb7a2ac732ee86da298b..f31cc1c3f7631d7cf5383e70c2a0b892df32c9db 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnknownInputChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnknownInputChannel.java @@ -79,6 +79,11 @@ class UnknownInputChannel extends InputChannel { this.memorySegmentProvider = memorySegmentProvider; } + @Override + public void resumeConsumption() { + throw new UnsupportedOperationException("UnknownInputChannel should never be blocked."); + } + @Override public void requestSubpartition(int subpartitionIndex) throws IOException { // Nothing to do here @@ -122,11 +127,11 @@ class UnknownInputChannel extends InputChannel { // ------------------------------------------------------------------------ public RemoteInputChannel toRemoteInputChannel(ConnectionID producerAddress) { - return new RemoteInputChannel(inputGate, channelIndex, partitionId, checkNotNull(producerAddress), + return new RemoteInputChannel(inputGate, getChannelIndex(), partitionId, checkNotNull(producerAddress), connectionManager, initialBackoff, maxBackoff, metrics, memorySegmentProvider); } public LocalInputChannel toLocalInputChannel() { - return new LocalInputChannel(inputGate, channelIndex, partitionId, partitionManager, taskEventPublisher, initialBackoff, maxBackoff, metrics); + return new LocalInputChannel(inputGate, getChannelIndex(), partitionId, partitionManager, taskEventPublisher, initialBackoff, maxBackoff, metrics); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateDataSetID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateDataSetID.java index c22ca1ee480746c455b4ca4f002850c479d2148d..b34e3f3915c0954e35fab1daadfe7bd8b0c6e581 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateDataSetID.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateDataSetID.java @@ -21,6 +21,8 @@ package org.apache.flink.runtime.jobgraph; import org.apache.flink.runtime.topology.ResultID; import org.apache.flink.util.AbstractID; +import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf; + import java.util.UUID; /** @@ -54,4 +56,19 @@ public class IntermediateDataSetID extends AbstractID implements ResultID { public IntermediateDataSetID(UUID from) { super(from.getLeastSignificantBits(), from.getMostSignificantBits()); } + + private IntermediateDataSetID(long lower, long upper) { + super(lower, upper); + } + + public void writeTo(ByteBuf buf) { + buf.writeLong(lowerPart); + buf.writeLong(upperPart); + } + + public static IntermediateDataSetID fromByteBuf(ByteBuf buf) { + final long lower = buf.readLong(); + final long upper = buf.readLong(); + return new IntermediateDataSetID(lower, upper); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateResultPartitionID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateResultPartitionID.java index 803c0003761b76cd439bb99d917ed6e559506960..f43a5b5d53f4c21abb5b82b3b0bac43d6189bf8d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateResultPartitionID.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateResultPartitionID.java @@ -18,38 +18,70 @@ package org.apache.flink.runtime.jobgraph; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.runtime.executiongraph.IntermediateResultPartition; import org.apache.flink.runtime.topology.ResultID; -import org.apache.flink.util.AbstractID; import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf; /** * Id identifying {@link IntermediateResultPartition}. */ -public class IntermediateResultPartitionID extends AbstractID implements ResultID { +public class IntermediateResultPartitionID implements ResultID { private static final long serialVersionUID = 1L; + private final IntermediateDataSetID intermediateDataSetID; + private final int partitionNum; + /** - * Creates an new random intermediate result partition ID. + * Creates an new random intermediate result partition ID for testing. */ + @VisibleForTesting public IntermediateResultPartitionID() { - super(); + this.partitionNum = -1; + this.intermediateDataSetID = new IntermediateDataSetID(); } - public IntermediateResultPartitionID(long lowerPart, long upperPart) { - super(lowerPart, upperPart); + /** + * Creates an new intermediate result partition ID with {@link IntermediateDataSetID} and the partitionNum. + */ + public IntermediateResultPartitionID(IntermediateDataSetID intermediateDataSetID, int partitionNum) { + this.intermediateDataSetID = intermediateDataSetID; + this.partitionNum = partitionNum; } public void writeTo(ByteBuf buf) { - buf.writeLong(this.lowerPart); - buf.writeLong(this.upperPart); + intermediateDataSetID.writeTo(buf); + buf.writeInt(partitionNum); } public static IntermediateResultPartitionID fromByteBuf(ByteBuf buf) { - long lower = buf.readLong(); - long upper = buf.readLong(); - return new IntermediateResultPartitionID(lower, upper); + final IntermediateDataSetID intermediateDataSetID = IntermediateDataSetID.fromByteBuf(buf); + final int partitionNum = buf.readInt(); + return new IntermediateResultPartitionID(intermediateDataSetID, partitionNum); + } + + @Override + public boolean equals(Object obj) { + if (obj == this) { + return true; + } else if (obj != null && obj.getClass() == getClass()) { + IntermediateResultPartitionID that = (IntermediateResultPartitionID) obj; + return that.intermediateDataSetID.equals(this.intermediateDataSetID) + && that.partitionNum == this.partitionNum; + } else { + return false; + } + } + + @Override + public int hashCode() { + return this.intermediateDataSetID.hashCode() ^ this.partitionNum; + } + + @Override + public String toString() { + return intermediateDataSetID.toString() + "#" + partitionNum; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraphUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraphUtils.java new file mode 100644 index 0000000000000000000000000000000000000000..ff786dca4c1c6e6be0b32902ce0bbb987cc8fc6a --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraphUtils.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.jobgraph; + +import org.apache.flink.api.common.cache.DistributedCache; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.util.FileUtils; +import org.apache.flink.util.FlinkRuntimeException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.file.Files; +import java.util.Collection; + +/** + * Utilities for generating {@link JobGraph}. + */ +public enum JobGraphUtils { + ; + + private static final Logger LOG = LoggerFactory.getLogger(JobGraphUtils.class); + + public static void addUserArtifactEntries(Collection> userArtifacts, JobGraph jobGraph) { + if (userArtifacts != null && !userArtifacts.isEmpty()) { + try { + java.nio.file.Path tmpDir = Files.createTempDirectory("flink-distributed-cache-" + jobGraph.getJobID()); + for (Tuple2 originalEntry : userArtifacts) { + Path filePath = new Path(originalEntry.f1.filePath); + boolean isLocalDir = false; + try { + FileSystem sourceFs = filePath.getFileSystem(); + isLocalDir = !sourceFs.isDistributedFS() && sourceFs.getFileStatus(filePath).isDir(); + } catch (IOException ioe) { + LOG.warn("Could not determine whether {} denotes a local path.", filePath, ioe); + } + // zip local directories because we only support file uploads + DistributedCache.DistributedCacheEntry entry; + if (isLocalDir) { + Path zip = FileUtils.compressDirectory(filePath, new Path(tmpDir.toString(), filePath.getName() + ".zip")); + entry = new DistributedCache.DistributedCacheEntry(zip.toString(), originalEntry.f1.isExecutable, true); + } else { + entry = new DistributedCache.DistributedCacheEntry(filePath.toString(), originalEntry.f1.isExecutable, false); + } + jobGraph.addUserArtifact(originalEntry.f0, entry); + } + } catch (IOException ioe) { + throw new FlinkRuntimeException("Could not compress distributed-cache artifacts.", ioe); + } + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/AbstractInvokable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/AbstractInvokable.java index 5770031fab79f11e7229528334d9eceabb902de8..9cfbcf180f63e55b6eae9e08fe3a992ba1655a74 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/AbstractInvokable.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/AbstractInvokable.java @@ -28,7 +28,9 @@ import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.operators.coordination.OperatorEvent; import org.apache.flink.util.FlinkException; import org.apache.flink.util.SerializedValue; +import org.apache.flink.util.function.ThrowingRunnable; +import java.io.IOException; import java.util.concurrent.Future; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -138,7 +140,7 @@ public abstract class AbstractInvokable { * * @return The environment of this task. */ - public Environment getEnvironment() { + public final Environment getEnvironment() { return this.environment; } @@ -147,7 +149,7 @@ public abstract class AbstractInvokable { * * @return user code class loader of this invokable. */ - public ClassLoader getUserCodeClassLoader() { + public final ClassLoader getUserCodeClassLoader() { return getEnvironment().getUserClassLoader(); } @@ -174,7 +176,7 @@ public abstract class AbstractInvokable { * * @return the task configuration object which was attached to the original {@link org.apache.flink.runtime.jobgraph.JobVertex} */ - public Configuration getTaskConfiguration() { + public final Configuration getTaskConfiguration() { return this.environment.getTaskConfiguration(); } @@ -231,10 +233,24 @@ public abstract class AbstractInvokable { * * @throws Exception Exceptions thrown as the result of triggering a checkpoint are forwarded. */ - public void triggerCheckpointOnBarrier(CheckpointMetaData checkpointMetaData, CheckpointOptions checkpointOptions, CheckpointMetrics checkpointMetrics) throws Exception { + public void triggerCheckpointOnBarrier(CheckpointMetaData checkpointMetaData, CheckpointOptions checkpointOptions, CheckpointMetrics checkpointMetrics) throws IOException { throw new UnsupportedOperationException(String.format("triggerCheckpointOnBarrier not supported by %s", this.getClass().getName())); } + /** + * This method performs some action asynchronously in the task thread. + * + * @param runnable the action to perform + * @param descriptionFormat the optional description for the command that is used for debugging and error-reporting. + * @param descriptionArgs the parameters used to format the final description string. + */ + public void executeInTaskThread( + ThrowingRunnable runnable, + String descriptionFormat, + Object... descriptionArgs) throws E { + throw new UnsupportedOperationException(String.format("runInTaskThread not supported by %s", this.getClass().getName())); + } + /** * Aborts a checkpoint as the result of receiving possibly some checkpoint barriers, * but at least one {@link org.apache.flink.runtime.io.network.api.CancelCheckpointMarker}. @@ -245,7 +261,7 @@ public abstract class AbstractInvokable { * @param checkpointId The ID of the checkpoint to be aborted. * @param cause The reason why the checkpoint was aborted during alignment */ - public void abortCheckpointOnBarrier(long checkpointId, Throwable cause) throws Exception { + public void abortCheckpointOnBarrier(long checkpointId, Throwable cause) throws IOException { throw new UnsupportedOperationException(String.format("abortCheckpointOnBarrier not supported by %s", this.getClass().getName())); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/CheckpointCoordinatorConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/CheckpointCoordinatorConfiguration.java index 36fbe3d74ef0cba5e927a59143542620654836b7..c90aa71b4b3c5d2dd6be9027255ced4495961ed3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/CheckpointCoordinatorConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/CheckpointCoordinatorConfiguration.java @@ -58,6 +58,8 @@ public class CheckpointCoordinatorConfiguration implements Serializable { */ private final boolean isExactlyOnce; + private final boolean isUnalignedCheckpoint; + private final boolean isPreferCheckpointForRecovery; public CheckpointCoordinatorConfiguration( @@ -67,6 +69,7 @@ public class CheckpointCoordinatorConfiguration implements Serializable { int maxConcurrentCheckpoints, CheckpointRetentionPolicy checkpointRetentionPolicy, boolean isExactlyOnce, + boolean isUnalignedCheckpoint, boolean isPreferCheckpointForRecovery, int tolerableCpFailureNumber) { @@ -83,6 +86,7 @@ public class CheckpointCoordinatorConfiguration implements Serializable { this.maxConcurrentCheckpoints = maxConcurrentCheckpoints; this.checkpointRetentionPolicy = Preconditions.checkNotNull(checkpointRetentionPolicy); this.isExactlyOnce = isExactlyOnce; + this.isUnalignedCheckpoint = isUnalignedCheckpoint; this.isPreferCheckpointForRecovery = isPreferCheckpointForRecovery; this.tolerableCheckpointFailureNumber = tolerableCpFailureNumber; } @@ -111,6 +115,10 @@ public class CheckpointCoordinatorConfiguration implements Serializable { return isExactlyOnce; } + public boolean isUnalignedCheckpoint() { + return isUnalignedCheckpoint; + } + public boolean isPreferCheckpointForRecovery() { return isPreferCheckpointForRecovery; } @@ -133,6 +141,7 @@ public class CheckpointCoordinatorConfiguration implements Serializable { minPauseBetweenCheckpoints == that.minPauseBetweenCheckpoints && maxConcurrentCheckpoints == that.maxConcurrentCheckpoints && isExactlyOnce == that.isExactlyOnce && + isUnalignedCheckpoint == that.isUnalignedCheckpoint && checkpointRetentionPolicy == that.checkpointRetentionPolicy && isPreferCheckpointForRecovery == that.isPreferCheckpointForRecovery && tolerableCheckpointFailureNumber == that.tolerableCheckpointFailureNumber; @@ -147,6 +156,7 @@ public class CheckpointCoordinatorConfiguration implements Serializable { maxConcurrentCheckpoints, checkpointRetentionPolicy, isExactlyOnce, + isUnalignedCheckpoint, isPreferCheckpointForRecovery, tolerableCheckpointFailureNumber); } @@ -159,6 +169,9 @@ public class CheckpointCoordinatorConfiguration implements Serializable { ", minPauseBetweenCheckpoints=" + minPauseBetweenCheckpoints + ", maxConcurrentCheckpoints=" + maxConcurrentCheckpoints + ", checkpointRetentionPolicy=" + checkpointRetentionPolicy + + ", isExactlyOnce=" + isExactlyOnce + + ", isUnalignedCheckpoint=" + isUnalignedCheckpoint + + ", isPreferCheckpointForRecovery=" + isPreferCheckpointForRecovery + ", tolerableCheckpointFailureNumber=" + tolerableCheckpointFailureNumber + '}'; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/PipelinedRegionConsumedBlockingPartitions.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/DefaultLogicalPipelinedRegion.java similarity index 50% rename from flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/PipelinedRegionConsumedBlockingPartitions.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/DefaultLogicalPipelinedRegion.java index 5cbce45f0114568b565e3d056291e2bb51e0713d..39d3c0da24e261b087e8c6f663b6bbfa7fd8fc04 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/PipelinedRegionConsumedBlockingPartitions.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/DefaultLogicalPipelinedRegion.java @@ -17,35 +17,38 @@ * under the License. */ -package org.apache.flink.runtime.executiongraph.failover.flip1.partitionrelease; +package org.apache.flink.runtime.jobgraph.topology; -import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; +import org.apache.flink.runtime.jobgraph.JobVertexID; import java.util.Set; +import java.util.stream.Collectors; import static org.apache.flink.util.Preconditions.checkNotNull; /** - * A set of intermediate result partitions that are incident to one {@link PipelinedRegion}. + * Set of {@link LogicalVertex} that are connected through pipelined {@link LogicalResult}. */ -class PipelinedRegionConsumedBlockingPartitions { +public class DefaultLogicalPipelinedRegion { - private final PipelinedRegion pipelinedRegion; + private final Set vertexIDs; - private final Set consumedBlockingPartitions; + public DefaultLogicalPipelinedRegion(final Set> logicalVertices) { + checkNotNull(logicalVertices); - PipelinedRegionConsumedBlockingPartitions( - final PipelinedRegion pipelinedRegion, - final Set consumedBlockingPartitions) { - this.pipelinedRegion = checkNotNull(pipelinedRegion); - this.consumedBlockingPartitions = checkNotNull(consumedBlockingPartitions); + this.vertexIDs = logicalVertices.stream() + .map(LogicalVertex::getId) + .collect(Collectors.toSet()); } - public Set getConsumedBlockingPartitions() { - return consumedBlockingPartitions; + public Set getVertexIDs() { + return vertexIDs; } - public PipelinedRegion getPipelinedRegion() { - return pipelinedRegion; + @Override + public String toString() { + return "DefaultLogicalPipelinedRegion{" + + "vertexIDs=" + vertexIDs + + '}'; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/DefaultLogicalTopology.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/DefaultLogicalTopology.java index 00b8d62e7091a4da443c19c513b4a2232783810d..e701d251f8794f86ecd1ef89526a907b7bcf0c09 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/DefaultLogicalTopology.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/DefaultLogicalTopology.java @@ -102,12 +102,12 @@ public class DefaultLogicalTopology implements LogicalTopology new IllegalArgumentException("can not find result: " + resultId)); } - public Set getLogicalPipelinedRegions() { + public Set getLogicalPipelinedRegions() { final Set> regionsRaw = PipelinedRegionComputeUtil.computePipelinedRegions(this); - final Set regions = new HashSet<>(); + final Set regions = new HashSet<>(); for (Set regionVertices : regionsRaw) { - regions.add(new LogicalPipelinedRegion(regionVertices)); + regions.add(new DefaultLogicalPipelinedRegion(regionVertices)); } return regions; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/LogicalPipelinedRegion.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/LogicalPipelinedRegion.java index 9b7cc8d6ff31f48de042daa7d6317769ed1dc2bb..0e19a26dd53efcc85c5b08273ccf3b3b196b93df 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/LogicalPipelinedRegion.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/LogicalPipelinedRegion.java @@ -19,36 +19,13 @@ package org.apache.flink.runtime.jobgraph.topology; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; +import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.JobVertexID; - -import java.util.Set; -import java.util.stream.Collectors; - -import static org.apache.flink.util.Preconditions.checkNotNull; +import org.apache.flink.runtime.topology.PipelinedRegion; /** - * Set of {@link LogicalVertex} that are connected through pipelined {@link LogicalResult}. + * Pipelined region on logical level, i.e., {@link JobVertex} level. */ -public class LogicalPipelinedRegion { - - private final Set vertexIDs; - - public LogicalPipelinedRegion(final Set> logicalVertices) { - checkNotNull(logicalVertices); - - this.vertexIDs = logicalVertices.stream() - .map(LogicalVertex::getId) - .collect(Collectors.toSet()); - } - - public Set getVertexIDs() { - return vertexIDs; - } - - @Override - public String toString() { - return "LogicalPipelinedRegion{" + - "vertexIDs=" + vertexIDs + - '}'; - } +public interface LogicalPipelinedRegion, R extends LogicalResult> extends PipelinedRegion { } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/LogicalTopology.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/LogicalTopology.java index 3c195cf46acca70f5f39146a1848cc9fb01acb57..b8a8609ae17f93b629ea7acea75e5804a58e6c2e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/LogicalTopology.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/LogicalTopology.java @@ -26,5 +26,5 @@ import org.apache.flink.runtime.topology.Topology; * Represents a logical topology, i.e. {@link JobGraph}. */ public interface LogicalTopology, R extends LogicalResult> - extends Topology { + extends Topology> { } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManagerProcessSpec.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManagerProcessSpec.java new file mode 100644 index 0000000000000000000000000000000000000000..25d90cd2b95c1f5007e4366d32d82090618edabb --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManagerProcessSpec.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.jobmanager; + +import org.apache.flink.runtime.util.config.memory.CommonProcessMemorySpec; +import org.apache.flink.runtime.util.config.memory.JvmMetaspaceAndOverhead; +import org.apache.flink.runtime.util.config.memory.jobmanager.JobManagerFlinkMemory; + +/** + * Describe the specifics of different resource dimensions of the JobManager process. + * + *

A JobManager's memory consists of the following components: + *

    + *
  • JVM Heap Memory
  • + *
  • Off-heap Memory
  • + *
  • JVM Metaspace
  • + *
  • JVM Overhead
  • + *
+ * We use Total Process Memory to refer to all the memory components, while Total Flink Memory refering to all + * the components except JVM Metaspace and JVM Overhead. + * + *

The relationships of JobManager memory components are shown below. + *

+ *               ┌ ─ ─ Total Process Memory  ─ ─ ┐
+ *                ┌ ─ ─ Total Flink Memory  ─ ─ ┐
+ *               │ ┌───────────────────────────┐ │
+ *  On-Heap ----- ││      JVM Heap Memory      ││
+ *               │ └───────────────────────────┘ │
+ *               │ ┌───────────────────────────┐ │
+ *            ┌─  ││       Off-heap Memory     ││
+ *            │  │ └───────────────────────────┘ │
+ *            │   └ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┘
+ *            │  │┌─────────────────────────────┐│
+ *  Off-Heap ─|   │        JVM Metaspace        │
+ *            │  │└─────────────────────────────┘│
+ *            │   ┌─────────────────────────────┐
+ *            └─ ││        JVM Overhead         ││
+ *                └─────────────────────────────┘
+ *               └ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┘
+ * 
+ */ +public class JobManagerProcessSpec extends CommonProcessMemorySpec { + private static final long serialVersionUID = 1L; + + JobManagerProcessSpec(JobManagerFlinkMemory flinkMemory, JvmMetaspaceAndOverhead jvmMetaspaceAndOverhead) { + super(flinkMemory, jvmMetaspaceAndOverhead); + } + + @Override + public String toString() { + return "JobManagerProcessSpec {" + + "jvmHeapSize=" + getJvmHeapMemorySize().toHumanReadableString() + ", " + + "offHeapSize=" + getJvmDirectMemorySize().toHumanReadableString() + ", " + + "jvmMetaspaceSize=" + getJvmMetaspaceSize().toHumanReadableString() + ", " + + "jvmOverheadSize=" + getJvmOverheadSize().toHumanReadableString() + '}'; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManagerProcessUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManagerProcessUtils.java new file mode 100644 index 0000000000000000000000000000000000000000..dfc0ef7e056a869e02dcac847340d97bd8a3ef77 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManagerProcessUtils.java @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.jobmanager; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.runtime.util.config.memory.CommonProcessMemorySpec; +import org.apache.flink.runtime.util.config.memory.JvmMetaspaceAndOverheadOptions; +import org.apache.flink.runtime.util.config.memory.LegacyMemoryOptions; +import org.apache.flink.runtime.util.config.memory.MemoryBackwardsCompatibilityUtils; +import org.apache.flink.runtime.util.config.memory.ProcessMemoryOptions; +import org.apache.flink.runtime.util.config.memory.ProcessMemoryUtils; +import org.apache.flink.runtime.util.config.memory.jobmanager.JobManagerFlinkMemory; +import org.apache.flink.runtime.util.config.memory.jobmanager.JobManagerFlinkMemoryUtils; + +import java.util.Collections; + +/** + * JobManager utils to calculate {@link JobManagerProcessSpec} and JVM args. + */ +public class JobManagerProcessUtils { + + static final ProcessMemoryOptions JM_PROCESS_MEMORY_OPTIONS = new ProcessMemoryOptions( + Collections.singletonList(JobManagerOptions.JVM_HEAP_MEMORY), + JobManagerOptions.TOTAL_FLINK_MEMORY, + JobManagerOptions.TOTAL_PROCESS_MEMORY, + new JvmMetaspaceAndOverheadOptions( + JobManagerOptions.JVM_METASPACE, + JobManagerOptions.JVM_OVERHEAD_MIN, + JobManagerOptions.JVM_OVERHEAD_MAX, + JobManagerOptions.JVM_OVERHEAD_FRACTION)); + + @SuppressWarnings("deprecation") + static final LegacyMemoryOptions JM_LEGACY_HEAP_OPTIONS = + new LegacyMemoryOptions( + "FLINK_JM_HEAP", + JobManagerOptions.JOB_MANAGER_HEAP_MEMORY, + JobManagerOptions.JOB_MANAGER_HEAP_MEMORY_MB); + + private static final ProcessMemoryUtils PROCESS_MEMORY_UTILS = new ProcessMemoryUtils<>( + JM_PROCESS_MEMORY_OPTIONS, + new JobManagerFlinkMemoryUtils()); + + private static final MemoryBackwardsCompatibilityUtils LEGACY_MEMORY_UTILS = new MemoryBackwardsCompatibilityUtils(JM_LEGACY_HEAP_OPTIONS); + + private JobManagerProcessUtils() { + } + + public static JobManagerProcessSpec processSpecFromConfigWithFallbackForLegacyHeap( + Configuration config, + ConfigOption newFallbackOptionForLegacyHeap) { + return processSpecFromConfig( + getConfigurationWithLegacyHeapSizeMappedToNewConfigOption(config, newFallbackOptionForLegacyHeap)); + } + + static JobManagerProcessSpec processSpecFromConfig(Configuration config) { + return createMemoryProcessSpec(PROCESS_MEMORY_UTILS.memoryProcessSpecFromConfig(config)); + } + + private static JobManagerProcessSpec createMemoryProcessSpec( + CommonProcessMemorySpec processMemory) { + return new JobManagerProcessSpec(processMemory.getFlinkMemory(), processMemory.getJvmMetaspaceAndOverhead()); + } + + static Configuration getConfigurationWithLegacyHeapSizeMappedToNewConfigOption( + Configuration configuration, + ConfigOption configOption) { + return LEGACY_MEMORY_UTILS.getConfWithLegacyHeapSizeMappedToNewConfigOption(configuration, configOption); + } + + @VisibleForTesting + public static JobManagerProcessSpec createDefaultJobManagerProcessSpec(int totalProcessMemoryMb) { + Configuration configuration = new Configuration(); + configuration.set(JobManagerOptions.TOTAL_PROCESS_MEMORY, MemorySize.ofMebiBytes(totalProcessMemoryMb)); + return processSpecFromConfig(configuration); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java index 474ea9387a8bdd81282f568e367af43f1779eeaa..543ef1295d25e837bf84e63d8f1ec4499ae3b66d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java @@ -87,6 +87,7 @@ import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; import org.apache.flink.runtime.taskexecutor.slot.SlotOffer; import org.apache.flink.runtime.taskmanager.TaskExecutionState; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; +import org.apache.flink.runtime.taskmanager.UnresolvedTaskManagerLocation; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.InstantiationUtil; @@ -571,9 +572,21 @@ public class JobMaster extends FencedRpcEndpoint implements JobMast @Override public CompletableFuture registerTaskManager( final String taskManagerRpcAddress, - final TaskManagerLocation taskManagerLocation, + final UnresolvedTaskManagerLocation unresolvedTaskManagerLocation, final Time timeout) { + final TaskManagerLocation taskManagerLocation; + try { + taskManagerLocation = TaskManagerLocation.fromUnresolvedLocation(unresolvedTaskManagerLocation); + } catch (Throwable throwable) { + final String errMsg = String.format( + "Could not accept TaskManager registration. TaskManager address %s cannot be resolved. %s", + unresolvedTaskManagerLocation.getExternalAddress(), + throwable.getMessage()); + log.error(errMsg); + return CompletableFuture.completedFuture(new RegistrationResponse.Decline(errMsg)); + } + final ResourceID taskManagerId = taskManagerLocation.getResourceID(); if (registeredTaskManagers.containsKey(taskManagerId)) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java index 506564dbb8bd108525eb94903709f8d159c431bd..1006dadf0aaab0e4377107f6c8b7b9ebe380cb69 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java @@ -43,7 +43,7 @@ import org.apache.flink.runtime.rpc.RpcTimeout; import org.apache.flink.runtime.taskexecutor.AccumulatorReport; import org.apache.flink.runtime.taskexecutor.slot.SlotOffer; import org.apache.flink.runtime.taskmanager.TaskExecutionState; -import org.apache.flink.runtime.taskmanager.TaskManagerLocation; +import org.apache.flink.runtime.taskmanager.UnresolvedTaskManagerLocation; import javax.annotation.Nullable; @@ -167,13 +167,13 @@ public interface JobMasterGateway extends * Registers the task manager at the job manager. * * @param taskManagerRpcAddress the rpc address of the task manager - * @param taskManagerLocation location of the task manager + * @param unresolvedTaskManagerLocation unresolved location of the task manager * @param timeout for the rpc call * @return Future registration response indicating whether the registration was successful or not */ CompletableFuture registerTaskManager( final String taskManagerRpcAddress, - final TaskManagerLocation taskManagerLocation, + final UnresolvedTaskManagerLocation unresolvedTaskManagerLocation, @RpcTimeout final Time timeout); /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/MiniDispatcherRestEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/MiniDispatcherRestEndpoint.java index f11b6694f210fa1316d05f625db7897cdabdbab3..dfb48b586454c4d377d8d4962242961da02b4135 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/MiniDispatcherRestEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/MiniDispatcherRestEndpoint.java @@ -25,6 +25,7 @@ import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.rest.RestServerEndpointConfiguration; import org.apache.flink.runtime.rest.handler.RestHandlerConfiguration; +import org.apache.flink.runtime.rest.handler.legacy.ExecutionGraphCache; import org.apache.flink.runtime.rest.handler.legacy.metrics.MetricFetcher; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.webmonitor.RestfulGateway; @@ -32,7 +33,7 @@ import org.apache.flink.runtime.webmonitor.WebMonitorEndpoint; import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; import java.io.IOException; -import java.util.concurrent.ExecutorService; +import java.util.concurrent.ScheduledExecutorService; /** * REST endpoint for the {@link JobClusterEntrypoint}. @@ -46,9 +47,10 @@ public class MiniDispatcherRestEndpoint extends WebMonitorEndpoint resourceManagerRetriever, TransientBlobService transientBlobService, - ExecutorService executor, + ScheduledExecutorService executor, MetricFetcher metricFetcher, LeaderElectionService leaderElectionService, + ExecutionGraphCache executionGraphCache, FatalErrorHandler fatalErrorHandler) throws IOException { super( endpointConfiguration, @@ -60,6 +62,7 @@ public class MiniDispatcherRestEndpoint extends WebMonitorEndpoint we can request these slots - // again after we regained the leadership - Set allocationIds = pendingRequests.keySetB(); - - for (AllocationID allocationId : allocationIds) { - resourceManagerGateway.cancelSlotRequest(allocationId); - } + cancelPendingSlotRequests(); // do not accept any requests jobMasterId = null; @@ -244,15 +239,23 @@ public class SlotPoolImpl implements SlotPool { clear(); } + private void cancelPendingSlotRequests() { + if (resourceManagerGateway != null) { + // cancel all pending allocations --> we can request these slots + // again after we regained the leadership + Set allocationIds = pendingRequests.keySetB(); + + for (AllocationID allocationId : allocationIds) { + resourceManagerGateway.cancelSlotRequest(allocationId); + } + } + } + @Override public void close() { log.info("Stopping SlotPool."); - // cancel all pending allocations - Set allocationIds = pendingRequests.keySetB(); - for (AllocationID allocationId : allocationIds) { - resourceManagerGateway.cancelSlotRequest(allocationId); - } + cancelPendingSlotRequests(); // release all registered slots by releasing the corresponding TaskExecutors for (ResourceID taskManagerResourceId : registeredTaskManagers) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricNames.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricNames.java index 51702ba53cc0030ceb2e609983e1749a95f84952..9cad0ed31697cc5ee522de7ff3811a78d0a40a93 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricNames.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricNames.java @@ -71,4 +71,6 @@ public class MetricNames { public static String currentInputWatermarkName(int index) { return String.format(IO_CURRENT_INPUT_WATERMARK_PATERN, index); } + + public static final String TASK_IDLE_TIME = "idleTimeMs" + SUFFIX_RATE; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java index 98824cc567277a4b606bd197adcd5feed84b513c..da15079d14bdf2b8d5a7ce675b300c2c0e8db60d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java @@ -23,15 +23,21 @@ import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.DelegatingConfiguration; import org.apache.flink.configuration.MetricOptions; +import org.apache.flink.core.plugin.PluginManager; import org.apache.flink.metrics.MetricConfig; import org.apache.flink.metrics.reporter.InstantiateViaFactory; import org.apache.flink.metrics.reporter.MetricReporter; import org.apache.flink.metrics.reporter.MetricReporterFactory; import org.apache.flink.runtime.metrics.scope.ScopeFormat; +import org.apache.flink.shaded.guava18.com.google.common.collect.Iterators; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; + +import java.io.File; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -120,31 +126,46 @@ public final class ReporterSetup { } private static ReporterSetup createReporterSetup(String reporterName, MetricConfig metricConfig, MetricReporter reporter) { - LOG.info("Configuring {} with {}.", reporterName, metricConfig); reporter.open(metricConfig); return new ReporterSetup(reporterName, metricConfig, reporter); } - public static List fromConfiguration(final Configuration configuration) { + public static List fromConfiguration(final Configuration configuration, @Nullable final PluginManager pluginManager) { String includedReportersString = configuration.getString(MetricOptions.REPORTERS_LIST, ""); + + Set namedReporters = findEnabledReportersInConfiguration(configuration, includedReportersString); + + if (namedReporters.isEmpty()) { + return Collections.emptyList(); + } + + final List> reporterConfigurations = loadReporterConfigurations(configuration, namedReporters); + + final Map reporterFactories = loadAvailableReporterFactories(pluginManager); + + return setupReporters(reporterFactories, reporterConfigurations); + } + + private static Set findEnabledReportersInConfiguration(Configuration configuration, String includedReportersString) { Set includedReporters = reporterListPattern.splitAsStream(includedReportersString) .filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+ .collect(Collectors.toSet()); // use a TreeSet to make the reporter order deterministic, which is useful for testing - Set namedReporters = new TreeSet<>(String::compareTo); - // scan entire configuration for "metric.reporter" keys and parse individual reporter configurations + Set namedOrderedReporters = new TreeSet<>(String::compareTo); + + // scan entire configuration for keys starting with METRICS_REPORTER_PREFIX and determine the set of enabled reporters for (String key : configuration.keySet()) { if (key.startsWith(ConfigConstants.METRICS_REPORTER_PREFIX)) { Matcher matcher = reporterClassPattern.matcher(key); if (matcher.matches()) { String reporterName = matcher.group(1); if (includedReporters.isEmpty() || includedReporters.contains(reporterName)) { - if (namedReporters.contains(reporterName)) { + if (namedOrderedReporters.contains(reporterName)) { LOG.warn("Duplicate class configuration detected for reporter {}.", reporterName); } else { - namedReporters.add(reporterName); + namedOrderedReporters.add(reporterName); } } else { LOG.info("Excluding reporter {}, not configured in reporter list ({}).", reporterName, includedReportersString); @@ -152,12 +173,11 @@ public final class ReporterSetup { } } } + return namedOrderedReporters; + } - if (namedReporters.isEmpty()) { - return Collections.emptyList(); - } - - List> reporterConfigurations = new ArrayList<>(namedReporters.size()); + private static List> loadReporterConfigurations(Configuration configuration, Set namedReporters) { + final List> reporterConfigurations = new ArrayList<>(namedReporters.size()); for (String namedReporter: namedReporters) { DelegatingConfiguration delegatingConfiguration = new DelegatingConfiguration( @@ -166,9 +186,46 @@ public final class ReporterSetup { reporterConfigurations.add(Tuple2.of(namedReporter, delegatingConfiguration)); } + return reporterConfigurations; + } - final Map reporterFactories = loadReporterFactories(); - List reporterArguments = new ArrayList<>(reporterConfigurations.size()); + private static Map loadAvailableReporterFactories(@Nullable PluginManager pluginManager) { + final Map reporterFactories = new HashMap<>(2); + final Iterator factoryIterator = getAllReporterFactories(pluginManager); + // do not use streams or for-each loops here because they do not allow catching individual ServiceConfigurationErrors + // such an error might be caused if the META-INF/services contains an entry to a non-existing factory class + while (factoryIterator.hasNext()) { + try { + MetricReporterFactory factory = factoryIterator.next(); + String factoryClassName = factory.getClass().getName(); + MetricReporterFactory existingFactory = reporterFactories.get(factoryClassName); + if (existingFactory == null) { + reporterFactories.put(factoryClassName, factory); + LOG.debug("Found reporter factory {} at {} ", + factoryClassName, + new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation().toURI()).getCanonicalPath()); + } else { + LOG.warn("Multiple implementations of the same reporter were found in 'lib' and/or 'plugins' directories for {}. It is recommended to remove redundant reporter JARs to resolve used versions' ambiguity.", factoryClassName); + } + } catch (Exception | ServiceConfigurationError e) { + LOG.warn("Error while loading reporter factory.", e); + } + } + + return Collections.unmodifiableMap(reporterFactories); + } + + private static Iterator getAllReporterFactories(@Nullable PluginManager pluginManager) { + final Iterator factoryIteratorSPI = ServiceLoader.load(MetricReporterFactory.class).iterator(); + final Iterator factoryIteratorPlugins = pluginManager != null + ? pluginManager.load(MetricReporterFactory.class) + : Collections.emptyIterator(); + + return Iterators.concat(factoryIteratorPlugins, factoryIteratorSPI); + } + + private static List setupReporters(Map reporterFactories, List> reporterConfigurations) { + List reporterSetups = new ArrayList<>(reporterConfigurations.size()); for (Tuple2 reporterConfiguration: reporterConfigurations) { String reporterName = reporterConfiguration.f0; Configuration reporterConfig = reporterConfiguration.f1; @@ -178,34 +235,14 @@ public final class ReporterSetup { metricReporterOptional.ifPresent(reporter -> { MetricConfig metricConfig = new MetricConfig(); reporterConfig.addAllToProperties(metricConfig); - - reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter)); + reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter)); }); } catch (Throwable t) { LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", reporterName, t); } } - return reporterArguments; - } - - private static Map loadReporterFactories() { - final ServiceLoader serviceLoader = ServiceLoader.load(MetricReporterFactory.class); - - final Map reporterFactories = new HashMap<>(2); - final Iterator factoryIterator = serviceLoader.iterator(); - // do not use streams or for-each loops here because they do not allow catching individual ServiceConfigurationErrors - // such an error might be caused if the META-INF/services contains an entry to a non-existing factory class - while (factoryIterator.hasNext()) { - try { - MetricReporterFactory factory = factoryIterator.next(); - reporterFactories.put(factory.getClass().getName(), factory); - } catch (Exception | ServiceConfigurationError e) { - LOG.warn("Error while loading reporter factory.", e); - } - } - - return Collections.unmodifiableMap(reporterFactories); + return reporterSetups; } private static Optional loadReporter( diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/SlotManagerMetricGroup.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/SlotManagerMetricGroup.java new file mode 100644 index 0000000000000000000000000000000000000000..797948c297de4be9c679960d0c9e2a487c282de0 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/SlotManagerMetricGroup.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.metrics.groups; + +import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; + +/** + * Metric group which is used by the {@link SlotManager} to register metrics. + */ +public class SlotManagerMetricGroup extends AbstractImitatingJobManagerMetricGroup { + SlotManagerMetricGroup(MetricRegistry registry, String hostname) { + super(registry, hostname); + } + + public static SlotManagerMetricGroup create(MetricRegistry metricRegistry, String hostname) { + return new SlotManagerMetricGroup(metricRegistry, hostname); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/TaskIOMetricGroup.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/TaskIOMetricGroup.java index d3fa8299159260856532359c8de120bd1c7ebe3a..e2b984a5c4894910a049e7af2d8c498a21eaabbb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/TaskIOMetricGroup.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/TaskIOMetricGroup.java @@ -45,6 +45,7 @@ public class TaskIOMetricGroup extends ProxyMetricGroup { private final Meter numRecordsInRate; private final Meter numRecordsOutRate; private final Meter numBuffersOutRate; + private final Meter idleTimePerSecond; public TaskIOMetricGroup(TaskMetricGroup parent) { super(parent); @@ -61,6 +62,8 @@ public class TaskIOMetricGroup extends ProxyMetricGroup { this.numBuffersOut = counter(MetricNames.IO_NUM_BUFFERS_OUT); this.numBuffersOutRate = meter(MetricNames.IO_NUM_BUFFERS_OUT_RATE, new MeterView(numBuffersOut)); + + this.idleTimePerSecond = meter(MetricNames.TASK_IDLE_TIME, new MeterView(new SimpleCounter())); } public IOMetrics createSnapshot() { @@ -91,6 +94,10 @@ public class TaskIOMetricGroup extends ProxyMetricGroup { return numBuffersOut; } + public Meter getIdleTimeMsPerSecond() { + return idleTimePerSecond; + } + // ============================================================================================ // Metric Reuse // ============================================================================================ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/UnregisteredMetricGroups.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/UnregisteredMetricGroups.java index 82381a68c9678f7ec4c3ff7cc7cd19ed07a2133f..1c215eecf3e8a52fd7779907830f68dd3faf4703 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/UnregisteredMetricGroups.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/UnregisteredMetricGroups.java @@ -41,6 +41,10 @@ public class UnregisteredMetricGroups { return new UnregisteredResourceManagerMetricGroup(); } + public static SlotManagerMetricGroup createUnregisteredSlotManagerMetricGroup() { + return new UnregisteredSlotManagerMetricGroup(); + } + public static JobManagerMetricGroup createUnregisteredJobManagerMetricGroup() { return new UnregisteredJobManagerMetricGroup(); } @@ -87,6 +91,17 @@ public class UnregisteredMetricGroups { } } + /** + * A safe drop-in replacement for {@link SlotManagerMetricGroup SlotManagerMetricGroups}. + */ + public static class UnregisteredSlotManagerMetricGroup extends SlotManagerMetricGroup { + private static final String UNREGISTERED_HOST = "UnregisteredHost"; + + UnregisteredSlotManagerMetricGroup() { + super(NoOpMetricRegistry.INSTANCE, UNREGISTERED_HOST); + } + } + /** * A safe drop-in replacement for {@link JobManagerMetricGroup}s. */ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/util/MetricUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/util/MetricUtils.java index cda453d4699c5580198c9a873ca1573809efb420..73e33ae8b5dc86bec185c02cd6c047cab06f45be 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/util/MetricUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/util/MetricUtils.java @@ -136,12 +136,10 @@ public class MetricUtils { final String portRange = configuration.getString(MetricOptions.QUERY_SERVICE_PORT); final int threadPriority = configuration.getInteger(MetricOptions.QUERY_SERVICE_THREAD_PRIORITY); - return AkkaRpcServiceUtils.createRpcService( - hostname, - portRange, - configuration, - METRICS_ACTOR_SYSTEM_NAME, - new BootstrapTools.FixedThreadPoolExecutorConfiguration(1, 1, threadPriority)); + return AkkaRpcServiceUtils.remoteServiceBuilder(configuration, hostname, portRange) + .withActorSystemName(METRICS_ACTOR_SYSTEM_NAME) + .withActorSystemExecutorConfiguration(new BootstrapTools.FixedThreadPoolExecutorConfiguration(1, 1, threadPriority)) + .createAndStart(); } private static void instantiateClassLoaderMetrics(MetricGroup metrics) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java index 45514289121b64163d89c9ada7a50b91f6a13f2c..1bfaf2e7fd813f78ac73fc89bd789522a20e872d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java @@ -66,8 +66,7 @@ import org.apache.flink.runtime.resourcemanager.StandaloneResourceManagerFactory import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.RpcUtils; -import org.apache.flink.runtime.rpc.akka.AkkaRpcService; -import org.apache.flink.runtime.rpc.akka.AkkaRpcServiceConfiguration; +import org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils; import org.apache.flink.runtime.taskexecutor.TaskExecutor; import org.apache.flink.runtime.taskexecutor.TaskManagerRunner; import org.apache.flink.runtime.util.ExecutorThreadFactory; @@ -82,8 +81,6 @@ import org.apache.flink.util.ExecutorUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.function.FunctionUtils; -import akka.actor.ActorSystem; -import com.typesafe.config.Config; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -209,14 +206,14 @@ public class MiniCluster implements JobExecutorService, AutoCloseableAsync { public CompletableFuture getRestAddress() { synchronized (lock) { - checkState(running, "MiniCluster is not yet running."); + checkState(running, "MiniCluster is not yet running or has already been shut down."); return webMonitorLeaderRetriever.getLeaderFuture().thenApply(FunctionUtils.uncheckedFunction(addressLeaderIdTuple -> new URI(addressLeaderIdTuple.f0))); } } public ClusterInformation getClusterInformation() { synchronized (lock) { - checkState(running, "MiniCluster is not yet running."); + checkState(running, "MiniCluster is not yet running or has already been shut down."); return new ClusterInformation("localhost", blobServer.getPort()); } } @@ -261,26 +258,40 @@ public class MiniCluster implements JobExecutorService, AutoCloseableAsync { // bring up all the RPC services LOG.info("Starting RPC Service(s)"); - AkkaRpcServiceConfiguration akkaRpcServiceConfig = AkkaRpcServiceConfiguration.fromConfiguration(configuration); - final RpcServiceFactory dispatcherResourceManagreComponentRpcServiceFactory; if (useSingleRpcService) { // we always need the 'commonRpcService' for auxiliary calls - commonRpcService = createRpcService(akkaRpcServiceConfig, false, null); + commonRpcService = createLocalRpcService(configuration); final CommonRpcServiceFactory commonRpcServiceFactory = new CommonRpcServiceFactory(commonRpcService); taskManagerRpcServiceFactory = commonRpcServiceFactory; dispatcherResourceManagreComponentRpcServiceFactory = commonRpcServiceFactory; } else { - // we always need the 'commonRpcService' for auxiliary calls - commonRpcService = createRpcService(akkaRpcServiceConfig, true, null); // start a new service per component, possibly with custom bind addresses + final String jobManagerExternalAddress = miniClusterConfiguration.getJobManagerExternalAddress(); + final String taskManagerExternalAddress = miniClusterConfiguration.getTaskManagerExternalAddress(); + final String jobManagerExternalPortRange = miniClusterConfiguration.getJobManagerExternalPortRange(); + final String taskManagerExternalPortRange = miniClusterConfiguration.getTaskManagerExternalPortRange(); final String jobManagerBindAddress = miniClusterConfiguration.getJobManagerBindAddress(); final String taskManagerBindAddress = miniClusterConfiguration.getTaskManagerBindAddress(); - dispatcherResourceManagreComponentRpcServiceFactory = new DedicatedRpcServiceFactory(akkaRpcServiceConfig, jobManagerBindAddress); - taskManagerRpcServiceFactory = new DedicatedRpcServiceFactory(akkaRpcServiceConfig, taskManagerBindAddress); + dispatcherResourceManagreComponentRpcServiceFactory = + new DedicatedRpcServiceFactory( + configuration, + jobManagerExternalAddress, + jobManagerExternalPortRange, + jobManagerBindAddress); + taskManagerRpcServiceFactory = + new DedicatedRpcServiceFactory( + configuration, + taskManagerExternalAddress, + taskManagerExternalPortRange, + taskManagerBindAddress); + + // we always need the 'commonRpcService' for auxiliary calls + // bind to the JobManager address with port 0 + commonRpcService = createRemoteRpcService(configuration, jobManagerBindAddress, 0); } RpcService metricQueryServiceRpcService = MetricUtils.startMetricsRpcService( @@ -405,8 +416,8 @@ public class MiniCluster implements JobExecutorService, AutoCloseableAsync { } @Nonnull - private DispatcherResourceManagerComponentFactory createDispatcherResourceManagerComponentFactory() { - return DefaultDispatcherResourceManagerComponentFactory.createSessionComponentFactory(StandaloneResourceManagerFactory.INSTANCE); + DispatcherResourceManagerComponentFactory createDispatcherResourceManagerComponentFactory() { + return DefaultDispatcherResourceManagerComponentFactory.createSessionComponentFactory(StandaloneResourceManagerFactory.getInstance()); } @VisibleForTesting @@ -675,7 +686,7 @@ public class MiniCluster implements JobExecutorService, AutoCloseableAsync { @VisibleForTesting protected CompletableFuture getDispatcherGatewayFuture() { synchronized (lock) { - checkState(running, "MiniCluster is not yet running."); + checkState(running, "MiniCluster is not yet running or has already been shut down."); return dispatcherGatewayRetriever.getFuture(); } } @@ -710,39 +721,58 @@ public class MiniCluster implements JobExecutorService, AutoCloseableAsync { protected MetricRegistryImpl createMetricRegistry(Configuration config) { return new MetricRegistryImpl( MetricRegistryConfiguration.fromConfiguration(config), - ReporterSetup.fromConfiguration(config)); + ReporterSetup.fromConfiguration(config, null)); } /** - * Factory method to instantiate the RPC service. - * - * @param akkaRpcServiceConfig - * The default RPC timeout for asynchronous "ask" requests. - * @param remoteEnabled - * True, if the RPC service should be reachable from other (remote) RPC services. - * @param bindAddress - * The address to bind the RPC service to. Only relevant when "remoteEnabled" is true. + * Factory method to instantiate the remote RPC service. * + * @param configuration Flink configuration. + * @param bindAddress The address to bind the RPC service to. + * @param bindPort The port range to bind the RPC service to. * @return The instantiated RPC service */ - protected RpcService createRpcService( - AkkaRpcServiceConfiguration akkaRpcServiceConfig, - boolean remoteEnabled, - String bindAddress) { - - final Config akkaConfig; - - if (remoteEnabled) { - akkaConfig = AkkaUtils.getAkkaConfig(akkaRpcServiceConfig.getConfiguration(), bindAddress, 0); - } else { - akkaConfig = AkkaUtils.getAkkaConfig(akkaRpcServiceConfig.getConfiguration()); - } - - final Config effectiveAkkaConfig = AkkaUtils.testDispatcherConfig().withFallback(akkaConfig); + protected RpcService createRemoteRpcService( + Configuration configuration, + String bindAddress, + int bindPort) throws Exception { + return AkkaRpcServiceUtils.remoteServiceBuilder(configuration, bindAddress, String.valueOf(bindPort)) + .withBindAddress(bindAddress) + .withBindPort(bindPort) + .withCustomConfig(AkkaUtils.testDispatcherConfig()) + .createAndStart(); + } - final ActorSystem actorSystem = AkkaUtils.createActorSystem(effectiveAkkaConfig); + /** + * Factory method to instantiate the remote RPC service. + * + * @param configuration Flink configuration. + * @param externalAddress The external address to access the RPC service. + * @param externalPortRange The external port range to access the RPC service. + * @param bindAddress The address to bind the RPC service to. + * @return The instantiated RPC service + */ + protected RpcService createRemoteRpcService( + Configuration configuration, + String externalAddress, + String externalPortRange, + String bindAddress) throws Exception { + return AkkaRpcServiceUtils.remoteServiceBuilder(configuration, externalAddress, externalPortRange) + .withBindAddress(bindAddress) + .withCustomConfig(AkkaUtils.testDispatcherConfig()) + .createAndStart(); + } - return new AkkaRpcService(actorSystem, akkaRpcServiceConfig); + /** + * Factory method to instantiate the local RPC service. + * + * @param configuration Flink configuration. + * @return The instantiated RPC service + */ + protected RpcService createLocalRpcService(Configuration configuration) throws Exception { + return AkkaRpcServiceUtils.localServiceBuilder(configuration) + .withCustomConfig(AkkaUtils.testDispatcherConfig()) + .createAndStart(); } // ------------------------------------------------------------------------ @@ -878,7 +908,7 @@ public class MiniCluster implements JobExecutorService, AutoCloseableAsync { * Internal factory for {@link RpcService}. */ protected interface RpcServiceFactory { - RpcService createRpcService(); + RpcService createRpcService() throws Exception; } /** @@ -903,17 +933,26 @@ public class MiniCluster implements JobExecutorService, AutoCloseableAsync { */ protected class DedicatedRpcServiceFactory implements RpcServiceFactory { - private final AkkaRpcServiceConfiguration akkaRpcServiceConfig; - private final String jobManagerBindAddress; - - DedicatedRpcServiceFactory(AkkaRpcServiceConfiguration akkaRpcServiceConfig, String jobManagerBindAddress) { - this.akkaRpcServiceConfig = akkaRpcServiceConfig; - this.jobManagerBindAddress = jobManagerBindAddress; + private final Configuration configuration; + private final String externalAddress; + private final String externalPortRange; + private final String bindAddress; + + DedicatedRpcServiceFactory( + Configuration configuration, + String externalAddress, + String externalPortRange, + String bindAddress) { + this.configuration = configuration; + this.externalAddress = externalAddress; + this.externalPortRange = externalPortRange; + this.bindAddress = bindAddress; } @Override - public RpcService createRpcService() { - final RpcService rpcService = MiniCluster.this.createRpcService(akkaRpcServiceConfig, true, jobManagerBindAddress); + public RpcService createRpcService() throws Exception { + final RpcService rpcService = MiniCluster.this.createRemoteRpcService( + configuration, externalAddress, externalPortRange, bindAddress); synchronized (lock) { rpcServices.add(rpcService); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java index 2bfeeed20e482d2385dbe8e7d34a8af180d07997..abe02704ca4dddbf2db7b563d98bf384afa17f72 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java @@ -29,9 +29,6 @@ import org.apache.flink.runtime.taskexecutor.TaskExecutorResourceUtils; import org.apache.flink.util.Preconditions; import org.apache.flink.util.StringUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import javax.annotation.Nullable; import static org.apache.flink.runtime.minicluster.RpcServiceSharing.SHARED; @@ -41,8 +38,6 @@ import static org.apache.flink.runtime.minicluster.RpcServiceSharing.SHARED; */ public class MiniClusterConfiguration { - private static final Logger LOG = LoggerFactory.getLogger(MiniClusterConfiguration.class); - static final String SCHEDULER_TYPE_KEY = JobManagerOptions.SCHEDULER.key(); private final UnmodifiableConfiguration configuration; @@ -99,16 +94,36 @@ public class MiniClusterConfiguration { return numTaskManagers; } + public String getJobManagerExternalAddress() { + return commonBindAddress != null ? + commonBindAddress : + configuration.getString(JobManagerOptions.ADDRESS, "localhost"); + } + + public String getTaskManagerExternalAddress() { + return commonBindAddress != null ? + commonBindAddress : + configuration.getString(TaskManagerOptions.HOST, "localhost"); + } + + public String getJobManagerExternalPortRange() { + return String.valueOf(configuration.getInteger(JobManagerOptions.PORT, 0)); + } + + public String getTaskManagerExternalPortRange() { + return configuration.getString(TaskManagerOptions.RPC_PORT); + } + public String getJobManagerBindAddress() { return commonBindAddress != null ? commonBindAddress : - configuration.getString(JobManagerOptions.ADDRESS, "localhost"); + configuration.getString(JobManagerOptions.BIND_HOST, "localhost"); } public String getTaskManagerBindAddress() { return commonBindAddress != null ? commonBindAddress : - configuration.getString(TaskManagerOptions.HOST, "localhost"); + configuration.getString(TaskManagerOptions.BIND_HOST, "localhost"); } public Time getRpcTimeout() { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/BatchTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/BatchTask.java index 61d4ef208dc2879855083cd183242284190e1444..d22d4720a59b46e9d63eae4c0fefca7fe3a7d018 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/BatchTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/BatchTask.java @@ -40,7 +40,7 @@ import org.apache.flink.runtime.io.network.api.reader.MutableRecordReader; import org.apache.flink.runtime.io.network.api.writer.ChannelSelector; import org.apache.flink.runtime.io.network.api.writer.RecordWriter; import org.apache.flink.runtime.io.network.api.writer.RecordWriterBuilder; -import org.apache.flink.runtime.io.network.partition.consumer.InputGate; +import org.apache.flink.runtime.io.network.partition.consumer.IndexedInputGate; import org.apache.flink.runtime.io.network.partition.consumer.UnionInputGate; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.memory.MemoryManager; @@ -680,7 +680,7 @@ public class BatchTask extends AbstractInvokable impleme getEnvironment().getTaskManagerInfo().getTmpDirectories()); } else if (groupSize > 1){ // union case - InputGate[] readers = new InputGate[groupSize]; + IndexedInputGate[] readers = new IndexedInputGate[groupSize]; for (int j = 0; j < groupSize; ++j) { readers[j] = getEnvironment().getInputGate(currentReaderOffset + j); } @@ -723,7 +723,7 @@ public class BatchTask extends AbstractInvokable impleme getEnvironment().getTaskManagerInfo().getTmpDirectories()); } else if (groupSize > 1){ // union case - InputGate[] readers = new InputGate[groupSize]; + IndexedInputGate[] readers = new IndexedInputGate[groupSize]; for (int j = 0; j < groupSize; ++j) { readers[j] = getEnvironment().getInputGate(currentReaderOffset + j); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinator.java index b5951c22d4932951e0a268e9ce3e29c04ed746bc..7f7f93c1d0c4adc9329d84917ab1bc28b9cbc1a3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinator.java @@ -67,7 +67,18 @@ public interface OperatorCoordinator extends AutoCloseable { CompletableFuture checkpointCoordinator(long checkpointId) throws Exception; - void checkpointComplete(long checkpointId) throws Exception; + /** + * Notifies the coordinator that the checkpoint with the given checkpointId completes and + * was committed. + * + *

Important: This method is not supposed to throw an exception, because by the + * time we notify that the checkpoint is complete, the checkpoint is committed and cannot be + * aborted any more. If the coordinator gets into an inconsistent state internally, it should + * fail the job ({@link Context#failJob(Throwable)}) instead. Any exception propagating from + * this method may be treated as a fatal error for the JobManager, crashing the JobManager, + * and leading to an expensive "master failover" procedure. + */ + void checkpointComplete(long checkpointId); void resetToCheckpoint(byte[] checkpointData) throws Exception; @@ -87,6 +98,8 @@ public interface OperatorCoordinator extends AutoCloseable { void failTask(int subtask, Throwable cause); void failJob(Throwable cause); + + int currentParallelism(); } // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/QueryableStateUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/QueryableStateUtils.java index 0b4d448280a51136a9efec1f3910ea8d1ceb99b7..8c8ff19cd2a63466be4c03cd91a919c9a4226481 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/QueryableStateUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/query/QueryableStateUtils.java @@ -26,7 +26,6 @@ import org.slf4j.LoggerFactory; import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; -import java.net.InetAddress; import java.util.Iterator; /** @@ -54,7 +53,7 @@ public final class QueryableStateUtils { * @return the {@link KvStateClientProxy client proxy}. */ public static KvStateClientProxy createKvStateClientProxy( - final InetAddress address, + final String address, final Iterator ports, final int eventLoopThreads, final int queryThreads, @@ -70,7 +69,7 @@ public final class QueryableStateUtils { String classname = "org.apache.flink.queryablestate.client.proxy.KvStateClientProxyImpl"; Class clazz = Class.forName(classname).asSubclass(KvStateClientProxy.class); Constructor constructor = clazz.getConstructor( - InetAddress.class, + String.class, Iterator.class, Integer.class, Integer.class, @@ -108,7 +107,7 @@ public final class QueryableStateUtils { * @return the {@link KvStateServer state server}. */ public static KvStateServer createKvStateServer( - final InetAddress address, + final String address, final Iterator ports, final int eventLoopThreads, final int queryThreads, @@ -126,7 +125,7 @@ public final class QueryableStateUtils { String classname = "org.apache.flink.queryablestate.server.KvStateServerImpl"; Class clazz = Class.forName(classname).asSubclass(KvStateServer.class); Constructor constructor = clazz.getConstructor( - InetAddress.class, + String.class, Iterator.class, Integer.class, Integer.class, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RegisteredRpcConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RegisteredRpcConnection.java index 8c78ede8e11798071243e462dbfe6aab9b2d3aac..b257e17a60ef388f0705d307827c1c925876b42c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RegisteredRpcConnection.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RegisteredRpcConnection.java @@ -100,6 +100,13 @@ public abstract class RegisteredRpcConnection env; - protected final int numSlotsPerTaskManager; - - protected final TaskExecutorProcessSpec taskExecutorProcessSpec; - - protected final int defaultMemoryMB; - - protected final Collection resourceProfilesPerWorker; - /** * The updated Flink configuration. The client uploaded configuration may be updated before passed on to * {@link ResourceManager}. For example, {@link TaskManagerOptions#MANAGED_MEMORY_SIZE}. @@ -67,47 +60,41 @@ public abstract class ActiveResourceManager env, RpcService rpcService, - String resourceManagerEndpointId, ResourceID resourceId, HighAvailabilityServices highAvailabilityServices, HeartbeatServices heartbeatServices, SlotManager slotManager, + ResourceManagerPartitionTrackerFactory clusterPartitionTrackerFactory, JobLeaderIdService jobLeaderIdService, ClusterInformation clusterInformation, FatalErrorHandler fatalErrorHandler, ResourceManagerMetricGroup resourceManagerMetricGroup) { super( rpcService, - resourceManagerEndpointId, resourceId, highAvailabilityServices, heartbeatServices, slotManager, + clusterPartitionTrackerFactory, jobLeaderIdService, clusterInformation, fatalErrorHandler, - resourceManagerMetricGroup); + resourceManagerMetricGroup, + AkkaUtils.getTimeoutAsTime(flinkConfig)); this.flinkConfig = flinkConfig; this.env = env; - this.numSlotsPerTaskManager = flinkConfig.getInteger(TaskManagerOptions.NUM_TASK_SLOTS); - double defaultCpus = getCpuCores(flinkConfig); - this.taskExecutorProcessSpec = TaskExecutorProcessUtils - .newProcessSpecBuilder(flinkConfig) - .withCpuCores(defaultCpus) - .build(); - this.defaultMemoryMB = taskExecutorProcessSpec.getTotalProcessMemorySize().getMebiBytes(); - - this.resourceProfilesPerWorker = TaskExecutorProcessUtils - .createDefaultWorkerSlotProfiles(taskExecutorProcessSpec, numSlotsPerTaskManager); - // Load the flink config uploaded by flink client this.flinkClientConfig = loadClientConfiguration(); + + pendingWorkerCounter = new PendingWorkerCounter(); } protected CompletableFuture getStopTerminationFutureOrCompletedExceptionally(@Nullable Throwable exception) { @@ -123,5 +110,75 @@ public abstract class ActiveResourceManager pendingWorkerNums; + + PendingWorkerCounter() { + pendingWorkerNums = new HashMap<>(); + } + + int getTotalNum() { + return pendingWorkerNums.values().stream().reduce(0, Integer::sum); + } + + int getNum(final WorkerResourceSpec workerResourceSpec) { + return pendingWorkerNums.getOrDefault(Preconditions.checkNotNull(workerResourceSpec), 0); + } + + int increaseAndGet(final WorkerResourceSpec workerResourceSpec) { + return pendingWorkerNums.compute( + Preconditions.checkNotNull(workerResourceSpec), + (ignored, num) -> num != null ? num + 1 : 1); + } + + int decreaseAndGet(final WorkerResourceSpec workerResourceSpec) { + final Integer newValue = pendingWorkerNums.compute( + Preconditions.checkNotNull(workerResourceSpec), + (ignored, num) -> { + Preconditions.checkState(num != null && num > 0, + "Cannot decrease, no pending worker of spec %s.", workerResourceSpec); + return num == 1 ? null : num - 1; + }); + return newValue != null ? newValue : 0; + } + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ActiveResourceManagerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ActiveResourceManagerFactory.java index f96d3ea20c541d59ff74fd7e5823e9b40a36027b..64078c9066402fe5b3f9038028c2a9b9bb170671 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ActiveResourceManagerFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ActiveResourceManagerFactory.java @@ -26,7 +26,7 @@ import org.apache.flink.runtime.clusterframework.types.ResourceIDRetrievable; import org.apache.flink.runtime.entrypoint.ClusterInformation; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.metrics.groups.ResourceManagerMetricGroup; +import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; @@ -41,7 +41,7 @@ import javax.annotation.Nullable; * * @param type of the {@link ResourceIDRetrievable} */ -public abstract class ActiveResourceManagerFactory implements ResourceManagerFactory { +public abstract class ActiveResourceManagerFactory extends ResourceManagerFactory { @Override public ResourceManager createResourceManager( @@ -53,8 +53,9 @@ public abstract class ActiveResourceManagerFactory createActiveResourceManager( - Configuration configuration, - ResourceID resourceId, - RpcService rpcService, - HighAvailabilityServices highAvailabilityServices, - HeartbeatServices heartbeatServices, - FatalErrorHandler fatalErrorHandler, - ClusterInformation clusterInformation, - @Nullable String webInterfaceUrl, - ResourceManagerMetricGroup resourceManagerMetricGroup) throws Exception; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ArbitraryWorkerResourceSpecFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ArbitraryWorkerResourceSpecFactory.java new file mode 100644 index 0000000000000000000000000000000000000000..5555a7f92715e8a3fba553af1a5106009edc04c4 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ArbitraryWorkerResourceSpecFactory.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.resourcemanager; + +import org.apache.flink.configuration.Configuration; + +/** + * Implementation of {@link WorkerResourceSpecFactory} that creates arbitrary {@link WorkerResourceSpec}. + * Used for scenarios where the values in the default {@link WorkerResourceSpec} does not matter. + */ +public class ArbitraryWorkerResourceSpecFactory extends WorkerResourceSpecFactory { + + public static final ArbitraryWorkerResourceSpecFactory INSTANCE = new ArbitraryWorkerResourceSpecFactory(); + + private ArbitraryWorkerResourceSpecFactory() {} + + @Override + public WorkerResourceSpec createDefaultWorkerResourceSpec(Configuration configuration) { + return WorkerResourceSpec.ZERO; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java index b5f7b25175c09801f03d53913259095dea059e0d..a32f042e600819299c0d31d57cb540f5f259db67 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java @@ -38,6 +38,10 @@ import org.apache.flink.runtime.heartbeat.HeartbeatTarget; import org.apache.flink.runtime.heartbeat.NoOpHeartbeatManager; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.instance.InstanceID; +import org.apache.flink.runtime.io.network.partition.DataSetMetaInfo; +import org.apache.flink.runtime.io.network.partition.ResourceManagerPartitionTracker; +import org.apache.flink.runtime.io.network.partition.ResourceManagerPartitionTrackerFactory; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobmaster.JobMaster; import org.apache.flink.runtime.jobmaster.JobMasterGateway; import org.apache.flink.runtime.jobmaster.JobMasterId; @@ -54,10 +58,12 @@ import org.apache.flink.runtime.resourcemanager.registration.JobManagerRegistrat import org.apache.flink.runtime.resourcemanager.registration.WorkerRegistration; import org.apache.flink.runtime.resourcemanager.slotmanager.ResourceActions; import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; +import org.apache.flink.runtime.rest.messages.LogInfo; import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerInfo; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.FencedRpcEndpoint; import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils; import org.apache.flink.runtime.taskexecutor.FileType; import org.apache.flink.runtime.taskexecutor.SlotReport; import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; @@ -76,6 +82,7 @@ import java.util.Objects; import java.util.Optional; import java.util.UUID; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; @@ -126,6 +133,8 @@ public abstract class ResourceManager /** The slot manager maintains the available slots. */ private final SlotManager slotManager; + private final ResourceManagerPartitionTracker clusterPartitionTracker; + private final ClusterInformation clusterInformation; private final ResourceManagerMetricGroup resourceManagerMetricGroup; @@ -149,17 +158,18 @@ public abstract class ResourceManager public ResourceManager( RpcService rpcService, - String resourceManagerEndpointId, ResourceID resourceId, HighAvailabilityServices highAvailabilityServices, HeartbeatServices heartbeatServices, SlotManager slotManager, + ResourceManagerPartitionTrackerFactory clusterPartitionTrackerFactory, JobLeaderIdService jobLeaderIdService, ClusterInformation clusterInformation, FatalErrorHandler fatalErrorHandler, - ResourceManagerMetricGroup resourceManagerMetricGroup) { + ResourceManagerMetricGroup resourceManagerMetricGroup, + Time rpcTimeout) { - super(rpcService, resourceManagerEndpointId, null); + super(rpcService, AkkaRpcServiceUtils.createRandomName(RESOURCE_MANAGER_NAME), null); this.resourceId = checkNotNull(resourceId); this.highAvailabilityServices = checkNotNull(highAvailabilityServices); @@ -177,6 +187,15 @@ public abstract class ResourceManager this.jobManagerHeartbeatManager = NoOpHeartbeatManager.getInstance(); this.taskManagerHeartbeatManager = NoOpHeartbeatManager.getInstance(); + + this.clusterPartitionTracker = checkNotNull(clusterPartitionTrackerFactory).get( + (taskExecutorResourceId, dataSetIds) -> taskExecutors.get(taskExecutorResourceId).getTaskExecutorGateway() + .releaseClusterPartitions(dataSetIds, rpcTimeout) + .exceptionally(throwable -> { + log.debug("Request for release of cluster partitions belonging to data sets {} was not successful.", dataSetIds, throwable); + throw new CompletionException(throwable); + }) + ); } @@ -205,7 +224,7 @@ public abstract class ResourceManager leaderElectionService.start(this); jobLeaderIdService.start(new JobLeaderIdActionsImpl()); - registerSlotAndTaskExecutorMetrics(); + registerTaskExecutorMetrics(); } catch (Exception e) { handleStartResourceManagerServicesException(e); } @@ -517,6 +536,8 @@ public abstract class ResourceManager taskManagerHeartbeatManager.getLastHeartbeatFrom(resourceId), slotManager.getNumberRegisteredSlotsOf(taskExecutor.getInstanceID()), slotManager.getNumberFreeSlotsOf(taskExecutor.getInstanceID()), + slotManager.getRegisteredResourceOf(taskExecutor.getInstanceID()), + slotManager.getFreeResourceOf(taskExecutor.getInstanceID()), taskExecutor.getHardwareDescription())); } @@ -539,6 +560,8 @@ public abstract class ResourceManager taskManagerHeartbeatManager.getLastHeartbeatFrom(resourceId), slotManager.getNumberRegisteredSlotsOf(instanceId), slotManager.getNumberFreeSlotsOf(instanceId), + slotManager.getRegisteredResourceOf(instanceId), + slotManager.getFreeResourceOf(instanceId), taskExecutor.getHardwareDescription()); return CompletableFuture.completedFuture(taskManagerInfo); @@ -549,12 +572,16 @@ public abstract class ResourceManager public CompletableFuture requestResourceOverview(Time timeout) { final int numberSlots = slotManager.getNumberRegisteredSlots(); final int numberFreeSlots = slotManager.getNumberFreeSlots(); + final ResourceProfile totalResource = slotManager.getRegisteredResource(); + final ResourceProfile freeResource = slotManager.getFreeResource(); return CompletableFuture.completedFuture( new ResourceOverview( taskExecutors.size(), numberSlots, - numberFreeSlots)); + numberFreeSlots, + totalResource, + freeResource)); } @Override @@ -582,19 +609,54 @@ public abstract class ResourceManager } @Override - public CompletableFuture requestTaskManagerFileUpload(ResourceID taskManagerId, FileType fileType, Time timeout) { - log.debug("Request file {} upload from TaskExecutor {}.", fileType, taskManagerId); + public CompletableFuture requestTaskManagerFileUploadByType(ResourceID taskManagerId, FileType fileType, Time timeout) { + log.debug("Request {} file upload from TaskExecutor {}.", fileType, taskManagerId); final WorkerRegistration taskExecutor = taskExecutors.get(taskManagerId); if (taskExecutor == null) { - log.debug("Requested file {} upload from unregistered TaskExecutor {}.", fileType, taskManagerId); + log.debug("Request upload of file {} from unregistered TaskExecutor {}.", fileType, taskManagerId); + return FutureUtils.completedExceptionally(new UnknownTaskExecutorException(taskManagerId)); + } else { + return taskExecutor.getTaskExecutorGateway().requestFileUploadByType(fileType, timeout); + } + } + + @Override + public CompletableFuture requestTaskManagerFileUploadByName(ResourceID taskManagerId, String fileName, Time timeout) { + log.debug("Request upload of file {} from TaskExecutor {}.", fileName, taskManagerId); + + final WorkerRegistration taskExecutor = taskExecutors.get(taskManagerId); + + if (taskExecutor == null) { + log.debug("Request upload of file {} from unregistered TaskExecutor {}.", fileName, taskManagerId); + return FutureUtils.completedExceptionally(new UnknownTaskExecutorException(taskManagerId)); + } else { + return taskExecutor.getTaskExecutorGateway().requestFileUploadByName(fileName, timeout); + } + } + + @Override + public CompletableFuture> requestTaskManagerLogList(ResourceID taskManagerId, Time timeout) { + final WorkerRegistration taskExecutor = taskExecutors.get(taskManagerId); + if (taskExecutor == null) { + log.debug("Requested log list from unregistered TaskExecutor {}.", taskManagerId); return FutureUtils.completedExceptionally(new UnknownTaskExecutorException(taskManagerId)); } else { - return taskExecutor.getTaskExecutorGateway().requestFileUpload(fileType, timeout); + return taskExecutor.getTaskExecutorGateway().requestLogList(timeout); } } + @Override + public CompletableFuture releaseClusterPartitions(IntermediateDataSetID dataSetId) { + return clusterPartitionTracker.releaseClusterPartitions(dataSetId); + } + + @Override + public CompletableFuture> listDataSets() { + return CompletableFuture.completedFuture(clusterPartitionTracker.listDataSets()); + } + // ------------------------------------------------------------------------ // Internal methods // ------------------------------------------------------------------------ @@ -719,13 +781,7 @@ public abstract class ResourceManager } } - private void registerSlotAndTaskExecutorMetrics() { - resourceManagerMetricGroup.gauge( - MetricNames.TASK_SLOTS_AVAILABLE, - () -> (long) slotManager.getNumberFreeSlots()); - resourceManagerMetricGroup.gauge( - MetricNames.TASK_SLOTS_TOTAL, - () -> (long) slotManager.getNumberRegisteredSlots()); + private void registerTaskExecutorMetrics() { resourceManagerMetricGroup.gauge( MetricNames.NUM_REGISTERED_TASK_MANAGERS, () -> (long) taskExecutors.size()); @@ -792,6 +848,7 @@ public abstract class ResourceManager // TODO :: suggest failed task executor to stop itself slotManager.unregisterTaskManager(workerRegistration.getInstanceID(), cause); + clusterPartitionTracker.processTaskExecutorShutdown(resourceID); workerRegistration.getTaskExecutorGateway().disconnectResourceManager(cause); } else { @@ -1025,13 +1082,13 @@ public abstract class ResourceManager @Nullable String optionalDiagnostics) throws ResourceManagerException; /** - * Allocates a resource using the resource profile. + * Allocates a resource using the worker resource specification. * - * @param resourceProfile The resource description - * @return Collection of {@link ResourceProfile} describing the launched slots + * @param workerResourceSpec workerResourceSpec specifies the size of the to be allocated resource + * @return whether the resource can be allocated */ @VisibleForTesting - public abstract Collection startNewWorker(ResourceProfile resourceProfile); + public abstract boolean startNewWorker(WorkerResourceSpec workerResourceSpec); /** * Callback when a worker was started. @@ -1069,9 +1126,9 @@ public abstract class ResourceManager } @Override - public Collection allocateResource(ResourceProfile resourceProfile) { + public boolean allocateResource(WorkerResourceSpec workerResourceSpec) { validateRunsInMainThread(); - return startNewWorker(resourceProfile); + return startNewWorker(workerResourceSpec); } @Override @@ -1138,6 +1195,7 @@ public abstract class ResourceManager InstanceID instanceId = workerRegistration.getInstanceID(); slotManager.reportSlotStatus(instanceId, payload.getSlotReport()); + clusterPartitionTracker.processTaskExecutorClusterPartitionReport(resourceID, payload.getClusterPartitionReport()); } } @@ -1180,8 +1238,12 @@ public abstract class ResourceManager // Resource Management // ------------------------------------------------------------------------ - protected int getNumberRequiredTaskManagerSlots() { - return slotManager.getNumberPendingTaskManagerSlots(); + protected int getNumberRequiredTaskManagers() { + return getRequiredResources().values().stream().reduce(0, Integer::sum); + } + + protected Map getRequiredResources() { + return slotManager.getRequiredResources(); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerFactory.java index 113ca66a2adaaa9c061df15fd50534d2d5028ed4..8d68faf437a595d5776a714e810aea025ef4062f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerFactory.java @@ -24,37 +24,78 @@ import org.apache.flink.runtime.clusterframework.types.ResourceIDRetrievable; import org.apache.flink.runtime.entrypoint.ClusterInformation; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.metrics.groups.ResourceManagerMetricGroup; +import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.util.ConfigurationException; import javax.annotation.Nullable; -import java.util.UUID; - /** * {@link ResourceManager} factory. * * @param type of the workers of the ResourceManager */ -public interface ResourceManagerFactory { - - ResourceManager createResourceManager( - Configuration configuration, - ResourceID resourceId, - RpcService rpcService, - HighAvailabilityServices highAvailabilityServices, - HeartbeatServices heartbeatServices, - FatalErrorHandler fatalErrorHandler, - ClusterInformation clusterInformation, - @Nullable String webInterfaceUrl, - ResourceManagerMetricGroup resourceManagerMetricGroup) throws Exception; - - default String generateEndpointIdWithUUID() { - return getEndpointId() + UUID.randomUUID(); +public abstract class ResourceManagerFactory { + + public ResourceManager createResourceManager( + Configuration configuration, + ResourceID resourceId, + RpcService rpcService, + HighAvailabilityServices highAvailabilityServices, + HeartbeatServices heartbeatServices, + FatalErrorHandler fatalErrorHandler, + ClusterInformation clusterInformation, + @Nullable String webInterfaceUrl, + MetricRegistry metricRegistry, + String hostname) throws Exception { + + final ResourceManagerMetricGroup resourceManagerMetricGroup = ResourceManagerMetricGroup.create(metricRegistry, hostname); + final SlotManagerMetricGroup slotManagerMetricGroup = SlotManagerMetricGroup.create(metricRegistry, hostname); + + final ResourceManagerRuntimeServices resourceManagerRuntimeServices = createResourceManagerRuntimeServices( + configuration, rpcService, highAvailabilityServices, slotManagerMetricGroup); + + return createResourceManager( + configuration, + resourceId, + rpcService, + highAvailabilityServices, + heartbeatServices, + fatalErrorHandler, + clusterInformation, + webInterfaceUrl, + resourceManagerMetricGroup, + resourceManagerRuntimeServices); } - default String getEndpointId() { - return ResourceManager.RESOURCE_MANAGER_NAME; + protected abstract ResourceManager createResourceManager( + Configuration configuration, + ResourceID resourceId, + RpcService rpcService, + HighAvailabilityServices highAvailabilityServices, + HeartbeatServices heartbeatServices, + FatalErrorHandler fatalErrorHandler, + ClusterInformation clusterInformation, + @Nullable String webInterfaceUrl, + ResourceManagerMetricGroup resourceManagerMetricGroup, + ResourceManagerRuntimeServices resourceManagerRuntimeServices) throws Exception; + + private ResourceManagerRuntimeServices createResourceManagerRuntimeServices( + Configuration configuration, + RpcService rpcService, + HighAvailabilityServices highAvailabilityServices, + SlotManagerMetricGroup slotManagerMetricGroup) throws ConfigurationException { + + return ResourceManagerRuntimeServices.fromConfiguration( + createResourceManagerRuntimeServicesConfiguration(configuration), + highAvailabilityServices, + rpcService.getScheduledExecutor(), + slotManagerMetricGroup); } + + protected abstract ResourceManagerRuntimeServicesConfiguration createResourceManagerRuntimeServicesConfiguration( + Configuration configuration) throws ConfigurationException; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java index 0d9dc4ae4c141e87b97aecba8382367ab36333ac..f3af8cb5d0d7539ef1971edb2d7b66b436563b94 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java @@ -28,11 +28,13 @@ import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.clusterframework.types.SlotID; import org.apache.flink.runtime.instance.InstanceID; +import org.apache.flink.runtime.io.network.partition.ClusterPartitionManager; import org.apache.flink.runtime.jobmaster.JobMaster; import org.apache.flink.runtime.jobmaster.JobMasterId; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.metrics.dump.MetricQueryService; import org.apache.flink.runtime.registration.RegistrationResponse; +import org.apache.flink.runtime.rest.messages.LogInfo; import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerInfo; import org.apache.flink.runtime.rpc.FencedRpcGateway; import org.apache.flink.runtime.rpc.RpcTimeout; @@ -49,7 +51,7 @@ import java.util.concurrent.CompletableFuture; /** * The {@link ResourceManager}'s RPC gateway interface. */ -public interface ResourceManagerGateway extends FencedRpcGateway { +public interface ResourceManagerGateway extends FencedRpcGateway, ClusterPartitionManager { /** * Register a {@link JobMaster} at the resource manager. @@ -215,5 +217,25 @@ public interface ResourceManagerGateway extends FencedRpcGateway requestTaskManagerFileUpload(ResourceID taskManagerId, FileType fileType, @RpcTimeout Time timeout); + CompletableFuture requestTaskManagerFileUploadByType(ResourceID taskManagerId, FileType fileType, @RpcTimeout Time timeout); + + /** + * Request the file upload from the given {@link TaskExecutor} to the cluster's {@link BlobServer}. The + * corresponding {@link TransientBlobKey} is returned. + * + * @param taskManagerId identifying the {@link TaskExecutor} to upload the specified file + * @param fileName name of the file to upload + * @param timeout for the asynchronous operation + * @return Future which is completed with the {@link TransientBlobKey} after uploading the file to the + * {@link BlobServer}. + */ + CompletableFuture requestTaskManagerFileUploadByName(ResourceID taskManagerId, String fileName, @RpcTimeout Time timeout); + + /** + * Request log list from the given {@link TaskExecutor}. + * @param taskManagerId identifying the {@link TaskExecutor} to get log list from + * @param timeout for the asynchronous operation + * @return Future which is completed with the historical log list + */ + CompletableFuture> requestTaskManagerLogList(ResourceID taskManagerId, @RpcTimeout Time timeout); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRuntimeServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRuntimeServices.java index 1d947611ec9fe4bde0f8cf3871784040d9ddc16f..7aaa0832365e2ab48bc1c3151a312c9296cf6635 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRuntimeServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRuntimeServices.java @@ -20,12 +20,9 @@ package org.apache.flink.runtime.resourcemanager; import org.apache.flink.runtime.concurrent.ScheduledExecutor; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.resourcemanager.slotmanager.AnyMatchingSlotMatchingStrategy; -import org.apache.flink.runtime.resourcemanager.slotmanager.LeastUtilizationSlotMatchingStrategy; +import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup; import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; -import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerConfiguration; import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerImpl; -import org.apache.flink.runtime.resourcemanager.slotmanager.SlotMatchingStrategy; import org.apache.flink.util.Preconditions; /** @@ -54,9 +51,13 @@ public class ResourceManagerRuntimeServices { public static ResourceManagerRuntimeServices fromConfiguration( ResourceManagerRuntimeServicesConfiguration configuration, HighAvailabilityServices highAvailabilityServices, - ScheduledExecutor scheduledExecutor) throws Exception { + ScheduledExecutor scheduledExecutor, + SlotManagerMetricGroup slotManagerMetricGroup) { - final SlotManager slotManager = createSlotManager(configuration, scheduledExecutor); + final SlotManager slotManager = new SlotManagerImpl( + scheduledExecutor, + configuration.getSlotManagerConfiguration(), + slotManagerMetricGroup); final JobLeaderIdService jobLeaderIdService = new JobLeaderIdService( highAvailabilityServices, @@ -65,24 +66,4 @@ public class ResourceManagerRuntimeServices { return new ResourceManagerRuntimeServices(slotManager, jobLeaderIdService); } - - private static SlotManager createSlotManager(ResourceManagerRuntimeServicesConfiguration configuration, ScheduledExecutor scheduledExecutor) { - final SlotManagerConfiguration slotManagerConfiguration = configuration.getSlotManagerConfiguration(); - - final SlotMatchingStrategy slotMatchingStrategy; - - if (slotManagerConfiguration.evenlySpreadOutSlots()) { - slotMatchingStrategy = LeastUtilizationSlotMatchingStrategy.INSTANCE; - } else { - slotMatchingStrategy = AnyMatchingSlotMatchingStrategy.INSTANCE; - } - - return new SlotManagerImpl( - slotMatchingStrategy, - scheduledExecutor, - slotManagerConfiguration.getTaskManagerRequestTimeout(), - slotManagerConfiguration.getSlotRequestTimeout(), - slotManagerConfiguration.getTaskManagerTimeout(), - slotManagerConfiguration.isWaitResultConsumedBeforeRelease()); - } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRuntimeServicesConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRuntimeServicesConfiguration.java index 8331b61266bddc6754c1776eeb1510067d56a2cf..e7c6c93f91228902d75b27ca495516b375d36e40 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRuntimeServicesConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRuntimeServicesConfiguration.java @@ -50,7 +50,9 @@ public class ResourceManagerRuntimeServicesConfiguration { // ---------------------------- Static methods ---------------------------------- - public static ResourceManagerRuntimeServicesConfiguration fromConfiguration(Configuration configuration) throws ConfigurationException { + public static ResourceManagerRuntimeServicesConfiguration fromConfiguration( + Configuration configuration, + WorkerResourceSpecFactory defaultWorkerResourceSpecFactory) throws ConfigurationException { final String strJobTimeout = configuration.getString(ResourceManagerOptions.JOB_TIMEOUT); final Time jobTimeout; @@ -62,7 +64,9 @@ public class ResourceManagerRuntimeServicesConfiguration { "value " + ResourceManagerOptions.JOB_TIMEOUT + '.', e); } - final SlotManagerConfiguration slotManagerConfiguration = SlotManagerConfiguration.fromConfiguration(configuration); + final WorkerResourceSpec defaultWorkerResourceSpec = defaultWorkerResourceSpecFactory.createDefaultWorkerResourceSpec(configuration); + final SlotManagerConfiguration slotManagerConfiguration = + SlotManagerConfiguration.fromConfiguration(configuration, defaultWorkerResourceSpec); return new ResourceManagerRuntimeServicesConfiguration(jobTimeout, slotManagerConfiguration); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceOverview.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceOverview.java index ec95759611c64979ec5d0bb676c1daf638942b84..75f342044b3abe700dd6f553942cd4eaff3cb559 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceOverview.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceOverview.java @@ -18,6 +18,8 @@ package org.apache.flink.runtime.resourcemanager; +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; + import java.io.Serializable; /** @@ -27,7 +29,7 @@ public class ResourceOverview implements Serializable { private static final long serialVersionUID = 7618746920569224557L; - private static final ResourceOverview EMPTY_RESOURCE_OVERVIEW = new ResourceOverview(0, 0, 0); + private static final ResourceOverview EMPTY_RESOURCE_OVERVIEW = new ResourceOverview(0, 0, 0, ResourceProfile.ZERO, ResourceProfile.ZERO); private final int numberTaskManagers; @@ -35,10 +37,16 @@ public class ResourceOverview implements Serializable { private final int numberFreeSlots; - public ResourceOverview(int numberTaskManagers, int numberRegisteredSlots, int numberFreeSlots) { + private final ResourceProfile totalResource; + + private final ResourceProfile freeResource; + + public ResourceOverview(int numberTaskManagers, int numberRegisteredSlots, int numberFreeSlots, ResourceProfile totalResource, ResourceProfile freeResource) { this.numberTaskManagers = numberTaskManagers; this.numberRegisteredSlots = numberRegisteredSlots; this.numberFreeSlots = numberFreeSlots; + this.totalResource = totalResource; + this.freeResource = freeResource; } public int getNumberTaskManagers() { @@ -53,6 +61,14 @@ public class ResourceOverview implements Serializable { return numberFreeSlots; } + public ResourceProfile getTotalResource() { + return totalResource; + } + + public ResourceProfile getFreeResource() { + return freeResource; + } + public static ResourceOverview empty() { return EMPTY_RESOURCE_OVERVIEW; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java index d8fb0a6e620ce26097a21bec4c382a16f4744416..f188442e8c244c596a68d773ed927276dae1233f 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java @@ -21,10 +21,10 @@ package org.apache.flink.runtime.resourcemanager; import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.clusterframework.types.ResourceID; -import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.entrypoint.ClusterInformation; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.io.network.partition.ResourceManagerPartitionTrackerFactory; import org.apache.flink.runtime.metrics.groups.ResourceManagerMetricGroup; import org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException; import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; @@ -34,8 +34,6 @@ import org.apache.flink.util.Preconditions; import javax.annotation.Nullable; -import java.util.Collection; -import java.util.Collections; import java.util.concurrent.TimeUnit; /** @@ -51,27 +49,29 @@ public class StandaloneResourceManager extends ResourceManager { public StandaloneResourceManager( RpcService rpcService, - String resourceManagerEndpointId, ResourceID resourceId, HighAvailabilityServices highAvailabilityServices, HeartbeatServices heartbeatServices, SlotManager slotManager, + ResourceManagerPartitionTrackerFactory clusterPartitionTrackerFactory, JobLeaderIdService jobLeaderIdService, ClusterInformation clusterInformation, FatalErrorHandler fatalErrorHandler, ResourceManagerMetricGroup resourceManagerMetricGroup, - Time startupPeriodTime) { + Time startupPeriodTime, + Time rpcTimeout) { super( rpcService, - resourceManagerEndpointId, resourceId, highAvailabilityServices, heartbeatServices, slotManager, + clusterPartitionTrackerFactory, jobLeaderIdService, clusterInformation, fatalErrorHandler, - resourceManagerMetricGroup); + resourceManagerMetricGroup, + rpcTimeout); this.startupPeriodTime = Preconditions.checkNotNull(startupPeriodTime); } @@ -85,8 +85,8 @@ public class StandaloneResourceManager extends ResourceManager { } @Override - public Collection startNewWorker(ResourceProfile resourceProfile) { - return Collections.emptyList(); + public boolean startNewWorker(WorkerResourceSpec workerResourceSpec) { + return false; } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManagerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManagerFactory.java index e5055a6c3ed66e3dbbc18717473f166199c5a913..c79a7dd251dd8dbef95abed8ef935adbdafbe80f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManagerFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManagerFactory.java @@ -21,52 +21,65 @@ package org.apache.flink.runtime.resourcemanager; import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ConfigurationUtils; +import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.entrypoint.ClusterInformation; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.io.network.partition.ResourceManagerPartitionTrackerImpl; import org.apache.flink.runtime.metrics.groups.ResourceManagerMetricGroup; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.util.ConfigurationException; import javax.annotation.Nullable; /** * {@link ResourceManagerFactory} which creates a {@link StandaloneResourceManager}. */ -public enum StandaloneResourceManagerFactory implements ResourceManagerFactory { - INSTANCE; +public final class StandaloneResourceManagerFactory extends ResourceManagerFactory { + + private static final StandaloneResourceManagerFactory INSTANCE = new StandaloneResourceManagerFactory(); + + private StandaloneResourceManagerFactory() {} + + public static StandaloneResourceManagerFactory getInstance() { + return INSTANCE; + } @Override - public ResourceManager createResourceManager( - Configuration configuration, - ResourceID resourceId, - RpcService rpcService, - HighAvailabilityServices highAvailabilityServices, - HeartbeatServices heartbeatServices, - FatalErrorHandler fatalErrorHandler, - ClusterInformation clusterInformation, - @Nullable String webInterfaceUrl, - ResourceManagerMetricGroup resourceManagerMetricGroup) throws Exception { - final ResourceManagerRuntimeServicesConfiguration resourceManagerRuntimeServicesConfiguration = ResourceManagerRuntimeServicesConfiguration.fromConfiguration(configuration); - final ResourceManagerRuntimeServices resourceManagerRuntimeServices = ResourceManagerRuntimeServices.fromConfiguration( - resourceManagerRuntimeServicesConfiguration, - highAvailabilityServices, - rpcService.getScheduledExecutor()); + protected ResourceManager createResourceManager( + Configuration configuration, + ResourceID resourceId, + RpcService rpcService, + HighAvailabilityServices highAvailabilityServices, + HeartbeatServices heartbeatServices, + FatalErrorHandler fatalErrorHandler, + ClusterInformation clusterInformation, + @Nullable String webInterfaceUrl, + ResourceManagerMetricGroup resourceManagerMetricGroup, + ResourceManagerRuntimeServices resourceManagerRuntimeServices) { final Time standaloneClusterStartupPeriodTime = ConfigurationUtils.getStandaloneClusterStartupPeriodTime(configuration); return new StandaloneResourceManager( rpcService, - getEndpointId(), resourceId, highAvailabilityServices, heartbeatServices, resourceManagerRuntimeServices.getSlotManager(), + ResourceManagerPartitionTrackerImpl::new, resourceManagerRuntimeServices.getJobLeaderIdService(), clusterInformation, fatalErrorHandler, resourceManagerMetricGroup, - standaloneClusterStartupPeriodTime); + standaloneClusterStartupPeriodTime, + AkkaUtils.getTimeoutAsTime(configuration)); + } + + @Override + protected ResourceManagerRuntimeServicesConfiguration createResourceManagerRuntimeServicesConfiguration( + Configuration configuration) throws ConfigurationException { + return ResourceManagerRuntimeServicesConfiguration.fromConfiguration(configuration, ArbitraryWorkerResourceSpecFactory.INSTANCE); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/WorkerResourceSpec.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/WorkerResourceSpec.java new file mode 100644 index 0000000000000000000000000000000000000000..6edbfe4d48d0756b6f6396138292c520a6300682 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/WorkerResourceSpec.java @@ -0,0 +1,169 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.resourcemanager; + +import org.apache.flink.api.common.resources.CPUResource; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec; +import org.apache.flink.util.Preconditions; + +import java.io.Serializable; +import java.util.Objects; + +/** + * Resource specification of a worker, mainly used by SlotManager requesting from ResourceManager. + */ +public final class WorkerResourceSpec implements Serializable { + + private static final long serialVersionUID = 1L; + + public static final WorkerResourceSpec ZERO = new Builder().build(); + + private final CPUResource cpuCores; + + private final MemorySize taskHeapSize; + + private final MemorySize taskOffHeapSize; + + private final MemorySize networkMemSize; + + private final MemorySize managedMemSize; + + private WorkerResourceSpec( + CPUResource cpuCores, + MemorySize taskHeapSize, + MemorySize taskOffHeapSize, + MemorySize networkMemSize, + MemorySize managedMemSize) { + + this.cpuCores = Preconditions.checkNotNull(cpuCores); + this.taskHeapSize = Preconditions.checkNotNull(taskHeapSize); + this.taskOffHeapSize = Preconditions.checkNotNull(taskOffHeapSize); + this.networkMemSize = Preconditions.checkNotNull(networkMemSize); + this.managedMemSize = Preconditions.checkNotNull(managedMemSize); + } + + public static WorkerResourceSpec fromTaskExecutorProcessSpec(final TaskExecutorProcessSpec taskExecutorProcessSpec) { + Preconditions.checkNotNull(taskExecutorProcessSpec); + return new WorkerResourceSpec( + taskExecutorProcessSpec.getCpuCores(), + taskExecutorProcessSpec.getTaskHeapSize(), + taskExecutorProcessSpec.getTaskOffHeapSize(), + taskExecutorProcessSpec.getNetworkMemSize(), + taskExecutorProcessSpec.getManagedMemorySize()); + } + + public CPUResource getCpuCores() { + return cpuCores; + } + + public MemorySize getTaskHeapSize() { + return taskHeapSize; + } + + public MemorySize getTaskOffHeapSize() { + return taskOffHeapSize; + } + + public MemorySize getNetworkMemSize() { + return networkMemSize; + } + + public MemorySize getManagedMemSize() { + return managedMemSize; + } + + @Override + public int hashCode() { + return Objects.hash(cpuCores, taskHeapSize, taskOffHeapSize, networkMemSize, managedMemSize); + } + + @Override + public boolean equals(Object obj) { + if (obj == this) { + return true; + } else if (obj != null && obj.getClass() == WorkerResourceSpec.class) { + WorkerResourceSpec that = (WorkerResourceSpec) obj; + return Objects.equals(this.cpuCores, that.cpuCores) && + Objects.equals(this.taskHeapSize, that.taskHeapSize) && + Objects.equals(this.taskOffHeapSize, that.taskOffHeapSize) && + Objects.equals(this.networkMemSize, that.networkMemSize) && + Objects.equals(this.managedMemSize, that.managedMemSize); + } + return false; + } + + @Override + public String toString() { + return "WorkerResourceSpec {" + + "cpuCores=" + cpuCores.getValue().doubleValue() + + ", taskHeapSize=" + taskHeapSize.toHumanReadableString() + + ", taskOffHeapSize=" + taskOffHeapSize.toHumanReadableString() + + ", networkMemSize=" + networkMemSize.toHumanReadableString() + + ", managedMemSize=" + managedMemSize.toHumanReadableString() + + "}"; + } + + /** + * Builder for {@link WorkerResourceSpec}. + */ + public static class Builder { + private CPUResource cpuCores = new CPUResource(0.0); + private MemorySize taskHeapSize = MemorySize.ZERO; + private MemorySize taskOffHeapSize = MemorySize.ZERO; + private MemorySize networkMemSize = MemorySize.ZERO; + private MemorySize managedMemSize = MemorySize.ZERO; + + public Builder() {} + + public Builder setCpuCores(double cpuCores) { + this.cpuCores = new CPUResource(cpuCores); + return this; + } + + public Builder setTaskHeapMemoryMB(int taskHeapMemoryMB) { + this.taskHeapSize = MemorySize.ofMebiBytes(taskHeapMemoryMB); + return this; + } + + public Builder setTaskOffHeapMemoryMB(int taskOffHeapMemoryMB) { + this.taskOffHeapSize = MemorySize.ofMebiBytes(taskOffHeapMemoryMB); + return this; + } + + public Builder setNetworkMemoryMB(int networkMemoryMB) { + this.networkMemSize = MemorySize.ofMebiBytes(networkMemoryMB); + return this; + } + + public Builder setManagedMemoryMB(int managedMemoryMB) { + this.managedMemSize = MemorySize.ofMebiBytes(managedMemoryMB); + return this; + } + + public WorkerResourceSpec build() { + return new WorkerResourceSpec( + cpuCores, + taskHeapSize, + taskOffHeapSize, + networkMemSize, + managedMemSize); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/WorkerResourceSpecFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/WorkerResourceSpecFactory.java new file mode 100644 index 0000000000000000000000000000000000000000..bee573890afa279f9d12944c5ddb5aa6f1aba24f --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/WorkerResourceSpecFactory.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.resourcemanager; + +import org.apache.flink.api.common.resources.CPUResource; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec; +import org.apache.flink.runtime.clusterframework.TaskExecutorProcessUtils; + +/** + * Factory for creating deployment specific default {@link WorkerResourceSpec}. + */ +public abstract class WorkerResourceSpecFactory { + + public abstract WorkerResourceSpec createDefaultWorkerResourceSpec(Configuration configuration); + + protected WorkerResourceSpec workerResourceSpecFromConfigAndCpu(Configuration configuration, CPUResource cpuResource) { + final TaskExecutorProcessSpec taskExecutorProcessSpec = TaskExecutorProcessUtils + .newProcessSpecBuilder(configuration) + .withCpuCores(cpuResource) + .build(); + return WorkerResourceSpec.fromTaskExecutorProcessSpec(taskExecutorProcessSpec); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/ResourceActions.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/ResourceActions.java index adf8f13db57ffca24322ae164e8366a495b6b218..4f0482d687aaec232161f4c33e9866751facf1bc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/ResourceActions.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/ResourceActions.java @@ -20,11 +20,8 @@ package org.apache.flink.runtime.resourcemanager.slotmanager; import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.clusterframework.types.AllocationID; -import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.instance.InstanceID; -import org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException; - -import java.util.Collection; +import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec; /** * Resource related actions which the {@link SlotManager} can perform. @@ -40,13 +37,12 @@ public interface ResourceActions { void releaseResource(InstanceID instanceId, Exception cause); /** - * Requests to allocate a resource with the given {@link ResourceProfile}. + * Requests to allocate a resource with the given {@link WorkerResourceSpec}. * - * @param resourceProfile for the to be allocated resource - * @return Collection of {@link ResourceProfile} describing the allocated slots - * @throws ResourceManagerException if the resource cannot be allocated + * @param workerResourceSpec for the to be allocated worker + * @return whether the resource can be allocated */ - Collection allocateResource(ResourceProfile resourceProfile) throws ResourceManagerException; + boolean allocateResource(WorkerResourceSpec workerResourceSpec); /** * Notifies that an allocation failure has occurred. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java index 334255bee42e86cd9bb62b0c19564030f7c51a7c..3d9c7970c34201d4aede48581abcd82f7a1d95a5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java @@ -25,18 +25,20 @@ import org.apache.flink.runtime.clusterframework.types.SlotID; import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.resourcemanager.ResourceManagerId; import org.apache.flink.runtime.resourcemanager.SlotRequest; +import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec; import org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException; import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection; import org.apache.flink.runtime.taskexecutor.SlotReport; +import java.util.Map; import java.util.concurrent.Executor; /** * The slot manager is responsible for maintaining a view on all registered task manager slots, - * their allocation and all pending slot requests. Whenever a new slot is registered or and + * their allocation and all pending slot requests. Whenever a new slot is registered or an * allocated slot is freed, then it tries to fulfill another pending slot request. Whenever there * are not enough slots available the slot manager will notify the resource manager about it via - * {@link ResourceActions#allocateResource(ResourceProfile)}. + * {@link ResourceActions#allocateResource(WorkerResourceSpec)}. * *

In order to free resources and avoid resource leaks, idling task managers (task managers whose * slots are currently not used) and pending slot requests time out triggering their release and @@ -51,7 +53,20 @@ public interface SlotManager extends AutoCloseable { int getNumberFreeSlotsOf(InstanceID instanceId); - int getNumberPendingTaskManagerSlots(); + /** + * Get number of workers SlotManager requested from {@link ResourceActions} that are not yet fulfilled. + * @return a map whose key set is all the unique resource specs of the pending workers, + * and the corresponding value is number of pending workers of that resource spec. + */ + Map getRequiredResources(); + + ResourceProfile getRegisteredResource(); + + ResourceProfile getRegisteredResourceOf(InstanceID instanceID); + + ResourceProfile getFreeResource(); + + ResourceProfile getFreeResourceOf(InstanceID instanceID); int getNumberPendingSlotRequests(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerConfiguration.java index f75771ca7fe5b40f3615370f34dbaa67398adb68..470e83838346cfce75af89ea7740477d2b49975a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerConfiguration.java @@ -24,7 +24,9 @@ import org.apache.flink.configuration.ClusterOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.configuration.ResourceManagerOptions; +import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec; import org.apache.flink.util.ConfigurationException; import org.apache.flink.util.Preconditions; @@ -42,20 +44,26 @@ public class SlotManagerConfiguration { private final Time slotRequestTimeout; private final Time taskManagerTimeout; private final boolean waitResultConsumedBeforeRelease; - private final boolean evenlySpreadOutSlots; + private final SlotMatchingStrategy slotMatchingStrategy; + private final WorkerResourceSpec defaultWorkerResourceSpec; + private final int numSlotsPerWorker; public SlotManagerConfiguration( Time taskManagerRequestTimeout, Time slotRequestTimeout, Time taskManagerTimeout, boolean waitResultConsumedBeforeRelease, - boolean evenlySpreadOutSlots) { + SlotMatchingStrategy slotMatchingStrategy, + WorkerResourceSpec defaultWorkerResourceSpec, + int numSlotsPerWorker) { this.taskManagerRequestTimeout = Preconditions.checkNotNull(taskManagerRequestTimeout); this.slotRequestTimeout = Preconditions.checkNotNull(slotRequestTimeout); this.taskManagerTimeout = Preconditions.checkNotNull(taskManagerTimeout); this.waitResultConsumedBeforeRelease = waitResultConsumedBeforeRelease; - this.evenlySpreadOutSlots = evenlySpreadOutSlots; + this.slotMatchingStrategy = slotMatchingStrategy; + this.defaultWorkerResourceSpec = defaultWorkerResourceSpec; + this.numSlotsPerWorker = numSlotsPerWorker; } public Time getTaskManagerRequestTimeout() { @@ -74,11 +82,22 @@ public class SlotManagerConfiguration { return waitResultConsumedBeforeRelease; } - public boolean evenlySpreadOutSlots() { - return evenlySpreadOutSlots; + public SlotMatchingStrategy getSlotMatchingStrategy() { + return slotMatchingStrategy; } - public static SlotManagerConfiguration fromConfiguration(Configuration configuration) throws ConfigurationException { + public WorkerResourceSpec getDefaultWorkerResourceSpec() { + return defaultWorkerResourceSpec; + } + + public int getNumSlotsPerWorker() { + return numSlotsPerWorker; + } + + public static SlotManagerConfiguration fromConfiguration( + Configuration configuration, + WorkerResourceSpec defaultWorkerResourceSpec) throws ConfigurationException { + final Time rpcTimeout; try { rpcTimeout = AkkaUtils.getTimeoutAsTime(configuration); @@ -95,13 +114,19 @@ public class SlotManagerConfiguration { configuration.getBoolean(ResourceManagerOptions.TASK_MANAGER_RELEASE_WHEN_RESULT_CONSUMED); boolean evenlySpreadOutSlots = configuration.getBoolean(ClusterOptions.EVENLY_SPREAD_OUT_SLOTS_STRATEGY); + final SlotMatchingStrategy slotMatchingStrategy = evenlySpreadOutSlots ? + LeastUtilizationSlotMatchingStrategy.INSTANCE : AnyMatchingSlotMatchingStrategy.INSTANCE; + + int numSlotsPerWorker = configuration.getInteger(TaskManagerOptions.NUM_TASK_SLOTS); return new SlotManagerConfiguration( rpcTimeout, slotRequestTimeout, taskManagerTimeout, waitResultConsumedBeforeRelease, - evenlySpreadOutSlots); + slotMatchingStrategy, + defaultWorkerResourceSpec, + numSlotsPerWorker); } private static Time getSlotRequestTimeout(final Configuration configuration) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerImpl.java index 33c0498c3b8d0d1d2f328883e159e4e5839effaa..45d53db6409ac5727d5489a9dc0a525f295e3eaf 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerImpl.java @@ -27,8 +27,11 @@ import org.apache.flink.runtime.clusterframework.types.SlotID; import org.apache.flink.runtime.concurrent.ScheduledExecutor; import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.metrics.MetricNames; +import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup; import org.apache.flink.runtime.resourcemanager.ResourceManagerId; import org.apache.flink.runtime.resourcemanager.SlotRequest; +import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec; import org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException; import org.apache.flink.runtime.resourcemanager.exceptions.UnfulfillableSlotRequestException; import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection; @@ -38,6 +41,7 @@ import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; import org.apache.flink.runtime.taskexecutor.exceptions.SlotAllocationException; import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException; import org.apache.flink.util.FlinkException; +import org.apache.flink.util.MathUtils; import org.apache.flink.util.OptionalConsumer; import org.apache.flink.util.Preconditions; @@ -48,7 +52,7 @@ import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.util.ArrayList; -import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.LinkedHashMap; @@ -125,20 +129,34 @@ public class SlotManagerImpl implements SlotManager { * */ private boolean failUnfulfillableRequest = true; + /** + * The default resource spec of workers to request. + */ + private final WorkerResourceSpec defaultWorkerResourceSpec; + + private final int numSlotsPerWorker; + + private final ResourceProfile defaultSlotResourceProfile; + + private final SlotManagerMetricGroup slotManagerMetricGroup; + public SlotManagerImpl( - SlotMatchingStrategy slotMatchingStrategy, ScheduledExecutor scheduledExecutor, - Time taskManagerRequestTimeout, - Time slotRequestTimeout, - Time taskManagerTimeout, - boolean waitResultConsumedBeforeRelease) { + SlotManagerConfiguration slotManagerConfiguration, + SlotManagerMetricGroup slotManagerMetricGroup) { - this.slotMatchingStrategy = Preconditions.checkNotNull(slotMatchingStrategy); this.scheduledExecutor = Preconditions.checkNotNull(scheduledExecutor); - this.taskManagerRequestTimeout = Preconditions.checkNotNull(taskManagerRequestTimeout); - this.slotRequestTimeout = Preconditions.checkNotNull(slotRequestTimeout); - this.taskManagerTimeout = Preconditions.checkNotNull(taskManagerTimeout); - this.waitResultConsumedBeforeRelease = waitResultConsumedBeforeRelease; + + Preconditions.checkNotNull(slotManagerConfiguration); + this.slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy(); + this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout(); + this.slotRequestTimeout = slotManagerConfiguration.getSlotRequestTimeout(); + this.taskManagerTimeout = slotManagerConfiguration.getTaskManagerTimeout(); + this.waitResultConsumedBeforeRelease = slotManagerConfiguration.isWaitResultConsumedBeforeRelease(); + this.defaultWorkerResourceSpec = slotManagerConfiguration.getDefaultWorkerResourceSpec(); + this.numSlotsPerWorker = slotManagerConfiguration.getNumSlotsPerWorker(); + this.defaultSlotResourceProfile = generateDefaultSlotResourceProfile(defaultWorkerResourceSpec, numSlotsPerWorker); + this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup); slots = new HashMap<>(16); freeSlots = new LinkedHashMap<>(16); @@ -189,6 +207,42 @@ public class SlotManagerImpl implements SlotManager { } @Override + public Map getRequiredResources() { + final int pendingWorkerNum = MathUtils.divideRoundUp(pendingSlots.size(), numSlotsPerWorker); + return pendingWorkerNum > 0 ? + Collections.singletonMap(defaultWorkerResourceSpec, pendingWorkerNum) : + Collections.emptyMap(); + } + + @Override + public ResourceProfile getRegisteredResource() { + return getResourceFromNumSlots(getNumberRegisteredSlots()); + } + + @Override + public ResourceProfile getRegisteredResourceOf(InstanceID instanceID) { + return getResourceFromNumSlots(getNumberRegisteredSlotsOf(instanceID)); + } + + @Override + public ResourceProfile getFreeResource() { + return getResourceFromNumSlots(getNumberFreeSlots()); + } + + @Override + public ResourceProfile getFreeResourceOf(InstanceID instanceID) { + return getResourceFromNumSlots(getNumberFreeSlotsOf(instanceID)); + } + + private ResourceProfile getResourceFromNumSlots(int numSlots) { + if (numSlots < 0 || defaultSlotResourceProfile == null) { + return ResourceProfile.UNKNOWN; + } else { + return defaultSlotResourceProfile.multiply(numSlots); + } + } + + @VisibleForTesting public int getNumberPendingTaskManagerSlots() { return pendingSlots.size(); } @@ -237,6 +291,17 @@ public class SlotManagerImpl implements SlotManager { 0L, slotRequestTimeout.toMilliseconds(), TimeUnit.MILLISECONDS); + + registerSlotManagerMetrics(); + } + + private void registerSlotManagerMetrics() { + slotManagerMetricGroup.gauge( + MetricNames.TASK_SLOTS_AVAILABLE, + () -> (long) getNumberFreeSlots()); + slotManagerMetricGroup.gauge( + MetricNames.TASK_SLOTS_TOTAL, + () -> (long) getNumberRegisteredSlots()); } /** @@ -284,6 +349,7 @@ public class SlotManagerImpl implements SlotManager { LOG.info("Closing the SlotManager."); suspend(); + slotManagerMetricGroup.close(); } // --------------------------------------------------------------------------------------------- @@ -793,23 +859,25 @@ public class SlotManagerImpl implements SlotManager { return false; } - private Optional allocateResource(ResourceProfile resourceProfile) throws ResourceManagerException { - final Collection requestedSlots = resourceActions.allocateResource(resourceProfile); - - if (requestedSlots.isEmpty()) { + private Optional allocateResource(ResourceProfile requestedSlotResourceProfile) { + if (!defaultSlotResourceProfile.isMatching(requestedSlotResourceProfile)) { + // requested resource profile is unfulfillable return Optional.empty(); - } else { - final Iterator slotIterator = requestedSlots.iterator(); - final PendingTaskManagerSlot pendingTaskManagerSlot = new PendingTaskManagerSlot(slotIterator.next()); - pendingSlots.put(pendingTaskManagerSlot.getTaskManagerSlotId(), pendingTaskManagerSlot); + } - while (slotIterator.hasNext()) { - final PendingTaskManagerSlot additionalPendingTaskManagerSlot = new PendingTaskManagerSlot(slotIterator.next()); - pendingSlots.put(additionalPendingTaskManagerSlot.getTaskManagerSlotId(), additionalPendingTaskManagerSlot); - } + if (!resourceActions.allocateResource(defaultWorkerResourceSpec)) { + // resource cannot be allocated + return Optional.empty(); + } - return Optional.of(pendingTaskManagerSlot); + PendingTaskManagerSlot pendingTaskManagerSlot = null; + for (int i = 0; i < numSlotsPerWorker; ++i) { + pendingTaskManagerSlot = new PendingTaskManagerSlot(defaultSlotResourceProfile); + pendingSlots.put(pendingTaskManagerSlot.getTaskManagerSlotId(), pendingTaskManagerSlot); } + + return Optional.of(Preconditions.checkNotNull(pendingTaskManagerSlot, + "At least one pending slot should be created.")); } private void assignPendingTaskManagerSlot(PendingSlotRequest pendingSlotRequest, PendingTaskManagerSlot pendingTaskManagerSlot) { @@ -1064,6 +1132,17 @@ public class SlotManagerImpl implements SlotManager { } } + @VisibleForTesting + static ResourceProfile generateDefaultSlotResourceProfile(WorkerResourceSpec workerResourceSpec, int numSlotsPerWorker) { + return ResourceProfile.newBuilder() + .setCpuCores(workerResourceSpec.getCpuCores().divide(numSlotsPerWorker)) + .setTaskHeapMemory(workerResourceSpec.getTaskHeapSize().divide(numSlotsPerWorker)) + .setTaskOffHeapMemory(workerResourceSpec.getTaskOffHeapSize().divide(numSlotsPerWorker)) + .setManagedMemory(workerResourceSpec.getManagedMemSize().divide(numSlotsPerWorker)) + .setNetworkMemory(workerResourceSpec.getNetworkMemSize().divide(numSlotsPerWorker)) + .build(); + } + // --------------------------------------------------------------------------------------------- // Internal timeout methods // --------------------------------------------------------------------------------------------- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskManagerSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskManagerSlot.java index 1eeb8d535f588ef3397ad06667d30ec7bb11a548..3cc05f42fa7f19d335d7f9d0bc461252e1f1c869 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskManagerSlot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskManagerSlot.java @@ -50,7 +50,7 @@ public class TaskManagerSlot implements TaskManagerSlotInformation { /** Allocation id for which this slot has been allocated. */ private AllocationID allocationId; - /** Allocation id for which this slot has been allocated. */ + /** Job id for which this slot has been allocated. */ @Nullable private JobID jobId; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/JobRestEndpointFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/JobRestEndpointFactory.java index 21df8218af8fd37d55144bb7e94e5fc363734ff7..33b752da60a62aadb509a572a844b9346ff19f04 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/JobRestEndpointFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/JobRestEndpointFactory.java @@ -31,7 +31,7 @@ import org.apache.flink.runtime.webmonitor.RestfulGateway; import org.apache.flink.runtime.webmonitor.WebMonitorEndpoint; import org.apache.flink.runtime.webmonitor.retriever.LeaderGatewayRetriever; -import java.util.concurrent.ExecutorService; +import java.util.concurrent.ScheduledExecutorService; /** * {@link RestEndpointFactory} which creates a {@link MiniDispatcherRestEndpoint}. @@ -45,7 +45,7 @@ public enum JobRestEndpointFactory implements RestEndpointFactory dispatcherGatewayRetriever, LeaderGatewayRetriever resourceManagerGatewayRetriever, TransientBlobService transientBlobService, - ExecutorService executor, + ScheduledExecutorService executor, MetricFetcher metricFetcher, LeaderElectionService leaderElectionService, FatalErrorHandler fatalErrorHandler) throws Exception { @@ -61,6 +61,7 @@ public enum JobRestEndpointFactory implements RestEndpointFactory { LeaderGatewayRetriever dispatcherGatewayRetriever, LeaderGatewayRetriever resourceManagerGatewayRetriever, TransientBlobService transientBlobService, - ExecutorService executor, + ScheduledExecutorService executor, MetricFetcher metricFetcher, LeaderElectionService leaderElectionService, FatalErrorHandler fatalErrorHandler) throws Exception; + + static ExecutionGraphCache createExecutionGraphCache(RestHandlerConfiguration restConfiguration) { + return new DefaultExecutionGraphCache( + restConfiguration.getTimeout(), + Time.milliseconds(restConfiguration.getRefreshInterval())); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpoint.java index d94e3671a9c649343f804c6ae6866bd878e244f1..a264037388aea855bf306ebc446d6276fb033bce 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpoint.java @@ -32,6 +32,7 @@ import org.apache.flink.runtime.rest.handler.router.RouterHandler; import org.apache.flink.runtime.rest.versioning.RestAPIVersion; import org.apache.flink.runtime.util.ExecutorThreadFactory; import org.apache.flink.util.AutoCloseableAsync; +import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.NetUtils; import org.apache.flink.util.Preconditions; @@ -63,9 +64,11 @@ import java.nio.file.Path; import java.util.Arrays; import java.util.Collections; import java.util.Comparator; +import java.util.IdentityHashMap; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; @@ -137,6 +140,7 @@ public abstract class RestServerEndpoint implements AutoCloseableAsync { final Router router = new Router(); final CompletableFuture restAddressFuture = new CompletableFuture<>(); + final Set distinctHandlers = Collections.newSetFromMap(new IdentityHashMap<>()); handlers = initializeHandlers(restAddressFuture); @@ -152,7 +156,12 @@ public abstract class RestServerEndpoint implements AutoCloseableAsync { RestHandlerUrlComparator.INSTANCE); handlers.forEach(handler -> { + if (distinctHandlers.contains(handler.f1)) { + throw new FlinkRuntimeException("Duplicate REST handler instance found." + + " Please ensure each instance is registered only once."); + } registerHandler(router, handler, log); + distinctHandlers.add(handler.f1); }); ChannelInitializer initializer = new ChannelInitializer() { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/SessionRestEndpointFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/SessionRestEndpointFactory.java index 15d5fe29b03adacc85053ed60fe9e47faf7ecefa..c59d220d87d3e29ed5da3a4481140f82f1a869ed 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/SessionRestEndpointFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/SessionRestEndpointFactory.java @@ -30,7 +30,7 @@ import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.webmonitor.WebMonitorEndpoint; import org.apache.flink.runtime.webmonitor.retriever.LeaderGatewayRetriever; -import java.util.concurrent.ExecutorService; +import java.util.concurrent.ScheduledExecutorService; /** * {@link RestEndpointFactory} which creates a {@link DispatcherRestEndpoint}. @@ -44,7 +44,7 @@ public enum SessionRestEndpointFactory implements RestEndpointFactory dispatcherGatewayRetriever, LeaderGatewayRetriever resourceManagerGatewayRetriever, TransientBlobService transientBlobService, - ExecutorService executor, + ScheduledExecutorService executor, MetricFetcher metricFetcher, LeaderElectionService leaderElectionService, FatalErrorHandler fatalErrorHandler) throws Exception { @@ -60,6 +60,7 @@ public enum SessionRestEndpointFactory implements RestEndpointFactory terminationFuture; + + /** + * Lock object to prevent concurrent calls to {@link #closeAsync()}. + */ + private final Object lock = new Object(); + protected AbstractHandler( @Nonnull GatewayRetriever leaderRetriever, @Nonnull Time timeout, @@ -221,7 +231,14 @@ public abstract class AbstractHandler closeAsync() { - return FutureUtils.composeAfterwards(closeHandlerAsync(), inFlightRequestTracker::awaitAsync); + synchronized (lock) { + if (terminationFuture == null) { + this.terminationFuture = FutureUtils.composeAfterwards(closeHandlerAsync(), inFlightRequestTracker::awaitAsync); + } else { + log.warn("The handler instance for {} had already been closed, but another attempt at closing it was made.", untypedResponseMessageHeaders.getTargetRestEndpointURL()); + } + return this.terminationFuture; + } } protected CompletableFuture closeHandlerAsync() { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/async/AsynchronousOperationResult.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/async/AsynchronousOperationResult.java index 480b1a852a1dc80e83889d0afd6d75985533e731..f09889174993669e2cc2e1bd5798c89f9527eb04 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/async/AsynchronousOperationResult.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/async/AsynchronousOperationResult.java @@ -36,7 +36,7 @@ public class AsynchronousOperationResult implements AsynchronouslyCreatedReso private static final String FIELD_NAME_STATUS = "status"; - private static final String FIELD_NAME_OPERATION = "operation"; + public static final String FIELD_NAME_OPERATION = "operation"; @JsonProperty(FIELD_NAME_STATUS) private final QueueStatus queueStatus; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/async/AsynchronousOperationStatusMessageHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/async/AsynchronousOperationStatusMessageHeaders.java index 61ce17518a54ed74e989d840bf070b5fe891d24c..a4adbc86eb26f6d772560bd2f15dbd1f067ee089 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/async/AsynchronousOperationStatusMessageHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/async/AsynchronousOperationStatusMessageHeaders.java @@ -38,7 +38,7 @@ public abstract class AsynchronousOperationStatusMessageHeaders getValueClass(); + public abstract Class getValueClass(); @Override public Class> getResponseClass() { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/cluster/AbstractJobManagerFileHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/cluster/AbstractJobManagerFileHandler.java new file mode 100644 index 0000000000000000000000000000000000000000..5d7f2bcc9388a32caee7bd09aa9d2d8778c901e3 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/cluster/AbstractJobManagerFileHandler.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.rest.handler.cluster; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.rest.handler.AbstractHandler; +import org.apache.flink.runtime.rest.handler.HandlerRequest; +import org.apache.flink.runtime.rest.handler.util.HandlerUtils; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; +import org.apache.flink.runtime.rest.messages.ErrorResponseBody; +import org.apache.flink.runtime.rest.messages.MessageParameters; +import org.apache.flink.runtime.rest.messages.UntypedResponseMessageHeaders; +import org.apache.flink.runtime.webmonitor.RestfulGateway; +import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; +import org.apache.flink.util.FlinkException; + +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext; +import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpRequest; +import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; + +import javax.annotation.Nullable; + +import java.io.File; +import java.util.Collections; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; + +/** + * Base class for serving files from the JobManager. + */ +public abstract class AbstractJobManagerFileHandler extends AbstractHandler { + + protected AbstractJobManagerFileHandler( + GatewayRetriever leaderRetriever, + Time timeout, + Map responseHeaders, + UntypedResponseMessageHeaders messageHeaders) { + super(leaderRetriever, timeout, responseHeaders, messageHeaders); + } + + @Override + protected CompletableFuture respondToRequest(ChannelHandlerContext ctx, HttpRequest httpRequest, HandlerRequest handlerRequest, RestfulGateway gateway) { + File file = getFile(handlerRequest); + if (file != null && file.exists()) { + try { + HandlerUtils.transferFile( + ctx, + file, + httpRequest); + } catch (FlinkException e) { + throw new CompletionException(new FlinkException("Could not transfer file to client.", e)); + } + return CompletableFuture.completedFuture(null); + } else { + return HandlerUtils.sendErrorResponse( + ctx, + httpRequest, + new ErrorResponseBody("This file does not exist in JobManager log dir."), + HttpResponseStatus.NOT_FOUND, + Collections.emptyMap()); + } + } + + @Nullable + protected abstract File getFile(HandlerRequest handlerRequest); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/cluster/JobManagerCustomLogHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/cluster/JobManagerCustomLogHandler.java new file mode 100644 index 0000000000000000000000000000000000000000..479ea53b91330e82e822676a1362c95c4389d897 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/cluster/JobManagerCustomLogHandler.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.rest.handler.cluster; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.rest.handler.HandlerRequest; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; +import org.apache.flink.runtime.rest.messages.LogFileNamePathParameter; +import org.apache.flink.runtime.rest.messages.UntypedResponseMessageHeaders; +import org.apache.flink.runtime.rest.messages.cluster.FileMessageParameters; +import org.apache.flink.runtime.webmonitor.RestfulGateway; +import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; + +import javax.annotation.Nullable; + +import java.io.File; +import java.util.Map; + +/** + * Rest handler which serves the custom log file from JobManager. + */ +public class JobManagerCustomLogHandler extends AbstractJobManagerFileHandler { + + private File logDir; + + public JobManagerCustomLogHandler( + GatewayRetriever leaderRetriever, + Time timeout, Map responseHeaders, + UntypedResponseMessageHeaders messageHeaders, + @Nullable File logDir) { + super(leaderRetriever, timeout, responseHeaders, messageHeaders); + + this.logDir = logDir; + } + + @Override + protected File getFile(HandlerRequest handlerRequest) { + if (logDir == null) { + return null; + } + String filename = handlerRequest.getPathParameter(LogFileNamePathParameter.class); + return new File(logDir, filename); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/cluster/JobManagerLogFileHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/cluster/JobManagerLogFileHandler.java new file mode 100644 index 0000000000000000000000000000000000000000..00a1021e6785b53b808f0963fd8545243a4acf16 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/cluster/JobManagerLogFileHandler.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.rest.handler.cluster; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.rest.handler.HandlerRequest; +import org.apache.flink.runtime.rest.messages.EmptyMessageParameters; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; +import org.apache.flink.runtime.rest.messages.UntypedResponseMessageHeaders; +import org.apache.flink.runtime.webmonitor.RestfulGateway; +import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; + +import javax.annotation.Nullable; + +import java.io.File; +import java.util.Map; + +/** + * Rest handler which serves the log files from JobManager. + */ +public class JobManagerLogFileHandler extends AbstractJobManagerFileHandler { + + private final File file; + + public JobManagerLogFileHandler( + GatewayRetriever leaderRetriever, + Time timeout, Map responseHeaders, + UntypedResponseMessageHeaders messageHeaders, + @Nullable File file) { + super(leaderRetriever, timeout, responseHeaders, messageHeaders); + + this.file = file; + } + + @Override + protected File getFile(HandlerRequest handlerRequest) { + return file; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/cluster/JobManagerLogListHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/cluster/JobManagerLogListHandler.java new file mode 100644 index 0000000000000000000000000000000000000000..e182f7ae5a740f5d1c6b2d7eb1753baf9300e783 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/cluster/JobManagerLogListHandler.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.rest.handler.cluster; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.rest.handler.AbstractRestHandler; +import org.apache.flink.runtime.rest.handler.HandlerRequest; +import org.apache.flink.runtime.rest.handler.RestHandlerException; +import org.apache.flink.runtime.rest.messages.EmptyMessageParameters; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; +import org.apache.flink.runtime.rest.messages.LogInfo; +import org.apache.flink.runtime.rest.messages.LogListInfo; +import org.apache.flink.runtime.rest.messages.MessageHeaders; +import org.apache.flink.runtime.webmonitor.RestfulGateway; +import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + +import java.io.File; +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; + +/** + * Handler which serves detailed JobManager log list information. + */ +public class JobManagerLogListHandler extends AbstractRestHandler { + + private final File logDir; + + public JobManagerLogListHandler( + GatewayRetriever leaderRetriever, + Time timeout, + Map responseHeaders, + MessageHeaders messageHeaders, + @Nullable File logDir) { + super(leaderRetriever, timeout, responseHeaders, messageHeaders); + + this.logDir = logDir; + } + + @Override + protected CompletableFuture handleRequest(@Nonnull HandlerRequest request, @Nonnull RestfulGateway gateway) throws RestHandlerException { + if (logDir == null) { + return CompletableFuture.completedFuture(new LogListInfo(Collections.emptyList())); + } + final File[] logFiles = logDir.listFiles(); + if (logFiles == null) { + return FutureUtils.completedExceptionally(new IOException("Could not list files in " + logDir)); + } + final List logsWithLength = Arrays.stream(logFiles) + .filter(File::isFile) + .map(logFile -> new LogInfo(logFile.getName(), logFile.length())) + .collect(Collectors.toList()); + return CompletableFuture.completedFuture(new LogListInfo(logsWithLength)); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/dataset/ClusterDataSetDeleteHandlers.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/dataset/ClusterDataSetDeleteHandlers.java new file mode 100644 index 0000000000000000000000000000000000000000..5cbee25d2c30ff2df8f01851263a3102fb6d6710 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/dataset/ClusterDataSetDeleteHandlers.java @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.rest.handler.dataset; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; +import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; +import org.apache.flink.runtime.rest.handler.HandlerRequest; +import org.apache.flink.runtime.rest.handler.RestHandlerException; +import org.apache.flink.runtime.rest.handler.async.AbstractAsynchronousOperationHandlers; +import org.apache.flink.runtime.rest.handler.async.AsynchronousOperationInfo; +import org.apache.flink.runtime.rest.handler.async.OperationKey; +import org.apache.flink.runtime.rest.handler.resourcemanager.AbstractResourceManagerHandler; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; +import org.apache.flink.runtime.rest.messages.TriggerId; +import org.apache.flink.runtime.rest.messages.TriggerIdPathParameter; +import org.apache.flink.runtime.rest.messages.dataset.ClusterDataSetDeleteStatusHeaders; +import org.apache.flink.runtime.rest.messages.dataset.ClusterDataSetDeleteStatusMessageParameters; +import org.apache.flink.runtime.rest.messages.dataset.ClusterDataSetDeleteTriggerHeaders; +import org.apache.flink.runtime.rest.messages.dataset.ClusterDataSetDeleteTriggerMessageParameters; +import org.apache.flink.runtime.rest.messages.dataset.ClusterDataSetIdPathParameter; +import org.apache.flink.runtime.webmonitor.RestfulGateway; +import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; +import org.apache.flink.util.SerializedThrowable; + +import java.util.Map; +import java.util.concurrent.CompletableFuture; + +/** + * Handler for {@link ClusterDataSetDeleteTriggerHeaders}. + */ +public class ClusterDataSetDeleteHandlers extends AbstractAsynchronousOperationHandlers { + + /** + * {@link TriggerHandler} implementation for the cluster data set delete operation. + */ + public class ClusterDataSetDeleteTriggerHandler extends TriggerHandler { + + private final GatewayRetriever resourceManagerGatewayRetriever; + + public ClusterDataSetDeleteTriggerHandler( + GatewayRetriever leaderRetriever, + Time timeout, + Map responseHeaders, + GatewayRetriever resourceManagerGatewayRetriever) { + super( + leaderRetriever, + timeout, + responseHeaders, + ClusterDataSetDeleteTriggerHeaders.INSTANCE); + this.resourceManagerGatewayRetriever = resourceManagerGatewayRetriever; + } + + @Override + protected CompletableFuture triggerOperation(HandlerRequest request, RestfulGateway gateway) throws RestHandlerException { + final IntermediateDataSetID clusterPartitionId = request.getPathParameter(ClusterDataSetIdPathParameter.class); + ResourceManagerGateway resourceManagerGateway = AbstractResourceManagerHandler.getResourceManagerGateway(resourceManagerGatewayRetriever); + return resourceManagerGateway.releaseClusterPartitions(clusterPartitionId); + } + + @Override + protected OperationKey createOperationKey(HandlerRequest request) { + return new OperationKey(new TriggerId()); + } + } + + /** + * {@link StatusHandler} implementation for the cluster data set delete operation. + */ + public class ClusterDataSetDeleteStatusHandler extends StatusHandler { + + public ClusterDataSetDeleteStatusHandler( + GatewayRetriever leaderRetriever, + Time timeout, + Map responseHeaders) { + super( + leaderRetriever, + timeout, + responseHeaders, + ClusterDataSetDeleteStatusHeaders.INSTANCE); + } + + @Override + protected OperationKey getOperationKey(HandlerRequest request) { + final TriggerId triggerId = request.getPathParameter(TriggerIdPathParameter.class); + return new OperationKey(triggerId); + } + + @Override + protected AsynchronousOperationInfo exceptionalOperationResultResponse(Throwable throwable) { + return AsynchronousOperationInfo.completeExceptional(new SerializedThrowable(throwable)); + } + + @Override + protected AsynchronousOperationInfo operationResultResponse(Void ignored) { + return AsynchronousOperationInfo.complete(); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/dataset/ClusterDataSetListHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/dataset/ClusterDataSetListHandler.java new file mode 100644 index 0000000000000000000000000000000000000000..3d10c3f50b636653d714100413b53cfe160b33cb --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/dataset/ClusterDataSetListHandler.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.rest.handler.dataset; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; +import org.apache.flink.runtime.rest.handler.HandlerRequest; +import org.apache.flink.runtime.rest.handler.RestHandlerException; +import org.apache.flink.runtime.rest.handler.resourcemanager.AbstractResourceManagerHandler; +import org.apache.flink.runtime.rest.messages.EmptyMessageParameters; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; +import org.apache.flink.runtime.rest.messages.dataset.ClusterDataSetListHeaders; +import org.apache.flink.runtime.rest.messages.dataset.ClusterDataSetListResponseBody; +import org.apache.flink.runtime.webmonitor.RestfulGateway; +import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; + +import javax.annotation.Nonnull; + +import java.util.Map; +import java.util.concurrent.CompletableFuture; + +/** + * Handler for {@link ClusterDataSetListHeaders}. + */ +public class ClusterDataSetListHandler extends AbstractResourceManagerHandler { + public ClusterDataSetListHandler( + GatewayRetriever leaderRetriever, + Time timeout, + Map responseHeaders, + GatewayRetriever resourceManagerGatewayRetriever) { + super(leaderRetriever, timeout, responseHeaders, ClusterDataSetListHeaders.INSTANCE, resourceManagerGatewayRetriever); + } + + @Override + protected CompletableFuture handleRequest(@Nonnull HandlerRequest request, @Nonnull ResourceManagerGateway gateway) throws RestHandlerException { + return gateway.listDataSets().thenApply(ClusterDataSetListResponseBody::from); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/checkpoints/CheckpointingStatisticsHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/checkpoints/CheckpointingStatisticsHandler.java index 39b5f271df5550ab63c7fc5f4306f1f0b6d332d9..1c3b8c309a5daf4b988cafe611a6ac5cdcf190a4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/checkpoints/CheckpointingStatisticsHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/checkpoints/CheckpointingStatisticsHandler.java @@ -107,7 +107,7 @@ public class CheckpointingStatisticsHandler extends AbstractExecutionGraphHandle final CheckpointingStatistics.Summary summary = new CheckpointingStatistics.Summary( MinMaxAvgStatistics.valueOf(checkpointStatsSummary.getStateSizeStats()), MinMaxAvgStatistics.valueOf(checkpointStatsSummary.getEndToEndDurationStats()), - MinMaxAvgStatistics.valueOf(checkpointStatsSummary.getAlignmentBufferedStats())); + new MinMaxAvgStatistics(0, 0, 0)); final CheckpointStatsHistory checkpointStatsHistory = checkpointStatsSnapshot.getHistory(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/checkpoints/TaskCheckpointStatisticDetailsHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/checkpoints/TaskCheckpointStatisticDetailsHandler.java index 1148a6aac86982f348a83f8da72cc3bd534b7504..61bdc2c62022187d0c255d957eef4c4b74402c51 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/checkpoints/TaskCheckpointStatisticDetailsHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/checkpoints/TaskCheckpointStatisticDetailsHandler.java @@ -131,7 +131,7 @@ public class TaskCheckpointStatisticDetailsHandler taskStatistics.getLatestAckTimestamp(), taskStatistics.getStateSize(), taskStatistics.getEndToEndDuration(checkpointStats.getTriggerTimestamp()), - taskStatistics.getAlignmentBuffered(), + 0, taskStatistics.getNumberOfSubtasks(), taskStatistics.getNumberOfAcknowledgedSubtasks(), summary, @@ -146,7 +146,7 @@ public class TaskCheckpointStatisticDetailsHandler MinMaxAvgStatistics.valueOf(taskStatisticsSummary.getAsyncCheckpointDurationStats())); final TaskCheckpointStatisticsWithSubtaskDetails.CheckpointAlignment checkpointAlignment = new TaskCheckpointStatisticsWithSubtaskDetails.CheckpointAlignment( - MinMaxAvgStatistics.valueOf(taskStatisticsSummary.getAlignmentBufferedStats()), + new MinMaxAvgStatistics(0, 0, 0), MinMaxAvgStatistics.valueOf(taskStatisticsSummary.getAlignmentDurationStats())); return new TaskCheckpointStatisticsWithSubtaskDetails.Summary( @@ -178,7 +178,7 @@ public class TaskCheckpointStatisticDetailsHandler subtask.getSyncCheckpointDuration(), subtask.getAsyncCheckpointDuration()), new SubtaskCheckpointStatistics.CompletedSubtaskCheckpointStatistics.CheckpointAlignment( - subtask.getAlignmentBuffered(), + 0, subtask.getAlignmentDuration()), subtask.getCheckpointStartDelay() )); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/rescaling/RescalingStatusHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/rescaling/RescalingStatusHeaders.java index 758980660c2810e84f16e6ebd806ae6f3378587d..835038e51ccfe670fcdda35462e79f9fe359cdbe 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/rescaling/RescalingStatusHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/rescaling/RescalingStatusHeaders.java @@ -43,7 +43,7 @@ public class RescalingStatusHeaders extends private RescalingStatusHeaders() {} @Override - protected Class getValueClass() { + public Class getValueClass() { return AsynchronousOperationInfo.class; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/savepoints/SavepointHandlers.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/savepoints/SavepointHandlers.java index 68d76fa9b62c57f33b5abca370fe51d1a0b7be6a..cc363d4e11d40d4836c7be5fc5e5d580b180d5d4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/savepoints/SavepointHandlers.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/savepoints/SavepointHandlers.java @@ -91,7 +91,7 @@ import java.util.concurrent.CompletableFuture; * "status": { * "id": "COMPLETED" * }, - * "savepoint": { + * "operation": { * "location": "/tmp/savepoint-d9813b-8a68e674325b" * } * } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/DefaultExecutionGraphCache.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/DefaultExecutionGraphCache.java new file mode 100644 index 0000000000000000000000000000000000000000..a01704e7331e605d9a599484ec800ac9fe3b0397 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/DefaultExecutionGraphCache.java @@ -0,0 +1,157 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.rest.handler.legacy; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; +import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; +import org.apache.flink.runtime.webmonitor.RestfulGateway; +import org.apache.flink.util.Preconditions; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.function.Function; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Default implementation of {@link ExecutionGraphCache}. + */ +public class DefaultExecutionGraphCache implements ExecutionGraphCache { + + private final Time timeout; + + private final Time timeToLive; + + private final ConcurrentHashMap cachedExecutionGraphs; + + private volatile boolean running = true; + + public DefaultExecutionGraphCache( + Time timeout, + Time timeToLive) { + this.timeout = checkNotNull(timeout); + this.timeToLive = checkNotNull(timeToLive); + + cachedExecutionGraphs = new ConcurrentHashMap<>(4); + } + + @Override + public void close() { + running = false; + + // clear all cached AccessExecutionGraphs + cachedExecutionGraphs.clear(); + } + + @Override + public int size() { + return cachedExecutionGraphs.size(); + } + + @Override + public CompletableFuture getExecutionGraph(JobID jobId, RestfulGateway restfulGateway) { + return getExecutionGraphInternal(jobId, restfulGateway).thenApply(Function.identity()); + } + + private CompletableFuture getExecutionGraphInternal(JobID jobId, RestfulGateway restfulGateway) { + Preconditions.checkState(running, "ExecutionGraphCache is no longer running"); + + while (true) { + final ExecutionGraphEntry oldEntry = cachedExecutionGraphs.get(jobId); + + final long currentTime = System.currentTimeMillis(); + + if (oldEntry != null && currentTime < oldEntry.getTTL()) { + final CompletableFuture executionGraphFuture = oldEntry.getExecutionGraphFuture(); + if (!executionGraphFuture.isCompletedExceptionally()) { + return executionGraphFuture; + } + // otherwise it must be completed exceptionally + } + + final ExecutionGraphEntry newEntry = new ExecutionGraphEntry(currentTime + timeToLive.toMilliseconds()); + + final boolean successfulUpdate; + + if (oldEntry == null) { + successfulUpdate = cachedExecutionGraphs.putIfAbsent(jobId, newEntry) == null; + } else { + successfulUpdate = cachedExecutionGraphs.replace(jobId, oldEntry, newEntry); + // cancel potentially outstanding futures + oldEntry.getExecutionGraphFuture().cancel(false); + } + + if (successfulUpdate) { + final CompletableFuture executionGraphFuture = restfulGateway.requestJob(jobId, timeout); + + executionGraphFuture.whenComplete( + (ArchivedExecutionGraph executionGraph, Throwable throwable) -> { + if (throwable != null) { + newEntry.getExecutionGraphFuture().completeExceptionally(throwable); + + // remove exceptionally completed entry because it doesn't help + cachedExecutionGraphs.remove(jobId, newEntry); + } else { + newEntry.getExecutionGraphFuture().complete(executionGraph); + } + }); + + if (!running) { + // delete newly added entry in case of a concurrent stopping operation + cachedExecutionGraphs.remove(jobId, newEntry); + } + + return newEntry.getExecutionGraphFuture(); + } + } + } + + @Override + public void cleanup() { + long currentTime = System.currentTimeMillis(); + + // remove entries which have exceeded their time to live + cachedExecutionGraphs.values().removeIf( + (ExecutionGraphEntry entry) -> currentTime >= entry.getTTL()); + } + + /** + * Wrapper containing the current execution graph and it's time to live (TTL). + */ + private static final class ExecutionGraphEntry { + private final long ttl; + + private final CompletableFuture executionGraphFuture; + + ExecutionGraphEntry(long ttl) { + this.ttl = ttl; + this.executionGraphFuture = new CompletableFuture<>(); + } + + public long getTTL() { + return ttl; + } + + public CompletableFuture getExecutionGraphFuture() { + return executionGraphFuture; + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/ExecutionGraphCache.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/ExecutionGraphCache.java index f634a62e223c667ffe634fe4cd4818973c841c8d..b8b94897b44609104b17932b9ef99a8cb69931bd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/ExecutionGraphCache.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/ExecutionGraphCache.java @@ -7,7 +7,7 @@ * "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 + * 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, @@ -19,57 +19,24 @@ package org.apache.flink.runtime.rest.handler.legacy; import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; import org.apache.flink.runtime.webmonitor.RestfulGateway; -import org.apache.flink.util.Preconditions; import java.io.Closeable; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentHashMap; -import java.util.function.Function; - -import static org.apache.flink.util.Preconditions.checkNotNull; /** * Cache for {@link ArchivedExecutionGraph} which are obtained from the Flink cluster. Every cache entry * has an associated time to live after which a new request will trigger the reloading of the * {@link ArchivedExecutionGraph} from the cluster. */ -public class ExecutionGraphCache implements Closeable { - - private final Time timeout; - - private final Time timeToLive; - - private final ConcurrentHashMap cachedExecutionGraphs; - - private volatile boolean running = true; - - public ExecutionGraphCache( - Time timeout, - Time timeToLive) { - this.timeout = checkNotNull(timeout); - this.timeToLive = checkNotNull(timeToLive); - - cachedExecutionGraphs = new ConcurrentHashMap<>(4); - } - - @Override - public void close() { - running = false; - - // clear all cached AccessExecutionGraphs - cachedExecutionGraphs.clear(); - } +public interface ExecutionGraphCache extends Closeable { /** * Gets the number of cache entries. */ - public int size() { - return cachedExecutionGraphs.size(); - } + int size(); /** * Gets the {@link AccessExecutionGraph} for the given {@link JobID} and caches it. The @@ -80,93 +47,16 @@ public class ExecutionGraphCache implements Closeable { * @param restfulGateway to request the {@link ArchivedExecutionGraph} from * @return Future containing the requested {@link ArchivedExecutionGraph} */ - public CompletableFuture getExecutionGraph(JobID jobId, RestfulGateway restfulGateway) { - return getExecutionGraphInternal(jobId, restfulGateway).thenApply(Function.identity()); - } - - private CompletableFuture getExecutionGraphInternal(JobID jobId, RestfulGateway restfulGateway) { - Preconditions.checkState(running, "ExecutionGraphCache is no longer running"); - - while (true) { - final ExecutionGraphEntry oldEntry = cachedExecutionGraphs.get(jobId); - - final long currentTime = System.currentTimeMillis(); - - if (oldEntry != null && currentTime < oldEntry.getTTL()) { - final CompletableFuture executionGraphFuture = oldEntry.getExecutionGraphFuture(); - if (!executionGraphFuture.isCompletedExceptionally()) { - return executionGraphFuture; - } - // otherwise it must be completed exceptionally - } - - final ExecutionGraphEntry newEntry = new ExecutionGraphEntry(currentTime + timeToLive.toMilliseconds()); - - final boolean successfulUpdate; - - if (oldEntry == null) { - successfulUpdate = cachedExecutionGraphs.putIfAbsent(jobId, newEntry) == null; - } else { - successfulUpdate = cachedExecutionGraphs.replace(jobId, oldEntry, newEntry); - // cancel potentially outstanding futures - oldEntry.getExecutionGraphFuture().cancel(false); - } - - if (successfulUpdate) { - final CompletableFuture executionGraphFuture = restfulGateway.requestJob(jobId, timeout); - - executionGraphFuture.whenComplete( - (ArchivedExecutionGraph executionGraph, Throwable throwable) -> { - if (throwable != null) { - newEntry.getExecutionGraphFuture().completeExceptionally(throwable); - - // remove exceptionally completed entry because it doesn't help - cachedExecutionGraphs.remove(jobId, newEntry); - } else { - newEntry.getExecutionGraphFuture().complete(executionGraph); - } - }); - - if (!running) { - // delete newly added entry in case of a concurrent stopping operation - cachedExecutionGraphs.remove(jobId, newEntry); - } - - return newEntry.getExecutionGraphFuture(); - } - } - } + CompletableFuture getExecutionGraph(JobID jobId, RestfulGateway restfulGateway); /** - * Perform the cleanup of out dated {@link ExecutionGraphEntry}. + * Perform the cleanup of out dated cache entries. */ - public void cleanup() { - long currentTime = System.currentTimeMillis(); - - // remove entries which have exceeded their time to live - cachedExecutionGraphs.values().removeIf( - (ExecutionGraphEntry entry) -> currentTime >= entry.getTTL()); - } + void cleanup(); /** - * Wrapper containing the current execution graph and it's time to live (TTL). + * Closes the execution graph cache. */ - private static final class ExecutionGraphEntry { - private final long ttl; - - private final CompletableFuture executionGraphFuture; - - ExecutionGraphEntry(long ttl) { - this.ttl = ttl; - this.executionGraphFuture = new CompletableFuture<>(); - } - - public long getTTL() { - return ttl; - } - - public CompletableFuture getExecutionGraphFuture() { - return executionGraphFuture; - } - } + @Override + void close(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/files/StaticFileServerHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/files/StaticFileServerHandler.java index a03fb9a18419efcc1563e1386f56bb35ff586d6c..c780618cdd1d2943565602bc2d194fdac2061cc8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/files/StaticFileServerHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/files/StaticFileServerHandler.java @@ -135,10 +135,6 @@ public class StaticFileServerHandler extends LeaderRet // make sure we request the "index.html" in case there is a directory request if (routedRequest.getPath().endsWith("/")) { requestPath = routedRequest.getPath() + "index.html"; - } - // in case the files being accessed are logs or stdout files, find appropriate paths. - else if (routedRequest.getPath().equals("/jobmanager/log") || routedRequest.getPath().equals("/jobmanager/stdout")) { - requestPath = ""; } else { requestPath = routedRequest.getPath(); } @@ -245,11 +241,8 @@ public class StaticFileServerHandler extends LeaderRet HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK); setContentTypeHeader(response, file); + setDateAndCacheHeaders(response, file); - // since the log and out files are rapidly changing, we don't want to browser to cache them - if (!(requestPath.contains("log") || requestPath.contains("out"))) { - setDateAndCacheHeaders(response, file); - } if (HttpHeaders.isKeepAlive(request)) { response.headers().set(CONNECTION, HttpHeaders.Values.KEEP_ALIVE); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/resourcemanager/AbstractResourceManagerHandler.java similarity index 76% rename from flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerHandler.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/resourcemanager/AbstractResourceManagerHandler.java index d17dbff8e244f51c961dd06c5a25a5228967371b..f8ce0c62cabee50b5ae0817cd876a4330298f317 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/resourcemanager/AbstractResourceManagerHandler.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rest.handler.taskmanager; +package org.apache.flink.runtime.rest.handler.resourcemanager; import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.resourcemanager.ResourceManager; @@ -37,22 +37,21 @@ import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseSt import javax.annotation.Nonnull; import java.util.Map; -import java.util.Optional; import java.util.concurrent.CompletableFuture; /** - * Base class for TaskManager related REST handler which need access to the {@link ResourceManager}. + * Base class for REST handlers which need access to the {@link ResourceManager}. * * @param type of the {@link RestfulGateway} * @param request type * @param

response type * @param message parameters type */ -abstract class AbstractTaskManagerHandler extends AbstractRestHandler { +public abstract class AbstractResourceManagerHandler extends AbstractRestHandler { private final GatewayRetriever resourceManagerGatewayRetriever; - protected AbstractTaskManagerHandler( + protected AbstractResourceManagerHandler( GatewayRetriever leaderRetriever, Time timeout, Map responseHeaders, @@ -65,13 +64,18 @@ abstract class AbstractTaskManagerHandler handleRequest(@Nonnull HandlerRequest request, @Nonnull T gateway) throws RestHandlerException { - Optional resourceManagerGatewayOptional = resourceManagerGatewayRetriever.getNow(); - - ResourceManagerGateway resourceManagerGateway = resourceManagerGatewayOptional.orElseThrow( - () -> new RestHandlerException("Cannot connect to ResourceManager right now. Please try to refresh.", HttpResponseStatus.NOT_FOUND)); + ResourceManagerGateway resourceManagerGateway = getResourceManagerGateway(resourceManagerGatewayRetriever); return handleRequest(request, resourceManagerGateway); } protected abstract CompletableFuture

handleRequest(@Nonnull HandlerRequest request, @Nonnull ResourceManagerGateway gateway) throws RestHandlerException; + + public static ResourceManagerGateway getResourceManagerGateway(GatewayRetriever resourceManagerGatewayRetriever) throws RestHandlerException { + return resourceManagerGatewayRetriever + .getNow() + .orElseThrow(() -> new RestHandlerException( + "Cannot connect to ResourceManager right now. Please try to refresh.", + HttpResponseStatus.NOT_FOUND)); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerFileHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerFileHandler.java index 17d1b178baced997c6660597a3f2c9f36190c8df..1c9d78a98b61f793e8c408d7133b754dc5f0f4cd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerFileHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerFileHandler.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.rest.handler.taskmanager; import org.apache.flink.api.common.time.Time; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.blob.TransientBlobKey; import org.apache.flink.runtime.blob.TransientBlobService; import org.apache.flink.runtime.clusterframework.types.ResourceID; @@ -28,6 +29,7 @@ import org.apache.flink.runtime.rest.NotFoundException; import org.apache.flink.runtime.rest.handler.AbstractHandler; import org.apache.flink.runtime.rest.handler.HandlerRequest; import org.apache.flink.runtime.rest.handler.RestHandlerException; +import org.apache.flink.runtime.rest.handler.util.HandlerUtils; import org.apache.flink.runtime.rest.messages.EmptyRequestBody; import org.apache.flink.runtime.rest.messages.UntypedResponseMessageHeaders; import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerIdPathParameter; @@ -43,40 +45,20 @@ import org.apache.flink.shaded.guava18.com.google.common.cache.CacheBuilder; import org.apache.flink.shaded.guava18.com.google.common.cache.CacheLoader; import org.apache.flink.shaded.guava18.com.google.common.cache.LoadingCache; import org.apache.flink.shaded.guava18.com.google.common.cache.RemovalNotification; -import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFuture; -import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFutureListener; import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext; -import org.apache.flink.shaded.netty4.io.netty.channel.DefaultFileRegion; -import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.DefaultHttpResponse; -import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpChunkedInput; -import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders; import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpRequest; -import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponse; import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; -import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.LastHttpContent; -import org.apache.flink.shaded.netty4.io.netty.handler.ssl.SslHandler; -import org.apache.flink.shaded.netty4.io.netty.handler.stream.ChunkedFile; -import org.apache.flink.shaded.netty4.io.netty.util.concurrent.Future; -import org.apache.flink.shaded.netty4.io.netty.util.concurrent.GenericFutureListener; import javax.annotation.Nonnull; import java.io.File; -import java.io.FileNotFoundException; import java.io.IOException; -import java.io.RandomAccessFile; -import java.nio.channels.FileChannel; import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; -import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders.Names.CONNECTION; -import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE; -import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus.OK; -import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpVersion.HTTP_1_1; - /** * Base class for serving files from the {@link TaskExecutor}. */ @@ -85,7 +67,7 @@ public abstract class AbstractTaskManagerFileHandler resourceManagerGatewayRetriever; private final TransientBlobService transientBlobService; - private final LoadingCache> fileBlobKeys; + private final LoadingCache, CompletableFuture> fileBlobKeys; protected AbstractTaskManagerFileHandler( @Nonnull GatewayRetriever leaderRetriever, @@ -106,10 +88,10 @@ public abstract class AbstractTaskManagerFileHandler>() { + new CacheLoader, CompletableFuture>() { @Override - public CompletableFuture load(ResourceID resourceId) throws Exception { - return loadTaskManagerFile(resourceId); + public CompletableFuture load(Tuple2 taskManagerIdAndFileName) throws Exception { + return loadTaskManagerFile(taskManagerIdAndFileName); } }); } @@ -118,9 +100,11 @@ public abstract class AbstractTaskManagerFileHandler respondToRequest(ChannelHandlerContext ctx, HttpRequest httpRequest, HandlerRequest handlerRequest, RestfulGateway gateway) throws RestHandlerException { final ResourceID taskManagerId = handlerRequest.getPathParameter(TaskManagerIdPathParameter.class); + String filename = getFileName(handlerRequest); + final Tuple2 taskManagerIdAndFileName = new Tuple2<>(taskManagerId, filename); final CompletableFuture blobKeyFuture; try { - blobKeyFuture = fileBlobKeys.get(taskManagerId); + blobKeyFuture = fileBlobKeys.get(taskManagerIdAndFileName); } catch (ExecutionException e) { final Throwable cause = ExceptionUtils.stripExecutionException(e); throw new RestHandlerException("Could not retrieve file blob key future.", HttpResponseStatus.INTERNAL_SERVER_ERROR, cause); @@ -136,7 +120,7 @@ public abstract class AbstractTaskManagerFileHandler requestFileUpload(ResourceManagerGateway resourceManagerGateway, ResourceID taskManagerResourceId); - - private CompletableFuture loadTaskManagerFile(ResourceID taskManagerResourceId) throws RestHandlerException { - log.debug("Load file from TaskManager {}.", taskManagerResourceId); + private CompletableFuture loadTaskManagerFile(Tuple2 taskManagerIdAndFileName) throws RestHandlerException { + log.debug("Load file from TaskManager {}.", taskManagerIdAndFileName.f0); final ResourceManagerGateway resourceManagerGateway = resourceManagerGatewayRetriever .getNow() @@ -183,10 +165,12 @@ public abstract class AbstractTaskManagerFileHandler> removalNotification) { + protected abstract CompletableFuture requestFileUpload(ResourceManagerGateway resourceManagerGateway, Tuple2 taskManagerIdAndFileName); + + private void removeBlob(RemovalNotification, CompletableFuture> removalNotification) { log.debug("Remove cached file for TaskExecutor {}.", removalNotification.getKey()); final CompletableFuture value = removalNotification.getValue(); @@ -196,71 +180,7 @@ public abstract class AbstractTaskManagerFileHandler> completionListener = future -> { - fileChannel.close(); - randomAccessFile.close(); - }; - - if (ctx.pipeline().get(SslHandler.class) == null) { - ctx.write( - new DefaultFileRegion(fileChannel, 0, fileLength), ctx.newProgressivePromise()) - .addListener(completionListener); - lastContentFuture = ctx.writeAndFlush(LastHttpContent.EMPTY_LAST_CONTENT); - - } else { - lastContentFuture = ctx - .writeAndFlush( - new HttpChunkedInput(new ChunkedFile(randomAccessFile, 0, fileLength, 8192)), - ctx.newProgressivePromise()) - .addListener(completionListener); - - // HttpChunkedInput will write the end marker (LastHttpContent) for us. - } - - // close the connection, if no keep-alive is needed - if (!HttpHeaders.isKeepAlive(httpRequest)) { - lastContentFuture.addListener(ChannelFutureListener.CLOSE); - } - } catch (IOException ex) { - fileChannel.close(); - throw ex; - } - } catch (IOException ioe) { - try { - randomAccessFile.close(); - } catch (IOException e) { - throw new FlinkException("Close file or channel error.", e); - } - - throw new FlinkException("Could not transfer file " + file + " to the client.", ioe); - } + protected String getFileName(HandlerRequest handlerRequest) { + return null; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerCustomLogHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerCustomLogHandler.java new file mode 100644 index 0000000000000000000000000000000000000000..daf24b3c98ea18e335d28243be5b2c40db1f6f8f --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerCustomLogHandler.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.rest.handler.taskmanager; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.blob.TransientBlobKey; +import org.apache.flink.runtime.blob.TransientBlobService; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; +import org.apache.flink.runtime.rest.handler.HandlerRequest; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; +import org.apache.flink.runtime.rest.messages.LogFileNamePathParameter; +import org.apache.flink.runtime.rest.messages.UntypedResponseMessageHeaders; +import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerFileMessageParameters; +import org.apache.flink.runtime.taskexecutor.TaskExecutor; +import org.apache.flink.runtime.webmonitor.RestfulGateway; +import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; + +import javax.annotation.Nonnull; + +import java.util.Map; +import java.util.concurrent.CompletableFuture; + +/** + * Rest handler which serves the custom file of the {@link TaskExecutor}. + */ +public class TaskManagerCustomLogHandler extends AbstractTaskManagerFileHandler { + + public TaskManagerCustomLogHandler( + @Nonnull GatewayRetriever leaderRetriever, + @Nonnull Time timeout, + @Nonnull Map responseHeaders, + @Nonnull UntypedResponseMessageHeaders untypedResponseMessageHeaders, + @Nonnull GatewayRetriever resourceManagerGatewayRetriever, + @Nonnull TransientBlobService transientBlobService, + @Nonnull Time cacheEntryDuration) { + super(leaderRetriever, timeout, responseHeaders, untypedResponseMessageHeaders, resourceManagerGatewayRetriever, transientBlobService, cacheEntryDuration); + + } + + @Override + protected CompletableFuture requestFileUpload(ResourceManagerGateway resourceManagerGateway, Tuple2 taskManagerIdAndFileName) { + return resourceManagerGateway.requestTaskManagerFileUploadByName(taskManagerIdAndFileName.f0, taskManagerIdAndFileName.f1, timeout); + } + + @Override + protected String getFileName(HandlerRequest handlerRequest) { + return handlerRequest.getPathParameter(LogFileNamePathParameter.class); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerDetailsHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerDetailsHandler.java index b9f80b34429010879c81b14593d28d5b9574bec4..9450cf38af9ec6943f9c494bbe508c7e6f405048 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerDetailsHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerDetailsHandler.java @@ -26,6 +26,7 @@ import org.apache.flink.runtime.rest.handler.HandlerRequest; import org.apache.flink.runtime.rest.handler.RestHandlerException; import org.apache.flink.runtime.rest.handler.legacy.metrics.MetricFetcher; import org.apache.flink.runtime.rest.handler.legacy.metrics.MetricStore; +import org.apache.flink.runtime.rest.handler.resourcemanager.AbstractResourceManagerHandler; import org.apache.flink.runtime.rest.messages.EmptyRequestBody; import org.apache.flink.runtime.rest.messages.MessageHeaders; import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerDetailsInfo; @@ -51,7 +52,7 @@ import java.util.concurrent.CompletionException; /** * Handler which serves detailed TaskManager information. */ -public class TaskManagerDetailsHandler extends AbstractTaskManagerHandler { +public class TaskManagerDetailsHandler extends AbstractResourceManagerHandler { private final MetricFetcher metricFetcher; private final MetricStore metricStore; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerLogFileHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerLogFileHandler.java index 0e90696d0ea2c90d2b6ebf473664e23a348a5b94..945b5d45896eacae0eda08a7cc995f72d6de244a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerLogFileHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerLogFileHandler.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.rest.handler.taskmanager; import org.apache.flink.api.common.time.Time; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.blob.TransientBlobKey; import org.apache.flink.runtime.blob.TransientBlobService; import org.apache.flink.runtime.clusterframework.types.ResourceID; @@ -53,7 +54,7 @@ public class TaskManagerLogFileHandler extends AbstractTaskManagerFileHandler requestFileUpload(ResourceManagerGateway resourceManagerGateway, ResourceID taskManagerResourceId) { - return resourceManagerGateway.requestTaskManagerFileUpload(taskManagerResourceId, FileType.LOG, timeout); + protected CompletableFuture requestFileUpload(ResourceManagerGateway resourceManagerGateway, Tuple2 taskManagerIdAndFileName) { + return resourceManagerGateway.requestTaskManagerFileUploadByType(taskManagerIdAndFileName.f0, FileType.LOG, timeout); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerLogListHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerLogListHandler.java new file mode 100644 index 0000000000000000000000000000000000000000..501bbfc5d3390346449426aa0478eabd51b1d2ca --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerLogListHandler.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.rest.handler.taskmanager; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; +import org.apache.flink.runtime.resourcemanager.exceptions.UnknownTaskExecutorException; +import org.apache.flink.runtime.rest.handler.HandlerRequest; +import org.apache.flink.runtime.rest.handler.RestHandlerException; +import org.apache.flink.runtime.rest.handler.resourcemanager.AbstractResourceManagerHandler; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; +import org.apache.flink.runtime.rest.messages.LogInfo; +import org.apache.flink.runtime.rest.messages.LogListInfo; +import org.apache.flink.runtime.rest.messages.MessageHeaders; +import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerIdPathParameter; +import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerMessageParameters; +import org.apache.flink.runtime.webmonitor.RestfulGateway; +import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.Preconditions; + +import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; + +import javax.annotation.Nonnull; + +import java.util.Collection; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; + +/** + * Handler which serves detailed TaskManager log list information. + */ +public class TaskManagerLogListHandler extends AbstractResourceManagerHandler { + + private final GatewayRetriever resourceManagerGatewayRetriever; + + public TaskManagerLogListHandler( + GatewayRetriever leaderRetriever, + Time timeout, + Map responseHeaders, + MessageHeaders messageHeaders, + GatewayRetriever resourceManagerGatewayRetriever) { + super(leaderRetriever, timeout, responseHeaders, messageHeaders, resourceManagerGatewayRetriever); + + this.resourceManagerGatewayRetriever = Preconditions.checkNotNull(resourceManagerGatewayRetriever); + } + + @Override + protected CompletableFuture handleRequest( + @Nonnull HandlerRequest request, + @Nonnull ResourceManagerGateway gateway) throws RestHandlerException { + final ResourceID taskManagerId = request.getPathParameter(TaskManagerIdPathParameter.class); + final ResourceManagerGateway resourceManagerGateway = getResourceManagerGateway(resourceManagerGatewayRetriever); + final CompletableFuture> logsWithLengthFuture = resourceManagerGateway.requestTaskManagerLogList(taskManagerId, timeout); + + return logsWithLengthFuture.thenApply(LogListInfo::new).exceptionally( + (throwable) -> { + final Throwable strippedThrowable = ExceptionUtils.stripCompletionException(throwable); + if (strippedThrowable instanceof UnknownTaskExecutorException) { + throw new CompletionException( + new RestHandlerException( + "Could not find TaskExecutor " + taskManagerId, + HttpResponseStatus.NOT_FOUND, + strippedThrowable + )); + } else { + throw new CompletionException(throwable); + } + }); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerStdoutFileHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerStdoutFileHandler.java index 0154224294030cc29303014267d1b0a3053bdf96..f68069d1ab1a8a453b8ab918e2615919d42690c3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerStdoutFileHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerStdoutFileHandler.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.rest.handler.taskmanager; import org.apache.flink.api.common.time.Time; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.blob.TransientBlobKey; import org.apache.flink.runtime.blob.TransientBlobService; import org.apache.flink.runtime.clusterframework.types.ResourceID; @@ -53,7 +54,7 @@ public class TaskManagerStdoutFileHandler extends AbstractTaskManagerFileHandler } @Override - protected CompletableFuture requestFileUpload(ResourceManagerGateway resourceManagerGateway, ResourceID taskManagerResourceId) { - return resourceManagerGateway.requestTaskManagerFileUpload(taskManagerResourceId, FileType.STDOUT, timeout); + protected CompletableFuture requestFileUpload(ResourceManagerGateway resourceManagerGateway, Tuple2 taskManagerIdAndFileName) { + return resourceManagerGateway.requestTaskManagerFileUploadByType(taskManagerIdAndFileName.f0, FileType.STDOUT, timeout); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagersHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagersHandler.java index 6e404d5deb55d87755e0fa0c5539e8d63963d369..5efefed85e70a3b877b965a3b4f7b59306449fdb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagersHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagersHandler.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.rest.handler.HandlerRequest; import org.apache.flink.runtime.rest.handler.RestHandlerException; +import org.apache.flink.runtime.rest.handler.resourcemanager.AbstractResourceManagerHandler; import org.apache.flink.runtime.rest.messages.EmptyMessageParameters; import org.apache.flink.runtime.rest.messages.EmptyRequestBody; import org.apache.flink.runtime.rest.messages.MessageHeaders; @@ -37,7 +38,7 @@ import java.util.concurrent.CompletableFuture; /** * Returns an overview over all registered TaskManagers of the cluster. */ -public class TaskManagersHandler extends AbstractTaskManagerHandler { +public class TaskManagersHandler extends AbstractResourceManagerHandler { public TaskManagersHandler( GatewayRetriever leaderRetriever, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/util/HandlerUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/util/HandlerUtils.java index b60afbb5948ee389f8f4a828a1d7da3af40d089d..2b1f307e9fe68611332a7ec7c40b5295f65d24f3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/util/HandlerUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/util/HandlerUtils.java @@ -23,6 +23,7 @@ import org.apache.flink.runtime.rest.messages.ErrorResponseBody; import org.apache.flink.runtime.rest.messages.ResponseBody; import org.apache.flink.runtime.rest.util.RestConstants; import org.apache.flink.runtime.rest.util.RestMapperUtils; +import org.apache.flink.util.FlinkException; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf; @@ -30,25 +31,36 @@ import org.apache.flink.shaded.netty4.io.netty.buffer.Unpooled; import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFuture; import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFutureListener; import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext; +import org.apache.flink.shaded.netty4.io.netty.channel.DefaultFileRegion; import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.DefaultHttpResponse; +import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpChunkedInput; import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders; import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpRequest; import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponse; import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.LastHttpContent; +import org.apache.flink.shaded.netty4.io.netty.handler.ssl.SslHandler; +import org.apache.flink.shaded.netty4.io.netty.handler.stream.ChunkedFile; +import org.apache.flink.shaded.netty4.io.netty.util.concurrent.Future; +import org.apache.flink.shaded.netty4.io.netty.util.concurrent.GenericFutureListener; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import javax.annotation.Nonnull; +import java.io.File; +import java.io.FileNotFoundException; import java.io.IOException; +import java.io.RandomAccessFile; import java.io.StringWriter; +import java.nio.channels.FileChannel; import java.util.Map; import java.util.concurrent.CompletableFuture; import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders.Names.CONNECTION; import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE; +import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus.OK; import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpVersion.HTTP_1_1; /** @@ -227,6 +239,74 @@ public class HandlerUtils { return toCompletableFuture(lastContentFuture); } + public static void transferFile(ChannelHandlerContext ctx, File file, HttpRequest httpRequest) throws FlinkException { + final RandomAccessFile randomAccessFile; + + try { + randomAccessFile = new RandomAccessFile(file, "r"); + } catch (FileNotFoundException e) { + throw new FlinkException("Can not find file " + file + ".", e); + } + + try { + + final long fileLength = randomAccessFile.length(); + final FileChannel fileChannel = randomAccessFile.getChannel(); + + try { + HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK); + response.headers().set(CONTENT_TYPE, "text/plain"); + + if (HttpHeaders.isKeepAlive(httpRequest)) { + response.headers().set(CONNECTION, HttpHeaders.Values.KEEP_ALIVE); + } + HttpHeaders.setContentLength(response, fileLength); + + // write the initial line and the header. + ctx.write(response); + + // write the content. + final ChannelFuture lastContentFuture; + final GenericFutureListener> completionListener = future -> { + fileChannel.close(); + randomAccessFile.close(); + }; + + if (ctx.pipeline().get(SslHandler.class) == null) { + ctx.write( + new DefaultFileRegion(fileChannel, 0, fileLength), ctx.newProgressivePromise()) + .addListener(completionListener); + lastContentFuture = ctx.writeAndFlush(LastHttpContent.EMPTY_LAST_CONTENT); + + } else { + lastContentFuture = ctx + .writeAndFlush( + new HttpChunkedInput(new ChunkedFile(randomAccessFile, 0, fileLength, 8192)), + ctx.newProgressivePromise()) + .addListener(completionListener); + + // HttpChunkedInput will write the end marker (LastHttpContent) for us. + } + + // close the connection, if no keep-alive is needed + if (!HttpHeaders.isKeepAlive(httpRequest)) { + lastContentFuture.addListener(ChannelFutureListener.CLOSE); + } + } catch (IOException ex) { + fileChannel.close(); + throw ex; + } + } catch (IOException ioe) { + try { + randomAccessFile.close(); + } catch (IOException e) { + throw new FlinkException("Close file or channel error.", e); + } + + throw new FlinkException("Could not transfer file " + file + " to the client.", ioe); + } + } + private static CompletableFuture toCompletableFuture(final ChannelFuture channelFuture) { final CompletableFuture completableFuture = new CompletableFuture<>(); channelFuture.addListener(future -> { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/LogFileNamePathParameter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/LogFileNamePathParameter.java new file mode 100644 index 0000000000000000000000000000000000000000..fa5477d15dbe58ecd1cfa2920b2aa7bc7ee435eb --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/LogFileNamePathParameter.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.rest.messages; + +import org.apache.flink.runtime.rest.handler.cluster.JobManagerCustomLogHandler; +import org.apache.flink.runtime.rest.handler.taskmanager.TaskManagerCustomLogHandler; + +/** + * Parameters used by {@link TaskManagerCustomLogHandler} and {@link JobManagerCustomLogHandler}. + */ +public class LogFileNamePathParameter extends MessagePathParameter { + + public static final String KEY = "filename"; + + public LogFileNamePathParameter() { + super(KEY); + } + + @Override + protected String convertFromString(String value) { + return value; + } + + @Override + protected String convertToString(String value) { + return value; + } + + @Override + public String getDescription() { + return "String value that identifies the file name from which to read."; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/LogInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/LogInfo.java new file mode 100644 index 0000000000000000000000000000000000000000..f26a2c82d86ccb1eba78ed60f6018384ed03977c --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/LogInfo.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.rest.messages; + +import org.apache.flink.util.Preconditions; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnore; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + +import java.io.Serializable; +import java.util.Objects; + +/** + * Contains information about one log of TaskManager. + */ +public class LogInfo implements Serializable { + + public static final String NAME = "name"; + + public static final String SIZE = "size"; + + private static final long serialVersionUID = -7371944349031708629L; + + @JsonProperty(NAME) + private final String name; + + @JsonProperty(SIZE) + private final long size; + + @JsonCreator + public LogInfo(@JsonProperty(NAME) String name, @JsonProperty(SIZE) long size) { + this.name = Preconditions.checkNotNull(name); + this.size = size; + } + + @JsonIgnore + public String getName() { + return name; + } + + @JsonIgnore + public long getSize() { + return size; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + LogInfo that = (LogInfo) o; + return Objects.equals(name, that.name) && size == that.size; + } + + @Override + public int hashCode() { + return Objects.hash(name, size); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/SessionDispatcherWithUUIDFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/LogListInfo.java similarity index 40% rename from flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/SessionDispatcherWithUUIDFactory.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/LogListInfo.java index 994dbf06c545f8e6750763700e3601adae8e1c60..40cce02bddcbd89f693b0acfb0f7cad6a9c8cfba 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/SessionDispatcherWithUUIDFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/LogListInfo.java @@ -16,38 +16,57 @@ * limitations under the License. */ -package org.apache.flink.runtime.minicluster; +package org.apache.flink.runtime.rest.messages; -import org.apache.flink.runtime.dispatcher.DefaultJobManagerRunnerFactory; -import org.apache.flink.runtime.dispatcher.DispatcherFactory; -import org.apache.flink.runtime.dispatcher.DispatcherId; -import org.apache.flink.runtime.dispatcher.DispatcherServices; -import org.apache.flink.runtime.dispatcher.PartialDispatcherServicesWithJobGraphStore; -import org.apache.flink.runtime.dispatcher.StandaloneDispatcher; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.util.Preconditions; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + +import java.io.Serializable; import java.util.Collection; +import java.util.Collections; +import java.util.Objects; /** - * {@link DispatcherFactory} which creates a {@link StandaloneDispatcher} which has an - * endpoint id with a random UUID suffix. + * Class containing a collection of {@link LogInfo}. */ -public enum SessionDispatcherWithUUIDFactory implements DispatcherFactory { - INSTANCE; +public class LogListInfo implements ResponseBody, Serializable { + + public static final String FIELD_NAME_LOGS = "logs"; + + private static final long serialVersionUID = 7531494560450830517L; + + @JsonProperty(FIELD_NAME_LOGS) + private final Collection logInfos; + + @JsonCreator + public LogListInfo(@JsonProperty(FIELD_NAME_LOGS) Collection logInfos) { + this.logInfos = Preconditions.checkNotNull(logInfos); + } + + public Collection getLogInfos() { + return logInfos; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + LogListInfo that = (LogListInfo) o; + return Objects.equals(logInfos, that.logInfos); + } @Override - public StandaloneDispatcher createDispatcher( - RpcService rpcService, - DispatcherId fencingToken, - Collection recoveredJobs, - PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore) throws Exception { - // create the default dispatcher - return new StandaloneDispatcher( - rpcService, - generateEndpointIdWithUUID(), - fencingToken, - recoveredJobs, - DispatcherServices.from(partialDispatcherServicesWithJobGraphStore, DefaultJobManagerRunnerFactory.INSTANCE)); + public int hashCode() { + return Objects.hash(logInfos); + } + + public static LogListInfo empty() { + return new LogListInfo(Collections.emptyList()); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/ResourceProfileInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/ResourceProfileInfo.java new file mode 100644 index 0000000000000000000000000000000000000000..455ae7585bb1a62c76eb914acbcb20c7ecd6f6e1 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/ResourceProfileInfo.java @@ -0,0 +1,159 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.rest.messages; + +import org.apache.flink.api.common.resources.Resource; +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Collections; +import java.util.Map; +import java.util.Objects; +import java.util.stream.Collectors; + +/** + * Contains information of a {@link ResourceProfile}. + */ +public class ResourceProfileInfo implements ResponseBody { + + public static final String FIELD_NAME_CPU = "cpuCores"; + + public static final String FIELD_NAME_TASK_HEAP = "taskHeapMemory"; + + public static final String FIELD_NAME_TASK_OFFHEAP = "taskOffHeapMemory"; + + public static final String FIELD_NAME_MANAGED = "managedMemory"; + + public static final String FIELD_NAME_NETWORK = "networkMemory"; + + public static final String FIELD_NAME_EXTENDED = "extendedResources"; + + @JsonProperty(FIELD_NAME_CPU) + private final double cpu; + + @JsonProperty(FIELD_NAME_TASK_HEAP) + private final int taskHeapMB; + + @JsonProperty(FIELD_NAME_TASK_OFFHEAP) + private final int taskOffHeapMB; + + @JsonProperty(FIELD_NAME_MANAGED) + private final int managedMB; + + @JsonProperty(FIELD_NAME_NETWORK) + private final int networkMB; + + @JsonProperty(FIELD_NAME_EXTENDED) + private final Map extendedResources; + + @JsonCreator + public ResourceProfileInfo( + @JsonProperty(FIELD_NAME_CPU) double cpu, + @JsonProperty(FIELD_NAME_TASK_HEAP) int taskHeapMB, + @JsonProperty(FIELD_NAME_TASK_OFFHEAP) int taskOffHeapMB, + @JsonProperty(FIELD_NAME_MANAGED) int managedMB, + @JsonProperty(FIELD_NAME_NETWORK) int networkMB, + @JsonProperty(FIELD_NAME_EXTENDED) Map extendedResources) { + this.cpu = cpu; + this.taskHeapMB = taskHeapMB; + this.taskOffHeapMB = taskOffHeapMB; + this.managedMB = managedMB; + this.networkMB = networkMB; + this.extendedResources = extendedResources; + } + + /** + * The ResourceProfile must not be UNKNOWN. + */ + private ResourceProfileInfo(ResourceProfile resourceProfile) { + this(resourceProfile.getCpuCores().getValue().doubleValue(), + resourceProfile.getTaskHeapMemory().getMebiBytes(), + resourceProfile.getTaskOffHeapMemory().getMebiBytes(), + resourceProfile.getManagedMemory().getMebiBytes(), + resourceProfile.getNetworkMemory().getMebiBytes(), + getExetendedResources(resourceProfile.getExtendedResources())); + } + + private ResourceProfileInfo() { + this(-1.0, -1, -1, -1, -1, Collections.emptyMap()); + } + + public static ResourceProfileInfo fromResrouceProfile(ResourceProfile resourceProfile) { + return resourceProfile.equals(ResourceProfile.UNKNOWN) ? new ResourceProfileInfo() : new ResourceProfileInfo(resourceProfile); + } + + private static Map getExetendedResources(Map exetendedResources) { + return exetendedResources.entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().getValue().doubleValue())); + } + + public double getCpu() { + return cpu; + } + + public int getTaskHeapMB() { + return taskHeapMB; + } + + public int getTaskOffHeapMB() { + return taskOffHeapMB; + } + + public int getManagedMB() { + return managedMB; + } + + public int getNetworkMB() { + return networkMB; + } + + public MapgetExtendedResources() { + return Collections.unmodifiableMap(extendedResources); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ResourceProfileInfo that = (ResourceProfileInfo) o; + return cpu == that.cpu && + taskHeapMB == that.taskHeapMB && + taskOffHeapMB == that.taskOffHeapMB && + managedMB == that.managedMB && + networkMB == that.networkMB && + Objects.equals(extendedResources, that.extendedResources); + } + + @Override + public int hashCode() { + return Objects.hash( + cpu, + taskHeapMB, + taskOffHeapMB, + managedMB, + networkMB, + extendedResources); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/CheckpointStatistics.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/CheckpointStatistics.java index f8aeb266aaacab6b4f912741dfa6e6021f3f20fa..86944465b6591545017e397d3dc09b8d662b72f5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/CheckpointStatistics.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/CheckpointStatistics.java @@ -165,10 +165,6 @@ public class CheckpointStatistics implements ResponseBody { return duration; } - public long getAlignmentBuffered() { - return alignmentBuffered; - } - public int getNumSubtasks() { return numSubtasks; } @@ -232,7 +228,7 @@ public class CheckpointStatistics implements ResponseBody { taskStateStat.getLatestAckTimestamp(), taskStateStat.getStateSize(), taskStateStat.getEndToEndDuration(checkpointStats.getTriggerTimestamp()), - taskStateStat.getAlignmentBuffered(), + 0, taskStateStat.getNumberOfSubtasks(), taskStateStat.getNumberOfAcknowledgedSubtasks())); } @@ -251,7 +247,7 @@ public class CheckpointStatistics implements ResponseBody { completedCheckpointStats.getLatestAckTimestamp(), completedCheckpointStats.getStateSize(), completedCheckpointStats.getEndToEndDuration(), - completedCheckpointStats.getAlignmentBuffered(), + 0, completedCheckpointStats.getNumberOfSubtasks(), completedCheckpointStats.getNumberOfAcknowledgedSubtasks(), checkpointStatisticsPerTask, @@ -268,7 +264,7 @@ public class CheckpointStatistics implements ResponseBody { failedCheckpointStats.getLatestAckTimestamp(), failedCheckpointStats.getStateSize(), failedCheckpointStats.getEndToEndDuration(), - failedCheckpointStats.getAlignmentBuffered(), + 0, failedCheckpointStats.getNumberOfSubtasks(), failedCheckpointStats.getNumberOfAcknowledgedSubtasks(), checkpointStatisticsPerTask, @@ -285,7 +281,7 @@ public class CheckpointStatistics implements ResponseBody { pendingCheckpointStats.getLatestAckTimestamp(), pendingCheckpointStats.getStateSize(), pendingCheckpointStats.getEndToEndDuration(), - pendingCheckpointStats.getAlignmentBuffered(), + 0, pendingCheckpointStats.getNumberOfSubtasks(), pendingCheckpointStats.getNumberOfAcknowledgedSubtasks(), checkpointStatisticsPerTask diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/CheckpointingStatistics.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/CheckpointingStatistics.java index 43c9248db33d907b5a14539f1786378616acc980..b0b69a767947f73b2177c564fba878fa51904242 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/CheckpointingStatistics.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/CheckpointingStatistics.java @@ -232,10 +232,6 @@ public class CheckpointingStatistics implements ResponseBody { return duration; } - public MinMaxAvgStatistics getAlignmentBuffered() { - return alignmentBuffered; - } - @Override public boolean equals(Object o) { if (this == o) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/SubtaskCheckpointStatistics.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/SubtaskCheckpointStatistics.java index 5f2e805a872f47ee6fb25c3bdbaf76311422690a..d45ceaac63da47af8b5fd5d970bcd0bbc5280342 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/SubtaskCheckpointStatistics.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/SubtaskCheckpointStatistics.java @@ -260,10 +260,6 @@ public class SubtaskCheckpointStatistics { this.alignmentDuration = alignmentDuration; } - public long getAlignmentBuffered() { - return alignmentBuffered; - } - public long getAlignmentDuration() { return alignmentDuration; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/TaskCheckpointStatistics.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/TaskCheckpointStatistics.java index efd000480dc903dbf1ab60bcf945e37510e8a40e..685f3e66944fa9519aa0a4bdc9e2ec633623ce93 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/TaskCheckpointStatistics.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/TaskCheckpointStatistics.java @@ -105,10 +105,6 @@ public class TaskCheckpointStatistics implements ResponseBody { return duration; } - public long getAlignmentBuffered() { - return alignmentBuffered; - } - public int getNumSubtasks() { return numSubtasks; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/cluster/FileMessageParameters.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/cluster/FileMessageParameters.java new file mode 100644 index 0000000000000000000000000000000000000000..a2503535e7ec25342d4dfd816269144768298244 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/cluster/FileMessageParameters.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.rest.messages.cluster; + +import org.apache.flink.runtime.rest.handler.cluster.JobManagerCustomLogHandler; +import org.apache.flink.runtime.rest.messages.LogFileNamePathParameter; +import org.apache.flink.runtime.rest.messages.MessageParameters; +import org.apache.flink.runtime.rest.messages.MessagePathParameter; +import org.apache.flink.runtime.rest.messages.MessageQueryParameter; + +import java.util.Collection; +import java.util.Collections; + +/** + * Parameters for {@link JobManagerCustomLogHandler}. + */ +public class FileMessageParameters extends MessageParameters { + + public final LogFileNamePathParameter logFileNamePathParameter = new LogFileNamePathParameter(); + + @Override + public Collection> getPathParameters() { + return Collections.singleton(logFileNamePathParameter); + } + + @Override + public Collection> getQueryParameters() { + return Collections.emptyList(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/cluster/JobManagerCustomLogHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/cluster/JobManagerCustomLogHeaders.java new file mode 100644 index 0000000000000000000000000000000000000000..8faaf95141615330782ecf3d613b0b89f8041294 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/cluster/JobManagerCustomLogHeaders.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.rest.messages.cluster; + +import org.apache.flink.runtime.rest.HttpMethodWrapper; +import org.apache.flink.runtime.rest.handler.cluster.JobManagerCustomLogHandler; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; +import org.apache.flink.runtime.rest.messages.LogFileNamePathParameter; +import org.apache.flink.runtime.rest.messages.UntypedResponseMessageHeaders; + +/** + * Headers for the {@link JobManagerCustomLogHandler}. + */ +public class JobManagerCustomLogHeaders implements UntypedResponseMessageHeaders { + + private static final JobManagerCustomLogHeaders INSTANCE = new JobManagerCustomLogHeaders(); + + private static final String URL = String.format("/jobmanager/logs/:%s", LogFileNamePathParameter.KEY); + + private JobManagerCustomLogHeaders() {} + + public static JobManagerCustomLogHeaders getInstance() { + return INSTANCE; + } + + @Override + public Class getRequestClass() { + return EmptyRequestBody.class; + } + + @Override + public FileMessageParameters getUnresolvedMessageParameters() { + return new FileMessageParameters(); + } + + @Override + public HttpMethodWrapper getHttpMethod() { + return HttpMethodWrapper.GET; + } + + @Override + public String getTargetRestEndpointURL() { + return URL; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/files/LogFileHandlerSpecification.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/cluster/JobManagerLogFileHeader.java similarity index 54% rename from flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/files/LogFileHandlerSpecification.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/cluster/JobManagerLogFileHeader.java index 30f40684a06b2b545f64f6922a4b3d5a28146fd1..42e5da96bbc395a028d543659086b9a7295d5eeb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/files/LogFileHandlerSpecification.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/cluster/JobManagerLogFileHeader.java @@ -16,21 +16,34 @@ * limitations under the License. */ -package org.apache.flink.runtime.rest.handler.legacy.files; +package org.apache.flink.runtime.rest.messages.cluster; import org.apache.flink.runtime.rest.HttpMethodWrapper; -import org.apache.flink.runtime.rest.handler.RestHandlerSpecification; +import org.apache.flink.runtime.rest.handler.cluster.JobManagerLogFileHandler; +import org.apache.flink.runtime.rest.messages.EmptyMessageParameters; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; +import org.apache.flink.runtime.rest.messages.UntypedResponseMessageHeaders; /** - * Rest handler specification for the log file of the main cluster component. + * Headers for the {@link JobManagerLogFileHandler}. */ -public class LogFileHandlerSpecification implements RestHandlerSpecification { +public class JobManagerLogFileHeader implements UntypedResponseMessageHeaders { - private static final LogFileHandlerSpecification INSTANCE = new LogFileHandlerSpecification(); + private static final JobManagerLogFileHeader INSTANCE = new JobManagerLogFileHeader(); private static final String URL = "/jobmanager/log"; - private LogFileHandlerSpecification() {} + private JobManagerLogFileHeader() {} + + @Override + public Class getRequestClass() { + return EmptyRequestBody.class; + } + + @Override + public EmptyMessageParameters getUnresolvedMessageParameters() { + return EmptyMessageParameters.getInstance(); + } @Override public HttpMethodWrapper getHttpMethod() { @@ -42,7 +55,7 @@ public class LogFileHandlerSpecification implements RestHandlerSpecification { return URL; } - public static LogFileHandlerSpecification getInstance() { + public static JobManagerLogFileHeader getInstance() { return INSTANCE; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/ConstantTextHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/cluster/JobManagerLogListHeaders.java similarity index 36% rename from flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/ConstantTextHandler.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/cluster/JobManagerLogListHeaders.java index 490a3c4a248f97047c91d201a06ae3ec4fb5be05..477f26feab4f51995b2d6fc4ba2152f4fba0a429 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/ConstantTextHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/cluster/JobManagerLogListHeaders.java @@ -16,42 +16,64 @@ * limitations under the License. */ -package org.apache.flink.runtime.rest.handler.legacy; - -import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.runtime.rest.handler.router.RoutedRequest; -import org.apache.flink.runtime.rest.handler.util.KeepAliveWrite; - -import org.apache.flink.shaded.netty4.io.netty.buffer.Unpooled; -import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler; -import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext; -import org.apache.flink.shaded.netty4.io.netty.channel.SimpleChannelInboundHandler; -import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.DefaultFullHttpResponse; -import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders; -import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponse; +package org.apache.flink.runtime.rest.messages.cluster; + +import org.apache.flink.runtime.rest.HttpMethodWrapper; +import org.apache.flink.runtime.rest.handler.cluster.JobManagerLogListHandler; +import org.apache.flink.runtime.rest.messages.EmptyMessageParameters; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; +import org.apache.flink.runtime.rest.messages.LogListInfo; +import org.apache.flink.runtime.rest.messages.MessageHeaders; + import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; -import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpVersion; /** - * Responder that returns a constant String. + * Headers for the {@link JobManagerLogListHandler}. */ -@ChannelHandler.Sharable -public class ConstantTextHandler extends SimpleChannelInboundHandler { +public class JobManagerLogListHeaders implements MessageHeaders { + + private static final JobManagerLogListHeaders INSTANCE = new JobManagerLogListHeaders(); + + private static final String URL = "/jobmanager/logs"; + + private JobManagerLogListHeaders() {} + + public static JobManagerLogListHeaders getInstance() { + return INSTANCE; + } + + @Override + public Class getResponseClass() { + return LogListInfo.class; + } - private final byte[] encodedText; + @Override + public HttpResponseStatus getResponseStatusCode() { + return HttpResponseStatus.OK; + } + + @Override + public String getDescription() { + return "Returns the list of log files on the JobManager."; + } - public ConstantTextHandler(String text) { - this.encodedText = text.getBytes(ConfigConstants.DEFAULT_CHARSET); + @Override + public Class getRequestClass() { + return EmptyRequestBody.class; } @Override - protected void channelRead0(ChannelHandlerContext ctx, RoutedRequest routed) throws Exception { - HttpResponse response = new DefaultFullHttpResponse( - HttpVersion.HTTP_1_1, HttpResponseStatus.OK, Unpooled.wrappedBuffer(encodedText)); + public EmptyMessageParameters getUnresolvedMessageParameters() { + return EmptyMessageParameters.getInstance(); + } - response.headers().set(HttpHeaders.Names.CONTENT_LENGTH, encodedText.length); - response.headers().set(HttpHeaders.Names.CONTENT_TYPE, "text/plain"); + @Override + public HttpMethodWrapper getHttpMethod() { + return HttpMethodWrapper.GET; + } - KeepAliveWrite.flush(ctx, routed.getRequest(), response); + @Override + public String getTargetRestEndpointURL() { + return URL; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/files/StdoutFileHandlerSpecification.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/cluster/JobManagerStdoutFileHeader.java similarity index 56% rename from flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/files/StdoutFileHandlerSpecification.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/cluster/JobManagerStdoutFileHeader.java index 52d56f21a51d22bf44915a73e64a81f1a5ecbdc1..7259fabf5db2a51ccdf55fd720949a581deb9646 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/files/StdoutFileHandlerSpecification.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/cluster/JobManagerStdoutFileHeader.java @@ -16,21 +16,33 @@ * limitations under the License. */ -package org.apache.flink.runtime.rest.handler.legacy.files; +package org.apache.flink.runtime.rest.messages.cluster; import org.apache.flink.runtime.rest.HttpMethodWrapper; -import org.apache.flink.runtime.rest.handler.RestHandlerSpecification; +import org.apache.flink.runtime.rest.messages.EmptyMessageParameters; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; +import org.apache.flink.runtime.rest.messages.UntypedResponseMessageHeaders; /** - * Rest handler specification for the stdout file of the main cluster component. + * Headers for the {@link JobManagerStdoutFileHandler}. */ -public class StdoutFileHandlerSpecification implements RestHandlerSpecification { +public class JobManagerStdoutFileHeader implements UntypedResponseMessageHeaders { - private static final StdoutFileHandlerSpecification INSTANCE = new StdoutFileHandlerSpecification(); + private static final JobManagerStdoutFileHeader INSTANCE = new JobManagerStdoutFileHeader(); private static final String URL = "/jobmanager/stdout"; - private StdoutFileHandlerSpecification() {} + private JobManagerStdoutFileHeader() {} + + @Override + public Class getRequestClass() { + return EmptyRequestBody.class; + } + + @Override + public EmptyMessageParameters getUnresolvedMessageParameters() { + return EmptyMessageParameters.getInstance(); + } @Override public HttpMethodWrapper getHttpMethod() { @@ -42,7 +54,7 @@ public class StdoutFileHandlerSpecification implements RestHandlerSpecification return URL; } - public static StdoutFileHandlerSpecification getInstance() { + public static JobManagerStdoutFileHeader getInstance() { return INSTANCE; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/dataset/ClusterDataSetDeleteStatusHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/dataset/ClusterDataSetDeleteStatusHeaders.java new file mode 100644 index 0000000000000000000000000000000000000000..138959eb80eae0047f2b94966a44312355cc1e6e --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/dataset/ClusterDataSetDeleteStatusHeaders.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.rest.messages.dataset; + +import org.apache.flink.runtime.rest.HttpMethodWrapper; +import org.apache.flink.runtime.rest.handler.async.AsynchronousOperationInfo; +import org.apache.flink.runtime.rest.handler.async.AsynchronousOperationStatusMessageHeaders; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; +import org.apache.flink.runtime.rest.messages.TriggerIdPathParameter; + +import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; + +/** + * Specification for retrieving the status for the delete operation of a cluster data set. + * + * @see ClusterDataSetDeleteStatusHeaders + */ +public class ClusterDataSetDeleteStatusHeaders extends AsynchronousOperationStatusMessageHeaders { + + public static final ClusterDataSetDeleteStatusHeaders INSTANCE = new ClusterDataSetDeleteStatusHeaders(); + + private static final String URL = ClusterDataSetListHeaders.URL + "/delete/:" + TriggerIdPathParameter.KEY; + + private ClusterDataSetDeleteStatusHeaders() { + } + + @Override + public HttpResponseStatus getResponseStatusCode() { + return HttpResponseStatus.OK; + } + + @Override + public String getDescription() { + return "Returns the status for the delete operation of a cluster data set."; + } + + @Override + public Class getValueClass() { + return AsynchronousOperationInfo.class; + } + + @Override + public Class getRequestClass() { + return EmptyRequestBody.class; + } + + @Override + public ClusterDataSetDeleteStatusMessageParameters getUnresolvedMessageParameters() { + return new ClusterDataSetDeleteStatusMessageParameters(); + } + + @Override + public HttpMethodWrapper getHttpMethod() { + return HttpMethodWrapper.GET; + } + + @Override + public String getTargetRestEndpointURL() { + return URL; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/dataset/ClusterDataSetDeleteStatusMessageParameters.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/dataset/ClusterDataSetDeleteStatusMessageParameters.java new file mode 100644 index 0000000000000000000000000000000000000000..18d14f2e4309b5bfa7e0cabd54009d4043990ab4 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/dataset/ClusterDataSetDeleteStatusMessageParameters.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.rest.messages.dataset; + +import org.apache.flink.runtime.rest.messages.MessageParameters; +import org.apache.flink.runtime.rest.messages.MessagePathParameter; +import org.apache.flink.runtime.rest.messages.MessageQueryParameter; +import org.apache.flink.runtime.rest.messages.TriggerIdPathParameter; + +import java.util.Collection; +import java.util.Collections; + +/** + * {@link MessageParameters} for the {@link ClusterDataSetDeleteStatusHeaders}. + */ +public class ClusterDataSetDeleteStatusMessageParameters extends MessageParameters { + + public final TriggerIdPathParameter triggerIdPathParameter = new TriggerIdPathParameter(); + + @Override + public Collection> getPathParameters() { + return Collections.singleton(triggerIdPathParameter); + } + + @Override + public Collection> getQueryParameters() { + return Collections.emptyList(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/dataset/ClusterDataSetDeleteTriggerHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/dataset/ClusterDataSetDeleteTriggerHeaders.java new file mode 100644 index 0000000000000000000000000000000000000000..9eadb98ad061f1f64a91c2bf2002101053d39cf7 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/dataset/ClusterDataSetDeleteTriggerHeaders.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.rest.messages.dataset; + +import org.apache.flink.runtime.rest.HttpMethodWrapper; +import org.apache.flink.runtime.rest.handler.async.AsynchronousOperationTriggerMessageHeaders; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; + +import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; + +/** + * Specification for triggering the deletion of a cluster data set. + * + * @see ClusterDataSetDeleteStatusHeaders + */ +public class ClusterDataSetDeleteTriggerHeaders extends AsynchronousOperationTriggerMessageHeaders { + + public static final ClusterDataSetDeleteTriggerHeaders INSTANCE = new ClusterDataSetDeleteTriggerHeaders(); + + private static final String URL = ClusterDataSetListHeaders.URL + "/:" + ClusterDataSetIdPathParameter.KEY; + + private ClusterDataSetDeleteTriggerHeaders() { + } + + @Override + public HttpResponseStatus getResponseStatusCode() { + return HttpResponseStatus.ACCEPTED; + } + + @Override + protected String getAsyncOperationDescription() { + return "Triggers the deletion of a cluster data set."; + } + + @Override + public Class getRequestClass() { + return EmptyRequestBody.class; + } + + @Override + public ClusterDataSetDeleteTriggerMessageParameters getUnresolvedMessageParameters() { + return new ClusterDataSetDeleteTriggerMessageParameters(); + } + + @Override + public HttpMethodWrapper getHttpMethod() { + return HttpMethodWrapper.DELETE; + } + + @Override + public String getTargetRestEndpointURL() { + return URL; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/dataset/ClusterDataSetDeleteTriggerMessageParameters.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/dataset/ClusterDataSetDeleteTriggerMessageParameters.java new file mode 100644 index 0000000000000000000000000000000000000000..1b40be24afcf46b72328c0dbe29fc1258caee569 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/dataset/ClusterDataSetDeleteTriggerMessageParameters.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.rest.messages.dataset; + +import org.apache.flink.runtime.rest.messages.MessageParameters; +import org.apache.flink.runtime.rest.messages.MessagePathParameter; +import org.apache.flink.runtime.rest.messages.MessageQueryParameter; + +import java.util.Collection; +import java.util.Collections; + +/** + * {@link MessageParameters} for the {@link ClusterDataSetDeleteTriggerHeaders}. + */ +public class ClusterDataSetDeleteTriggerMessageParameters extends MessageParameters { + + public final ClusterDataSetIdPathParameter clusterDataSetIdPathParameter = new ClusterDataSetIdPathParameter(); + + @Override + public Collection> getPathParameters() { + return Collections.singleton(clusterDataSetIdPathParameter); + } + + @Override + public Collection> getQueryParameters() { + return Collections.emptyList(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/dataset/ClusterDataSetEntry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/dataset/ClusterDataSetEntry.java new file mode 100644 index 0000000000000000000000000000000000000000..1bc4c9089f7578ad29492348f1ea2825d662ec16 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/dataset/ClusterDataSetEntry.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.rest.messages.dataset; + +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnore; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + +/** + * The entry for a single cluster data set. + * + * @see ClusterDataSetListResponseBody + */ +class ClusterDataSetEntry { + private static final String FIELD_NAME_DATA_SET_ID = "id"; + private static final String FIELD_NAME_COMPLETE = "isComplete"; + + @JsonProperty(FIELD_NAME_DATA_SET_ID) + private final String dataSetId; + + @JsonProperty(FIELD_NAME_COMPLETE) + private final boolean isComplete; + + ClusterDataSetEntry(IntermediateDataSetID dataSetId, boolean isComplete) { + this(dataSetId.toHexString(), isComplete); + } + + @JsonCreator + private ClusterDataSetEntry( + @JsonProperty(FIELD_NAME_DATA_SET_ID) String dataSetId, + @JsonProperty(FIELD_NAME_COMPLETE) boolean isComplete) { + this.dataSetId = dataSetId; + this.isComplete = isComplete; + } + + @JsonIgnore + public String getDataSetId() { + return dataSetId; + } + + @JsonIgnore + public boolean isComplete() { + return isComplete; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/dataset/ClusterDataSetIdPathParameter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/dataset/ClusterDataSetIdPathParameter.java new file mode 100644 index 0000000000000000000000000000000000000000..ab3637d7b5420598540f0849706853d684e4732a --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/dataset/ClusterDataSetIdPathParameter.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.rest.messages.dataset; + +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; +import org.apache.flink.runtime.rest.messages.MessagePathParameter; +import org.apache.flink.util.AbstractID; +import org.apache.flink.util.StringUtils; + +/** + * Path parameter identifying cluster data sets. + */ +public class ClusterDataSetIdPathParameter extends MessagePathParameter { + + public static final String KEY = "datasetid"; + + public ClusterDataSetIdPathParameter() { + super(KEY); + } + + @Override + protected IntermediateDataSetID convertFromString(String value) { + return new IntermediateDataSetID(new AbstractID(StringUtils.hexStringToByte(value))); + } + + @Override + protected String convertToString(IntermediateDataSetID value) { + return value.toString(); + } + + @Override + public String getDescription() { + return "32-character hexadecimal string value that identifies a cluster data set."; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/dataset/ClusterDataSetListHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/dataset/ClusterDataSetListHeaders.java new file mode 100644 index 0000000000000000000000000000000000000000..85eec34b15b1e2ef5f26c208a250b614a6970f3c --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/dataset/ClusterDataSetListHeaders.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.rest.messages.dataset; + +import org.apache.flink.runtime.rest.HttpMethodWrapper; +import org.apache.flink.runtime.rest.messages.EmptyMessageParameters; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; +import org.apache.flink.runtime.rest.messages.MessageHeaders; + +import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; + +/** + * Specification for retrieving an overview over all available cluster partitions. + */ +public class ClusterDataSetListHeaders implements MessageHeaders { + + public static final ClusterDataSetListHeaders INSTANCE = new ClusterDataSetListHeaders(); + + static final String URL = "/datasets"; + + private ClusterDataSetListHeaders() { + } + + @Override + public Class getResponseClass() { + return ClusterDataSetListResponseBody.class; + } + + @Override + public HttpResponseStatus getResponseStatusCode() { + return HttpResponseStatus.OK; + } + + @Override + public String getDescription() { + return "Returns all cluster data sets."; + } + + @Override + public Class getRequestClass() { + return EmptyRequestBody.class; + } + + @Override + public EmptyMessageParameters getUnresolvedMessageParameters() { + return EmptyMessageParameters.getInstance(); + } + + @Override + public HttpMethodWrapper getHttpMethod() { + return HttpMethodWrapper.GET; + } + + @Override + public String getTargetRestEndpointURL() { + return URL; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/dataset/ClusterDataSetListResponseBody.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/dataset/ClusterDataSetListResponseBody.java new file mode 100644 index 0000000000000000000000000000000000000000..957a0c316895685b067e4b9de932ec625b1eac97 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/dataset/ClusterDataSetListResponseBody.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.rest.messages.dataset; + +import org.apache.flink.runtime.io.network.partition.DataSetMetaInfo; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; +import org.apache.flink.runtime.rest.messages.ResponseBody; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnore; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +/** + * {@link ResponseBody} for {@link ClusterDataSetListHeaders}. + */ +public class ClusterDataSetListResponseBody implements ResponseBody { + private static final String FIELD_NAME_PARTITIONS = "dataSets"; + + @JsonProperty(FIELD_NAME_PARTITIONS) + private final List dataSets; + + @JsonCreator + private ClusterDataSetListResponseBody(@JsonProperty(FIELD_NAME_PARTITIONS) List dataSets) { + this.dataSets = dataSets; + } + + public static ClusterDataSetListResponseBody from(Map dataSets) { + final List convertedInfo = dataSets.entrySet().stream() + .map(entry -> { + DataSetMetaInfo metaInfo = entry.getValue(); + int numRegisteredPartitions = metaInfo.getNumRegisteredPartitions().orElse(0); + int numTotalPartition = metaInfo.getNumTotalPartitions(); + return new ClusterDataSetEntry(entry.getKey(), numRegisteredPartitions == numTotalPartition); + }) + .collect(Collectors.toList()); + return new ClusterDataSetListResponseBody(convertedInfo); + } + + @JsonIgnore + public List getDataSets() { + return dataSets; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointDisposalStatusHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointDisposalStatusHeaders.java index 5759cbdcfa57529af9bd8f808a1fd082fd245fa4..9f278175c71a38f9907b36e29fe783642a18487c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointDisposalStatusHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointDisposalStatusHeaders.java @@ -69,7 +69,7 @@ public class SavepointDisposalStatusHeaders extends AsynchronousOperationStatusM } @Override - protected Class getValueClass() { + public Class getValueClass() { return AsynchronousOperationInfo.class; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointStatusHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointStatusHeaders.java index d9215b9d6f9ab9220cbe62ebd521fb4db7a57fb5..645b31f572dfba96d0625575fd9dc5f5835a4b8f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointStatusHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointStatusHeaders.java @@ -72,7 +72,7 @@ public class SavepointStatusHeaders } @Override - protected Class getValueClass() { + public Class getValueClass() { return SavepointInfo.class; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerCustomLogHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerCustomLogHeaders.java new file mode 100644 index 0000000000000000000000000000000000000000..b1480193d778c47154c34ff0ec2378dd4c90f509 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerCustomLogHeaders.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.rest.messages.taskmanager; + +import org.apache.flink.runtime.rest.HttpMethodWrapper; +import org.apache.flink.runtime.rest.handler.taskmanager.TaskManagerCustomLogHandler; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; +import org.apache.flink.runtime.rest.messages.LogFileNamePathParameter; +import org.apache.flink.runtime.rest.messages.UntypedResponseMessageHeaders; + +/** + * Headers for the {@link TaskManagerCustomLogHandler}. + */ +public class TaskManagerCustomLogHeaders implements UntypedResponseMessageHeaders { + + private static final TaskManagerCustomLogHeaders INSTANCE = new TaskManagerCustomLogHeaders(); + + private static final String URL = String.format("/taskmanagers/:%s/logs/:%s", TaskManagerIdPathParameter.KEY, LogFileNamePathParameter.KEY); + + @Override + public Class getRequestClass() { + return EmptyRequestBody.class; + } + + @Override + public TaskManagerFileMessageParameters getUnresolvedMessageParameters() { + return new TaskManagerFileMessageParameters(); + } + + @Override + public HttpMethodWrapper getHttpMethod() { + return HttpMethodWrapper.GET; + } + + @Override + public String getTargetRestEndpointURL() { + return URL; + } + + public static TaskManagerCustomLogHeaders getInstance() { + return INSTANCE; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerDetailsInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerDetailsInfo.java index f830ac889af6f96b3c1eca4483dc64d38d4a6798..70dd0c9ac9459e751fc43eb1d3aff73df978bcfc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerDetailsInfo.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerDetailsInfo.java @@ -20,6 +20,7 @@ package org.apache.flink.runtime.rest.messages.taskmanager; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.instance.HardwareDescription; +import org.apache.flink.runtime.rest.messages.ResourceProfileInfo; import org.apache.flink.runtime.rest.messages.json.ResourceIDDeserializer; import org.apache.flink.runtime.taskexecutor.TaskExecutor; import org.apache.flink.util.Preconditions; @@ -49,6 +50,8 @@ public class TaskManagerDetailsInfo extends TaskManagerInfo { @JsonProperty(FIELD_NAME_LAST_HEARTBEAT) long lastHeartbeat, @JsonProperty(FIELD_NAME_NUMBER_SLOTS) int numberSlots, @JsonProperty(FIELD_NAME_NUMBER_AVAILABLE_SLOTS) int numberAvailableSlots, + @JsonProperty(FIELD_NAME_TOTAL_RESOURCE) ResourceProfileInfo totalResource, + @JsonProperty(FIELD_NAME_AVAILABLE_RESOURCE) ResourceProfileInfo freeResource, @JsonProperty(FIELD_NAME_HARDWARE) HardwareDescription hardwareDescription, @JsonProperty(FIELD_NAME_METRICS) TaskManagerMetricsInfo taskManagerMetrics) { super( @@ -58,6 +61,8 @@ public class TaskManagerDetailsInfo extends TaskManagerInfo { lastHeartbeat, numberSlots, numberAvailableSlots, + totalResource, + freeResource, hardwareDescription); this.taskManagerMetrics = Preconditions.checkNotNull(taskManagerMetrics); @@ -71,6 +76,8 @@ public class TaskManagerDetailsInfo extends TaskManagerInfo { taskManagerInfo.getLastHeartbeat(), taskManagerInfo.getNumberSlots(), taskManagerInfo.getNumberAvailableSlots(), + taskManagerInfo.getTotalResource(), + taskManagerInfo.getFreeResource(), taskManagerInfo.getHardwareDescription(), taskManagerMetrics); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerFileMessageParameters.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerFileMessageParameters.java new file mode 100644 index 0000000000000000000000000000000000000000..3cde97c3eebe2e25c9ee6492f55a28f6e3c1b7f7 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerFileMessageParameters.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.rest.messages.taskmanager; + +import org.apache.flink.runtime.rest.handler.taskmanager.TaskManagerCustomLogHandler; +import org.apache.flink.runtime.rest.messages.LogFileNamePathParameter; +import org.apache.flink.runtime.rest.messages.MessagePathParameter; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; + +/** + * Parameters for {@link TaskManagerCustomLogHandler}. + */ +public class TaskManagerFileMessageParameters extends TaskManagerMessageParameters { + + public final LogFileNamePathParameter logFileNamePathParameter = new LogFileNamePathParameter(); + + @Override + public Collection> getPathParameters() { + return Collections.unmodifiableCollection(Arrays.asList( + logFileNamePathParameter, + taskManagerIdParameter + )); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerInfo.java index 9ce9f76afe9709e4ecb50b76ff277b1bb2801fec..954e9cb9b73a0225535a461c59dcf75ad11409f2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerInfo.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerInfo.java @@ -19,7 +19,9 @@ package org.apache.flink.runtime.rest.messages.taskmanager; import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.instance.HardwareDescription; +import org.apache.flink.runtime.rest.messages.ResourceProfileInfo; import org.apache.flink.runtime.rest.messages.ResponseBody; import org.apache.flink.runtime.rest.messages.json.ResourceIDDeserializer; import org.apache.flink.runtime.rest.messages.json.ResourceIDSerializer; @@ -51,6 +53,10 @@ public class TaskManagerInfo implements ResponseBody, Serializable { public static final String FIELD_NAME_NUMBER_AVAILABLE_SLOTS = "freeSlots"; + public static final String FIELD_NAME_TOTAL_RESOURCE = "totalResource"; + + public static final String FIELD_NAME_AVAILABLE_RESOURCE = "freeResource"; + public static final String FIELD_NAME_HARDWARE = "hardware"; private static final long serialVersionUID = 1L; @@ -74,6 +80,12 @@ public class TaskManagerInfo implements ResponseBody, Serializable { @JsonProperty(FIELD_NAME_NUMBER_AVAILABLE_SLOTS) private final int numberAvailableSlots; + @JsonProperty(FIELD_NAME_TOTAL_RESOURCE) + private final ResourceProfileInfo totalResource; + + @JsonProperty(FIELD_NAME_AVAILABLE_RESOURCE) + private final ResourceProfileInfo freeResource; + @JsonProperty(FIELD_NAME_HARDWARE) private final HardwareDescription hardwareDescription; @@ -85,6 +97,8 @@ public class TaskManagerInfo implements ResponseBody, Serializable { @JsonProperty(FIELD_NAME_LAST_HEARTBEAT) long lastHeartbeat, @JsonProperty(FIELD_NAME_NUMBER_SLOTS) int numberSlots, @JsonProperty(FIELD_NAME_NUMBER_AVAILABLE_SLOTS) int numberAvailableSlots, + @JsonProperty(FIELD_NAME_TOTAL_RESOURCE) ResourceProfileInfo totalResource, + @JsonProperty(FIELD_NAME_AVAILABLE_RESOURCE) ResourceProfileInfo freeResource, @JsonProperty(FIELD_NAME_HARDWARE) HardwareDescription hardwareDescription) { this.resourceId = Preconditions.checkNotNull(resourceId); this.address = Preconditions.checkNotNull(address); @@ -92,9 +106,32 @@ public class TaskManagerInfo implements ResponseBody, Serializable { this.lastHeartbeat = lastHeartbeat; this.numberSlots = numberSlots; this.numberAvailableSlots = numberAvailableSlots; + this.totalResource = totalResource; + this.freeResource = freeResource; this.hardwareDescription = Preconditions.checkNotNull(hardwareDescription); } + public TaskManagerInfo( + ResourceID resourceId, + String address, + int dataPort, + long lastHeartbeat, + int numberSlots, + int numberAvailableSlots, + ResourceProfile totalResource, + ResourceProfile freeResource, + HardwareDescription hardwareDescription) { + this(resourceId, + address, + dataPort, + lastHeartbeat, + numberSlots, + numberAvailableSlots, + ResourceProfileInfo.fromResrouceProfile(totalResource), + ResourceProfileInfo.fromResrouceProfile(freeResource), + hardwareDescription); + } + public ResourceID getResourceId() { return resourceId; } @@ -119,6 +156,14 @@ public class TaskManagerInfo implements ResponseBody, Serializable { return numberAvailableSlots; } + public ResourceProfileInfo getTotalResource() { + return totalResource; + } + + public ResourceProfileInfo getFreeResource() { + return freeResource; + } + public HardwareDescription getHardwareDescription() { return hardwareDescription; } @@ -136,6 +181,8 @@ public class TaskManagerInfo implements ResponseBody, Serializable { lastHeartbeat == that.lastHeartbeat && numberSlots == that.numberSlots && numberAvailableSlots == that.numberAvailableSlots && + Objects.equals(totalResource, that.totalResource) && + Objects.equals(freeResource, that.freeResource) && Objects.equals(resourceId, that.resourceId) && Objects.equals(address, that.address) && Objects.equals(hardwareDescription, that.hardwareDescription); @@ -150,6 +197,8 @@ public class TaskManagerInfo implements ResponseBody, Serializable { lastHeartbeat, numberSlots, numberAvailableSlots, + totalResource, + freeResource, hardwareDescription); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerLogsHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerLogsHeaders.java new file mode 100644 index 0000000000000000000000000000000000000000..d666ef07468b822aeb93eb123bf04b1b9c81690b --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerLogsHeaders.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.rest.messages.taskmanager; + +import org.apache.flink.runtime.rest.HttpMethodWrapper; +import org.apache.flink.runtime.rest.handler.taskmanager.TaskManagerLogListHandler; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; +import org.apache.flink.runtime.rest.messages.LogListInfo; +import org.apache.flink.runtime.rest.messages.MessageHeaders; + +import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; + +/** + * Headers for the {@link TaskManagerLogListHandler}. + */ +public class TaskManagerLogsHeaders implements MessageHeaders { + + private static final TaskManagerLogsHeaders INSTANCE = new TaskManagerLogsHeaders(); + + private static final String URL = String.format("/taskmanagers/:%s/logs", TaskManagerIdPathParameter.KEY); + + private TaskManagerLogsHeaders() {} + + public static TaskManagerLogsHeaders getInstance() { + return INSTANCE; + } + + @Override + public Class getResponseClass() { + return LogListInfo.class; + } + + @Override + public HttpResponseStatus getResponseStatusCode() { + return HttpResponseStatus.OK; + } + + @Override + public String getDescription() { + return "Returns the list of log files on a TaskManager."; + } + + @Override + public Class getRequestClass() { + return EmptyRequestBody.class; + } + + @Override + public TaskManagerMessageParameters getUnresolvedMessageParameters() { + return new TaskManagerMessageParameters(); + } + + @Override + public HttpMethodWrapper getHttpMethod() { + return HttpMethodWrapper.GET; + } + + @Override + public String getTargetRestEndpointURL() { + return URL; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerMessageParameters.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerMessageParameters.java index 4a761215bc102d7424f9012c26e3953f3ff8d47d..aafe4eee191667ddfd39d461b102a3ec38a84003 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerMessageParameters.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerMessageParameters.java @@ -31,7 +31,7 @@ import java.util.Collections; */ public class TaskManagerMessageParameters extends MessageParameters { - public TaskManagerIdPathParameter taskManagerIdParameter = new TaskManagerIdPathParameter(); + public final TaskManagerIdPathParameter taskManagerIdParameter = new TaskManagerIdPathParameter(); @Override public Collection> getPathParameters() { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcUtils.java index 1378d6f9fc1ea8e3f3bbb852208a79d80102f346..5f86e8f4780f26cb10784b04ffb46b39d6c367ab 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcUtils.java @@ -20,8 +20,11 @@ package org.apache.flink.runtime.rpc; import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.util.AutoCloseableAsync; +import java.time.Duration; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.HashSet; import java.util.Set; @@ -29,6 +32,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; /** * Utility functions for Flink's RPC implementation. @@ -42,6 +46,8 @@ public class RpcUtils { */ public static final Time INF_TIMEOUT = Time.seconds(21474835); + public static final Duration INF_DURATION = Duration.ofSeconds(21474835); + /** * Extracts all {@link RpcGateway} interfaces implemented by the given clazz. * @@ -77,6 +83,21 @@ public class RpcUtils { rpcEndpoint.closeAsync().get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); } + /** + * Shuts the given {@link RpcEndpoint RpcEndpoints} down and waits for their termination. + * + * @param rpcEndpoints to shut down + * @param timeout for this operation + * @throws InterruptedException if the operation has been interrupted + * @throws ExecutionException if a problem occurred + * @throws TimeoutException if a timeout occurred + */ + public static void terminateRpcEndpoints( + Time timeout, + RpcEndpoint... rpcEndpoints) throws InterruptedException, ExecutionException, TimeoutException { + terminateAsyncCloseables(Arrays.asList(rpcEndpoints), timeout); + } + /** * Shuts the given rpc service down and waits for its termination. * @@ -102,11 +123,19 @@ public class RpcUtils { public static void terminateRpcServices( Time timeout, RpcService... rpcServices) throws InterruptedException, ExecutionException, TimeoutException { - final Collection> terminationFutures = new ArrayList<>(rpcServices.length); + terminateAsyncCloseables( + Arrays.stream(rpcServices) + .map(rpcService -> (AutoCloseableAsync) rpcService::stopService) + .collect(Collectors.toList()), + timeout); + } + + private static void terminateAsyncCloseables(Collection closeables, Time timeout) throws InterruptedException, ExecutionException, TimeoutException { + final Collection> terminationFutures = new ArrayList<>(closeables.size()); - for (RpcService service : rpcServices) { - if (service != null) { - terminationFutures.add(service.stopService()); + for (AutoCloseableAsync closeableAsync : closeables) { + if (closeableAsync != null) { + terminationFutures.add(closeableAsync.closeAsync()); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java index 25fb38b24f37606376a5ae1076c99bca3a466f21..7ad37762fc4d2d6b5010c3003a2aea977f08cb6a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java @@ -119,7 +119,7 @@ class AkkaRpcActor extends AbstractActor { this.rpcEndpointTerminationResult = RpcEndpointTerminationResult.failure( new AkkaRpcException( String.format("RpcEndpoint %s has not been properly stopped.", rpcEndpoint.getEndpointId()))); - this.state = StoppedState.INSTANCE; + this.state = StoppedState.STOPPED; } @Override @@ -164,18 +164,23 @@ class AkkaRpcActor extends AbstractActor { } private void handleControlMessage(ControlMessages controlMessage) { - switch (controlMessage) { - case START: - state = state.start(this); - break; - case STOP: - state = state.stop(); - break; - case TERMINATE: - state.terminate(this); - break; - default: - handleUnknownControlMessage(controlMessage); + try { + switch (controlMessage) { + case START: + state = state.start(this); + break; + case STOP: + state = state.stop(); + break; + case TERMINATE: + state = state.terminate(this); + break; + default: + handleUnknownControlMessage(controlMessage); + } + } catch (Exception e) { + this.rpcEndpointTerminationResult = RpcEndpointTerminationResult.failure(e); + throw e; } } @@ -462,19 +467,19 @@ class AkkaRpcActor extends AbstractActor { interface State { default State start(AkkaRpcActor akkaRpcActor) { - throw new AkkaRpcInvalidStateException(invalidStateTransitionMessage(StartedState.INSTANCE)); + throw new AkkaRpcInvalidStateException(invalidStateTransitionMessage(StartedState.STARTED)); } default State stop() { - throw new AkkaRpcInvalidStateException(invalidStateTransitionMessage(StoppedState.INSTANCE)); + throw new AkkaRpcInvalidStateException(invalidStateTransitionMessage(StoppedState.STOPPED)); } default State terminate(AkkaRpcActor akkaRpcActor) { - throw new AkkaRpcInvalidStateException(invalidStateTransitionMessage(TerminatingState.INSTANCE)); + throw new AkkaRpcInvalidStateException(invalidStateTransitionMessage(TerminatingState.TERMINATING)); } default State finishTermination() { - return TerminatedState.INSTANCE; + return TerminatedState.TERMINATED; } default boolean isRunning() { @@ -488,16 +493,16 @@ class AkkaRpcActor extends AbstractActor { @SuppressWarnings("Singleton") enum StartedState implements State { - INSTANCE; + STARTED; @Override public State start(AkkaRpcActor akkaRpcActor) { - return INSTANCE; + return STARTED; } @Override public State stop() { - return StoppedState.INSTANCE; + return StoppedState.STOPPED; } @Override @@ -523,7 +528,7 @@ class AkkaRpcActor extends AbstractActor { terminationFuture.whenComplete((ignored, throwable) -> akkaRpcActor.stop(RpcEndpointTerminationResult.of(throwable))); - return TerminatingState.INSTANCE; + return TerminatingState.TERMINATING; } @Override @@ -534,7 +539,7 @@ class AkkaRpcActor extends AbstractActor { @SuppressWarnings("Singleton") enum StoppedState implements State { - INSTANCE; + STOPPED; @Override public State start(AkkaRpcActor akkaRpcActor) { @@ -552,25 +557,30 @@ class AkkaRpcActor extends AbstractActor { akkaRpcActor.mainThreadValidator.exitMainThread(); } - return StartedState.INSTANCE; + return StartedState.STARTED; } @Override public State stop() { - return INSTANCE; + return STOPPED; } @Override public State terminate(AkkaRpcActor akkaRpcActor) { akkaRpcActor.stop(RpcEndpointTerminationResult.success()); - return TerminatingState.INSTANCE; + return TerminatingState.TERMINATING; } } @SuppressWarnings("Singleton") enum TerminatingState implements State { - INSTANCE; + TERMINATING; + + @Override + public State terminate(AkkaRpcActor akkaRpcActor) { + return TERMINATING; + } @Override public boolean isRunning() { @@ -579,7 +589,7 @@ class AkkaRpcActor extends AbstractActor { } enum TerminatedState implements State { - INSTANCE + TERMINATED } private static final class RpcEndpointTerminationResult { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java index 0c41f053e68acb11c7f0a9e781606ffeaa9b7ebe..ae4dfcc6c06a97498680cf57b5977e4fa5eb015a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.rpc.akka; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.concurrent.FutureUtils; @@ -31,16 +32,20 @@ import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.RpcServer; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.RpcUtils; +import org.apache.flink.runtime.rpc.akka.exceptions.AkkaRpcRuntimeException; import org.apache.flink.runtime.rpc.exceptions.RpcConnectionException; import org.apache.flink.runtime.rpc.messages.HandshakeSuccessMessage; import org.apache.flink.runtime.rpc.messages.RemoteHandshakeMessage; +import org.apache.flink.runtime.util.ExecutorThreadFactory; +import org.apache.flink.util.AutoCloseableAsync; +import org.apache.flink.util.ExecutorUtils; +import org.apache.flink.util.TimeUtils; -import akka.actor.ActorIdentity; +import akka.actor.AbstractActor; import akka.actor.ActorRef; import akka.actor.ActorSelection; import akka.actor.ActorSystem; import akka.actor.Address; -import akka.actor.Identify; import akka.actor.Props; import akka.dispatch.Futures; import akka.pattern.Patterns; @@ -64,6 +69,8 @@ import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; import java.util.concurrent.Executor; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; import java.util.function.Function; @@ -85,7 +92,7 @@ public class AkkaRpcService implements RpcService { private static final Logger LOG = LoggerFactory.getLogger(AkkaRpcService.class); - static final int VERSION = 1; + static final int VERSION = 2; private final Object lock = new Object(); @@ -104,8 +111,11 @@ public class AkkaRpcService implements RpcService { private final CompletableFuture terminationFuture; + private final Supervisor supervisor; + private volatile boolean stopped; + @VisibleForTesting public AkkaRpcService(final ActorSystem actorSystem, final AkkaRpcServiceConfiguration configuration) { this.actorSystem = checkNotNull(actorSystem, "actor system"); this.configuration = checkNotNull(configuration, "akka rpc service configuration"); @@ -131,6 +141,15 @@ public class AkkaRpcService implements RpcService { terminationFuture = new CompletableFuture<>(); stopped = false; + + supervisor = startSupervisorActor(); + } + + private Supervisor startSupervisorActor() { + final ExecutorService terminationFutureExecutor = Executors.newSingleThreadExecutor(new ExecutorThreadFactory("AkkaRpcService-Supervisor-Termination-Future-Executor")); + final ActorRef actorRef = SupervisorActor.startSupervisorActor(actorSystem, terminationFutureExecutor); + + return Supervisor.create(actorRef, terminationFutureExecutor); } public ActorSystem getActorSystem() { @@ -199,32 +218,9 @@ public class AkkaRpcService implements RpcService { public RpcServer startServer(C rpcEndpoint) { checkNotNull(rpcEndpoint, "rpc endpoint"); - CompletableFuture terminationFuture = new CompletableFuture<>(); - final Props akkaRpcActorProps; - - if (rpcEndpoint instanceof FencedRpcEndpoint) { - akkaRpcActorProps = Props.create( - FencedAkkaRpcActor.class, - rpcEndpoint, - terminationFuture, - getVersion(), - configuration.getMaximumFramesize()); - } else { - akkaRpcActorProps = Props.create( - AkkaRpcActor.class, - rpcEndpoint, - terminationFuture, - getVersion(), - configuration.getMaximumFramesize()); - } - - ActorRef actorRef; - - synchronized (lock) { - checkState(!stopped, "RpcService is stopped"); - actorRef = actorSystem.actorOf(akkaRpcActorProps, rpcEndpoint.getEndpointId()); - actors.put(actorRef, rpcEndpoint); - } + final SupervisorActor.ActorRegistration actorRegistration = registerAkkaRpcActor(rpcEndpoint); + final ActorRef actorRef = actorRegistration.getActorRef(); + final CompletableFuture actorTerminationFuture = actorRegistration.getTerminationFuture(); LOG.info("Starting RPC endpoint for {} at {} .", rpcEndpoint.getClass().getName(), actorRef.path()); @@ -252,7 +248,7 @@ public class AkkaRpcService implements RpcService { actorRef, configuration.getTimeout(), configuration.getMaximumFramesize(), - terminationFuture, + actorTerminationFuture, ((FencedRpcEndpoint) rpcEndpoint)::getFencingToken, captureAskCallstacks); @@ -264,7 +260,7 @@ public class AkkaRpcService implements RpcService { actorRef, configuration.getTimeout(), configuration.getMaximumFramesize(), - terminationFuture, + actorTerminationFuture, captureAskCallstacks); } @@ -282,6 +278,40 @@ public class AkkaRpcService implements RpcService { return server; } + private SupervisorActor.ActorRegistration registerAkkaRpcActor(C rpcEndpoint) { + final Class akkaRpcActorType; + + if (rpcEndpoint instanceof FencedRpcEndpoint) { + akkaRpcActorType = FencedAkkaRpcActor.class; + } else { + akkaRpcActorType = AkkaRpcActor.class; + } + + synchronized (lock) { + checkState(!stopped, "RpcService is stopped"); + + final SupervisorActor.StartAkkaRpcActorResponse startAkkaRpcActorResponse = SupervisorActor.startAkkaRpcActor( + supervisor.getActor(), + actorTerminationFuture -> Props.create( + akkaRpcActorType, + rpcEndpoint, + actorTerminationFuture, + getVersion(), + configuration.getMaximumFramesize()), + rpcEndpoint.getEndpointId()); + + final SupervisorActor.ActorRegistration actorRegistration = startAkkaRpcActorResponse.orElseThrow(cause -> new AkkaRpcRuntimeException( + String.format("Could not create the %s for %s.", + AkkaRpcActor.class.getSimpleName(), + rpcEndpoint.getEndpointId()), + cause)); + + actors.put(actorRegistration.getActorRef(), rpcEndpoint); + + return actorRegistration; + } + } + @Override public RpcServer fenceRpcServer(RpcServer rpcServer, F fencingToken) { if (rpcServer instanceof AkkaBasedEndpoint) { @@ -348,8 +378,12 @@ public class AkkaRpcService implements RpcService { akkaRpcActorsTerminationFuture = terminateAkkaRpcActors(); } - final CompletableFuture actorSystemTerminationFuture = FutureUtils.composeAfterwards( + final CompletableFuture supervisorTerminationFuture = FutureUtils.composeAfterwards( akkaRpcActorsTerminationFuture, + supervisor::closeAsync); + + final CompletableFuture actorSystemTerminationFuture = FutureUtils.composeAfterwards( + supervisorTerminationFuture, () -> FutureUtils.toJava(actorSystem.terminate())); actorSystemTerminationFuture.whenComplete( @@ -447,22 +481,7 @@ public class AkkaRpcService implements RpcService { LOG.debug("Try to connect to remote RPC endpoint with address {}. Returning a {} gateway.", address, clazz.getName()); - final ActorSelection actorSel = actorSystem.actorSelection(address); - - final Future identify = Patterns - .ask(actorSel, new Identify(42), configuration.getTimeout().toMilliseconds()) - .mapTo(ClassTag$.MODULE$.apply(ActorIdentity.class)); - - final CompletableFuture identifyFuture = FutureUtils.toJava(identify); - - final CompletableFuture actorRefFuture = identifyFuture.thenApply( - (ActorIdentity actorIdentity) -> { - if (actorIdentity.getRef() == null) { - throw new CompletionException(new RpcConnectionException("Could not connect to rpc endpoint under address " + address + '.')); - } else { - return actorIdentity.getRef(); - } - }); + final CompletableFuture actorRefFuture = resolveActorAddress(address); final CompletableFuture handshakeFuture = actorRefFuture.thenCompose( (ActorRef actorRef) -> FutureUtils.toJava( @@ -490,4 +509,44 @@ public class AkkaRpcService implements RpcService { }, actorSystem.dispatcher()); } + + private CompletableFuture resolveActorAddress(String address) { + final ActorSelection actorSel = actorSystem.actorSelection(address); + + return actorSel.resolveOne(TimeUtils.toDuration(configuration.getTimeout())) + .toCompletableFuture() + .exceptionally(error -> { + throw new CompletionException( + new RpcConnectionException(String.format("Could not connect to rpc endpoint under address %s.", address), error)); + }); + } + + // --------------------------------------------------------------------------------------- + // Private inner classes + // --------------------------------------------------------------------------------------- + + private static final class Supervisor implements AutoCloseableAsync { + + private final ActorRef actor; + + private final ExecutorService terminationFutureExecutor; + + private Supervisor(ActorRef actor, ExecutorService terminationFutureExecutor) { + this.actor = actor; + this.terminationFutureExecutor = terminationFutureExecutor; + } + + private static Supervisor create(ActorRef actorRef, ExecutorService terminationFutureExecutor) { + return new Supervisor(actorRef, terminationFutureExecutor); + } + + public ActorRef getActor() { + return actor; + } + + @Override + public CompletableFuture closeAsync() { + return ExecutorUtils.nonBlockingShutdown(30L, TimeUnit.SECONDS, terminationFutureExecutor); + } + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceUtils.java index 0c4b877dd9367dc1618c8e1485a1ef29eb8bdebc..bf003e66a7edeb8b4d019acc593614dfb6a9bdae 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceUtils.java @@ -18,27 +18,28 @@ package org.apache.flink.runtime.rpc.akka; -import com.typesafe.config.Config; -import com.typesafe.config.ConfigFactory; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.configuration.AkkaOptions; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.clusterframework.BootstrapTools; import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils; import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils.AddressResolution; import org.apache.flink.runtime.net.SSLUtils; -import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.util.NetUtils; import org.apache.flink.util.Preconditions; import akka.actor.ActorSystem; +import com.typesafe.config.Config; +import com.typesafe.config.ConfigFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import javax.annotation.Nonnull; +import javax.annotation.Nullable; -import java.io.IOException; import java.net.InetAddress; import java.net.UnknownHostException; +import java.util.Optional; import java.util.concurrent.atomic.AtomicLong; import static org.apache.flink.util.NetUtils.isValidClientPort; @@ -56,6 +57,8 @@ public class AkkaRpcServiceUtils { private static final String AKKA_TCP = "akka.tcp"; private static final String AKKA_SSL_TCP = "akka.ssl.tcp"; + static final String SUPERVISOR_NAME = "rpc"; + private static final String SIMPLE_AKKA_CONFIG_TEMPLATE = "akka {remote {netty.tcp {maximum-frame-size = %s}}}"; @@ -68,75 +71,35 @@ public class AkkaRpcServiceUtils { // RPC instantiation // ------------------------------------------------------------------------ - /** - * Utility method to create RPC service from configuration and hostname, port. - * - * @param hostname The hostname/address that describes the TaskManager's data location. - * @param portRangeDefinition The port range to start TaskManager on. - * @param configuration The configuration for the TaskManager. - * @return The rpc service which is used to start and connect to the TaskManager RpcEndpoint . - * @throws IOException Thrown, if the actor system can not bind to the address - * @throws Exception Thrown is some other error occurs while creating akka actor system - */ - public static RpcService createRpcService( - String hostname, - String portRangeDefinition, - Configuration configuration) throws Exception { - final ActorSystem actorSystem = BootstrapTools.startActorSystem(configuration, hostname, portRangeDefinition, LOG); - return instantiateAkkaRpcService(configuration, actorSystem); - } + public static AkkaRpcService createRemoteRpcService( + Configuration configuration, + @Nullable String externalAddress, + String externalPortRange, + @Nullable String bindAddress, + @SuppressWarnings("OptionalUsedAsFieldOrParameterType") Optional bindPort) throws Exception { + final AkkaRpcServiceBuilder akkaRpcServiceBuilder = + AkkaRpcServiceUtils.remoteServiceBuilder(configuration, externalAddress, externalPortRange); + + if (bindAddress != null) { + akkaRpcServiceBuilder.withBindAddress(bindAddress); + } - /** - * Utility method to create RPC service from configuration and hostname, port. - * - * @param hostname The hostname/address that describes the TaskManager's data location. - * @param port If true, the TaskManager will not initiate the TCP network stack. - * @param configuration The configuration for the TaskManager. - * @return The rpc service which is used to start and connect to the TaskManager RpcEndpoint . - * @throws IOException Thrown, if the actor system can not bind to the address - * @throws Exception Thrown is some other error occurs while creating akka actor system - */ - public static RpcService createRpcService( - String hostname, - int port, - Configuration configuration) throws Exception { - final ActorSystem actorSystem = BootstrapTools.startActorSystem(configuration, hostname, port, LOG); - return instantiateAkkaRpcService(configuration, actorSystem); - } + bindPort.ifPresent(akkaRpcServiceBuilder::withBindPort); - /** - * Utility method to create RPC service from configuration and hostname, port. - * - * @param hostname The hostname/address that describes the TaskManager's data location. - * @param portRangeDefinition The port range to start TaskManager on. - * @param configuration The configuration for the TaskManager. - * @param actorSystemName The actor system name of the RpcService. - * @param actorSystemExecutorConfiguration The configuration of the executor of the actor system. - * @return The rpc service which is used to start and connect to the TaskManager RpcEndpoint . - * @throws IOException Thrown, if the actor system can not bind to the address - * @throws Exception Thrown is some other error occurs while creating akka actor system - */ - public static RpcService createRpcService( - String hostname, - String portRangeDefinition, - Configuration configuration, - String actorSystemName, - @Nonnull BootstrapTools.ActorSystemExecutorConfiguration actorSystemExecutorConfiguration) throws Exception { + return akkaRpcServiceBuilder.createAndStart(); + } - final ActorSystem actorSystem = BootstrapTools.startActorSystem( - configuration, - actorSystemName, - hostname, - portRangeDefinition, - LOG, - actorSystemExecutorConfiguration); + public static AkkaRpcServiceBuilder remoteServiceBuilder(Configuration configuration, @Nullable String externalAddress, String externalPortRange) { + return new AkkaRpcServiceBuilder(configuration, LOG, externalAddress, externalPortRange); + } - return instantiateAkkaRpcService(configuration, actorSystem); + @VisibleForTesting + public static AkkaRpcServiceBuilder remoteServiceBuilder(Configuration configuration, @Nullable String externalAddress, int externalPort) { + return remoteServiceBuilder(configuration, externalAddress, String.valueOf(externalPort)); } - @Nonnull - private static RpcService instantiateAkkaRpcService(Configuration configuration, ActorSystem actorSystem) { - return new AkkaRpcService(actorSystem, AkkaRpcServiceConfiguration.fromConfiguration(configuration)); + public static AkkaRpcServiceBuilder localServiceBuilder(Configuration configuration) { + return new AkkaRpcServiceBuilder(configuration, LOG); } // ------------------------------------------------------------------------ @@ -195,8 +158,6 @@ public class AkkaRpcServiceUtils { checkNotNull(endpointName, "endpointName is null"); checkArgument(isValidClientPort(port), "port must be in [1, 65535]"); - final String protocolPrefix = akkaProtocol == AkkaProtocol.SSL_TCP ? AKKA_SSL_TCP : AKKA_TCP; - if (addressResolution == AddressResolution.TRY_ADDRESS_RESOLUTION) { // Fail fast if the hostname cannot be resolved //noinspection ResultOfMethodCallIgnored @@ -205,7 +166,46 @@ public class AkkaRpcServiceUtils { final String hostPort = NetUtils.unresolvedHostAndPortToNormalizedString(hostname, port); - return String.format("%s://flink@%s/user/%s", protocolPrefix, hostPort, endpointName); + return internalRpcUrl(endpointName, Optional.of(new RemoteAddressInformation(hostPort, akkaProtocol))); + } + + public static String getLocalRpcUrl(String endpointName) { + return internalRpcUrl(endpointName, Optional.empty()); + } + + private static final class RemoteAddressInformation { + private final String hostnameAndPort; + private final AkkaProtocol akkaProtocol; + + private RemoteAddressInformation(String hostnameAndPort, AkkaProtocol akkaProtocol) { + this.hostnameAndPort = hostnameAndPort; + this.akkaProtocol = akkaProtocol; + } + + private String getHostnameAndPort() { + return hostnameAndPort; + } + + private AkkaProtocol getAkkaProtocol() { + return akkaProtocol; + } + } + + private static String internalRpcUrl(String endpointName, Optional remoteAddressInformation) { + final String protocolPrefix = remoteAddressInformation.map(rai -> akkaProtocolToString(rai.getAkkaProtocol())).orElse("akka"); + final Optional optionalHostnameAndPort = remoteAddressInformation.map(RemoteAddressInformation::getHostnameAndPort); + + final StringBuilder url = new StringBuilder(String.format("%s://flink", protocolPrefix)); + optionalHostnameAndPort.ifPresent(hostPort -> url.append("@").append(hostPort)); + + url.append("/user/").append(SUPERVISOR_NAME).append("/").append(endpointName); + + // protocolPrefix://flink[@hostname:port]/user/rpc/endpointName + return url.toString(); + } + + private static String akkaProtocolToString(AkkaProtocol akkaProtocol) { + return akkaProtocol == AkkaProtocol.SSL_TCP ? AKKA_SSL_TCP : AKKA_TCP; } /** @@ -235,6 +235,16 @@ public class AkkaRpcServiceUtils { return prefix + '_' + nameOffset; } + /** + * Creates a wildcard name symmetric to {@link #createRandomName(String)}. + * + * @param prefix prefix of the wildcard name + * @return wildcard name starting with the prefix + */ + public static String createWildcardName(String prefix) { + return prefix + "_*"; + } + // ------------------------------------------------------------------------ // RPC service configuration // ------------------------------------------------------------------------ @@ -246,6 +256,113 @@ public class AkkaRpcServiceUtils { return akkaConfig.getBytes(MAXIMUM_FRAME_SIZE_PATH); } + // ------------------------------------------------------------------------ + // RPC service builder + // ------------------------------------------------------------------------ + + /** + * Builder for {@link AkkaRpcService}. + */ + public static class AkkaRpcServiceBuilder { + + private final Configuration configuration; + private final Logger logger; + @Nullable private final String externalAddress; + @Nullable private final String externalPortRange; + + private String actorSystemName = AkkaUtils.getFlinkActorSystemName(); + @Nullable private BootstrapTools.ActorSystemExecutorConfiguration actorSystemExecutorConfiguration = null; + @Nullable private Config customConfig = null; + private String bindAddress = NetUtils.getWildcardIPAddress(); + @Nullable private Integer bindPort = null; + + /** + * Builder for creating a remote RPC service. + */ + private AkkaRpcServiceBuilder( + final Configuration configuration, + final Logger logger, + @Nullable final String externalAddress, + final String externalPortRange) { + this.configuration = Preconditions.checkNotNull(configuration); + this.logger = Preconditions.checkNotNull(logger); + this.externalAddress = externalAddress == null ? InetAddress.getLoopbackAddress().getHostAddress() : externalAddress; + this.externalPortRange = Preconditions.checkNotNull(externalPortRange); + } + + /** + * Builder for creating a local RPC service. + */ + private AkkaRpcServiceBuilder( + final Configuration configuration, + final Logger logger) { + this.configuration = Preconditions.checkNotNull(configuration); + this.logger = Preconditions.checkNotNull(logger); + this.externalAddress = null; + this.externalPortRange = null; + } + + public AkkaRpcServiceBuilder withActorSystemName(final String actorSystemName) { + this.actorSystemName = Preconditions.checkNotNull(actorSystemName); + return this; + } + + public AkkaRpcServiceBuilder withActorSystemExecutorConfiguration( + final BootstrapTools.ActorSystemExecutorConfiguration actorSystemExecutorConfiguration) { + this.actorSystemExecutorConfiguration = actorSystemExecutorConfiguration; + return this; + } + + public AkkaRpcServiceBuilder withCustomConfig(final Config customConfig) { + this.customConfig = customConfig; + return this; + } + + public AkkaRpcServiceBuilder withBindAddress(final String bindAddress) { + this.bindAddress = Preconditions.checkNotNull(bindAddress); + return this; + } + + public AkkaRpcServiceBuilder withBindPort(int bindPort) { + Preconditions.checkArgument(NetUtils.isValidHostPort(bindPort), "Invalid port number: " + bindPort); + this.bindPort = bindPort; + return this; + } + + public AkkaRpcService createAndStart() throws Exception { + if (actorSystemExecutorConfiguration == null) { + actorSystemExecutorConfiguration = BootstrapTools.ForkJoinExecutorConfiguration.fromConfiguration(configuration); + } + + final ActorSystem actorSystem; + + if (externalAddress == null) { + // create local actor system + actorSystem = BootstrapTools.startLocalActorSystem( + configuration, + actorSystemName, + logger, + actorSystemExecutorConfiguration, + customConfig); + } else { + // create remote actor system + actorSystem = BootstrapTools.startRemoteActorSystem( + configuration, + actorSystemName, + externalAddress, + externalPortRange, + bindAddress, + Optional.ofNullable(bindPort), + logger, + actorSystemExecutorConfiguration, + customConfig); + } + + return new AkkaRpcService(actorSystem, AkkaRpcServiceConfiguration.fromConfiguration(configuration)); + } + + } + // ------------------------------------------------------------------------ /** This class is not meant to be instantiated. */ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/SupervisorActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/SupervisorActor.java new file mode 100644 index 0000000000000000000000000000000000000000..1a2be626bf486eee404fcda7d86367042480b69e --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/SupervisorActor.java @@ -0,0 +1,366 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.rpc.akka; + +import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.rpc.RpcUtils; +import org.apache.flink.runtime.rpc.akka.exceptions.AkkaRpcException; +import org.apache.flink.runtime.rpc.akka.exceptions.AkkaUnknownMessageException; +import org.apache.flink.util.Preconditions; + +import akka.AkkaException; +import akka.actor.AbstractActor; +import akka.actor.ActorRef; +import akka.actor.ActorSystem; +import akka.actor.ChildRestartStats; +import akka.actor.Props; +import akka.actor.SupervisorStrategy; +import akka.japi.pf.DeciderBuilder; +import akka.pattern.Patterns; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.Executor; +import java.util.function.Function; + +import scala.PartialFunction; +import scala.collection.Iterable; + +/** + * Supervisor actor which is responsible for starting {@link AkkaRpcActor} instances and monitoring + * when the actors have terminated. + */ +class SupervisorActor extends AbstractActor { + + private static final Logger LOG = LoggerFactory.getLogger(SupervisorActor.class); + + private final Executor terminationFutureExecutor; + + private final Map registeredAkkaRpcActors; + + SupervisorActor(Executor terminationFutureExecutor) { + this.terminationFutureExecutor = terminationFutureExecutor; + this.registeredAkkaRpcActors = new HashMap<>(); + } + + @Override + public Receive createReceive() { + return receiveBuilder() + .match(StartAkkaRpcActor.class, this::createStartAkkaRpcActorMessage) + .matchAny(this::handleUnknownMessage) + .build(); + } + + @Override + public void postStop() throws Exception { + LOG.debug("Stopping supervisor actor."); + + super.postStop(); + + for (AkkaRpcActorRegistration actorRegistration : registeredAkkaRpcActors.values()) { + terminateAkkaRpcActorOnStop(actorRegistration); + } + + registeredAkkaRpcActors.clear(); + } + + @Override + public SupervisorActorSupervisorStrategy supervisorStrategy() { + return new SupervisorActorSupervisorStrategy(); + } + + private void terminateAkkaRpcActorOnStop(AkkaRpcActorRegistration akkaRpcActorRegistration) { + akkaRpcActorRegistration.terminateExceptionally( + new AkkaRpcException( + String.format("Unexpected closing of %s with name %s.", getClass().getSimpleName(), akkaRpcActorRegistration.getEndpointId())), + terminationFutureExecutor); + } + + private void createStartAkkaRpcActorMessage(StartAkkaRpcActor startAkkaRpcActor) { + final String endpointId = startAkkaRpcActor.getEndpointId(); + final AkkaRpcActorRegistration akkaRpcActorRegistration = new AkkaRpcActorRegistration(endpointId); + + final Props akkaRpcActorProps = startAkkaRpcActor.getPropsFactory().create(akkaRpcActorRegistration.getInternalTerminationFuture()); + + LOG.debug("Starting {} with name {}.", akkaRpcActorProps.actorClass().getSimpleName(), endpointId); + + try { + final ActorRef actorRef = getContext().actorOf(akkaRpcActorProps, endpointId); + + registeredAkkaRpcActors.put(actorRef, akkaRpcActorRegistration); + + getSender().tell(StartAkkaRpcActorResponse.success( + ActorRegistration.create( + actorRef, + akkaRpcActorRegistration.getExternalTerminationFuture())), + getSelf()); + } catch (AkkaException akkaException) { + getSender().tell(StartAkkaRpcActorResponse.failure(akkaException), getSelf()); + } + } + + private void akkaRpcActorTerminated(ActorRef actorRef) { + final AkkaRpcActorRegistration actorRegistration = removeAkkaRpcActor(actorRef); + + LOG.debug("AkkaRpcActor {} has terminated.", actorRef.path()); + actorRegistration.terminate(terminationFutureExecutor); + } + + private void akkaRpcActorFailed(ActorRef actorRef, Throwable cause) { + LOG.warn("AkkaRpcActor {} has failed. Shutting it down now.", actorRef.path(), cause); + + for (Map.Entry registeredAkkaRpcActor : registeredAkkaRpcActors.entrySet()) { + final ActorRef otherActorRef = registeredAkkaRpcActor.getKey(); + if (otherActorRef.equals(actorRef)) { + final AkkaRpcException error = new AkkaRpcException(String.format("Stopping actor %s because it failed.", actorRef.path()), cause); + registeredAkkaRpcActor.getValue().markFailed(error); + } else { + final AkkaRpcException siblingException = new AkkaRpcException(String.format("Stopping actor %s because its sibling %s has failed.", otherActorRef.path(), actorRef.path())); + registeredAkkaRpcActor.getValue().markFailed(siblingException); + } + } + + getContext().getSystem().terminate(); + } + + private AkkaRpcActorRegistration removeAkkaRpcActor(ActorRef actorRef) { + return Optional.ofNullable(registeredAkkaRpcActors.remove(actorRef)) + .orElseThrow(() -> new IllegalStateException(String.format("Could not find actor %s.", actorRef.path()))); + } + + private void handleUnknownMessage(Object msg) { + final AkkaUnknownMessageException cause = new AkkaUnknownMessageException(String.format("Cannot handle unknown message %s.", msg)); + getSender().tell(new akka.actor.Status.Failure(cause), getSelf()); + throw cause; + } + + public static String getActorName() { + return AkkaRpcServiceUtils.SUPERVISOR_NAME; + } + + public static ActorRef startSupervisorActor(ActorSystem actorSystem, Executor terminationFutureExecutor) { + final Props supervisorProps = Props.create(SupervisorActor.class, terminationFutureExecutor).withDispatcher("akka.actor.supervisor-dispatcher"); + return actorSystem.actorOf(supervisorProps, getActorName()); + } + + public static StartAkkaRpcActorResponse startAkkaRpcActor( + ActorRef supervisor, + StartAkkaRpcActor.PropsFactory propsFactory, + String endpointId) { + return Patterns.ask( + supervisor, + createStartAkkaRpcActorMessage( + propsFactory, + endpointId), + RpcUtils.INF_DURATION).toCompletableFuture() + .thenApply(SupervisorActor.StartAkkaRpcActorResponse.class::cast) + .join(); + } + + public static StartAkkaRpcActor createStartAkkaRpcActorMessage( + StartAkkaRpcActor.PropsFactory propsFactory, + String endpointId) { + return StartAkkaRpcActor.create(propsFactory, endpointId); + } + + // ----------------------------------------------------------------------------- + // Internal classes + // ----------------------------------------------------------------------------- + + private final class SupervisorActorSupervisorStrategy extends SupervisorStrategy { + + @Override + public PartialFunction decider() { + return DeciderBuilder.match( + Exception.class, e -> SupervisorStrategy.stop() + ).build(); + } + + @Override + public boolean loggingEnabled() { + return false; + } + + @Override + public void handleChildTerminated(akka.actor.ActorContext context, ActorRef child, Iterable children) { + akkaRpcActorTerminated(child); + } + + @Override + public void processFailure(akka.actor.ActorContext context, boolean restart, ActorRef child, Throwable cause, ChildRestartStats stats, Iterable children) { + Preconditions.checkArgument(!restart, "The supervisor strategy should never restart an actor."); + + akkaRpcActorFailed(child, cause); + } + } + + private static final class AkkaRpcActorRegistration { + private final String endpointId; + + private final CompletableFuture internalTerminationFuture; + + private final CompletableFuture externalTerminationFuture; + + @Nullable + private Throwable errorCause; + + private AkkaRpcActorRegistration(String endpointId) { + this.endpointId = endpointId; + internalTerminationFuture = new CompletableFuture<>(); + externalTerminationFuture = new CompletableFuture<>(); + errorCause = null; + } + + private CompletableFuture getInternalTerminationFuture() { + return internalTerminationFuture; + } + + private CompletableFuture getExternalTerminationFuture() { + return externalTerminationFuture; + } + + private String getEndpointId() { + return endpointId; + } + + private void terminate(Executor terminationFutureExecutor) { + CompletableFuture terminationFuture = internalTerminationFuture; + + if (errorCause != null) { + if (!internalTerminationFuture.completeExceptionally(errorCause)) { + // we have another failure reason -> let's add it + terminationFuture = internalTerminationFuture.handle( + (ignored, throwable) -> { + if (throwable != null) { + errorCause.addSuppressed(throwable); + } + + throw new CompletionException(errorCause); + }); + } + } else { + internalTerminationFuture.completeExceptionally( + new AkkaRpcException( + String.format("RpcEndpoint %s did not complete the internal termination future.", endpointId))); + } + + FutureUtils.forwardAsync(terminationFuture, externalTerminationFuture, terminationFutureExecutor); + } + + private void terminateExceptionally(Throwable cause, Executor terminationFutureExecutor) { + terminationFutureExecutor.execute(() -> externalTerminationFuture.completeExceptionally(cause)); + } + + public void markFailed(Throwable cause) { + if (errorCause == null) { + errorCause = cause; + } else { + errorCause.addSuppressed(cause); + } + } + } + + // ----------------------------------------------------------------------------- + // Messages + // ----------------------------------------------------------------------------- + + static final class StartAkkaRpcActor { + private final PropsFactory propsFactory; + private final String endpointId; + + private StartAkkaRpcActor(PropsFactory propsFactory, String endpointId) { + this.propsFactory = propsFactory; + this.endpointId = endpointId; + } + + public String getEndpointId() { + return endpointId; + } + + public PropsFactory getPropsFactory() { + return propsFactory; + } + + private static StartAkkaRpcActor create(PropsFactory propsFactory, String endpointId) { + return new StartAkkaRpcActor(propsFactory, endpointId); + } + + interface PropsFactory { + Props create(CompletableFuture terminationFuture); + } + } + + static final class ActorRegistration { + private final ActorRef actorRef; + private final CompletableFuture terminationFuture; + + private ActorRegistration(ActorRef actorRef, CompletableFuture terminationFuture) { + this.actorRef = actorRef; + this.terminationFuture = terminationFuture; + } + + public ActorRef getActorRef() { + return actorRef; + } + + public CompletableFuture getTerminationFuture() { + return terminationFuture; + } + + public static ActorRegistration create(ActorRef actorRef, CompletableFuture terminationFuture) { + return new ActorRegistration(actorRef, terminationFuture); + } + } + + static final class StartAkkaRpcActorResponse { + @Nullable + private final ActorRegistration actorRegistration; + + @Nullable + private final Throwable error; + + private StartAkkaRpcActorResponse(@Nullable ActorRegistration actorRegistration, @Nullable Throwable error) { + this.actorRegistration = actorRegistration; + this.error = error; + } + + public ActorRegistration orElseThrow(Function throwableFunction) throws X { + if (actorRegistration != null) { + return actorRegistration; + } else { + throw throwableFunction.apply(error); + } + } + + public static StartAkkaRpcActorResponse success(ActorRegistration actorRegistration) { + return new StartAkkaRpcActorResponse(actorRegistration, null); + } + + public static StartAkkaRpcActorResponse failure(Throwable error) { + return new StartAkkaRpcActorResponse(null, error); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/exceptions/AkkaRpcRuntimeException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/exceptions/AkkaRpcRuntimeException.java new file mode 100644 index 0000000000000000000000000000000000000000..2a36c41a273944ae8bc00b890e3cb4dbbd2959cb --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/exceptions/AkkaRpcRuntimeException.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.rpc.akka.exceptions; + +import org.apache.flink.runtime.rpc.exceptions.RpcRuntimeException; + +/** + * Base class for Akka RPC related runtime exceptions. + */ +public class AkkaRpcRuntimeException extends RpcRuntimeException { + public AkkaRpcRuntimeException(String message) { + super(message); + } + + public AkkaRpcRuntimeException(Throwable cause) { + super(cause); + } + + public AkkaRpcRuntimeException(String message, Throwable cause) { + super(message, cause); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/exceptions/AkkaUnknownMessageException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/exceptions/AkkaUnknownMessageException.java index 7504761591c14bed169358346be14c71796095bb..1251b6d309157cf408ecd0fc1e7bb1a0332996fe 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/exceptions/AkkaUnknownMessageException.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/exceptions/AkkaUnknownMessageException.java @@ -22,7 +22,7 @@ package org.apache.flink.runtime.rpc.akka.exceptions; * Exception which indicates that the AkkaRpcActor has received an * unknown message type. */ -public class AkkaUnknownMessageException extends AkkaRpcException { +public class AkkaUnknownMessageException extends AkkaRpcRuntimeException { private static final long serialVersionUID = 1691338049911020814L; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/RpcRuntimeException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/RpcRuntimeException.java new file mode 100644 index 0000000000000000000000000000000000000000..666e83f7043d836e327c37f40e7b783751b62f33 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/RpcRuntimeException.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.rpc.exceptions; + +/** + * Base class for RPC related runtime exceptions. + */ +public class RpcRuntimeException extends RuntimeException { + + public RpcRuntimeException(String message) { + super(message); + } + + public RpcRuntimeException(Throwable cause) { + super(cause); + } + + public RpcRuntimeException(String message, Throwable cause) { + super(message, cause); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java index 2049ec8c31685dc0cc1202df5c11c63ec3ad58f8..f31f84b87ff9d0751e36d8adfd4090ffb617b3da 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java @@ -34,7 +34,7 @@ import org.apache.flink.runtime.executiongraph.failover.flip1.FailureHandlingRes import org.apache.flink.runtime.executiongraph.failover.flip1.RestartBackoffTimeStrategy; import org.apache.flink.runtime.executiongraph.restart.ThrowingRestartStrategy; import org.apache.flink.runtime.io.network.partition.JobMasterPartitionTracker; -import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException; import org.apache.flink.runtime.jobmaster.LogicalSlot; @@ -42,7 +42,6 @@ import org.apache.flink.runtime.jobmaster.slotpool.ThrowingSlotProvider; import org.apache.flink.runtime.metrics.groups.JobManagerJobMetricGroup; import org.apache.flink.runtime.rest.handler.legacy.backpressure.BackPressureStatsTracker; import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; -import org.apache.flink.runtime.scheduler.strategy.LazyFromSourcesSchedulingStrategy; import org.apache.flink.runtime.scheduler.strategy.SchedulingStrategy; import org.apache.flink.runtime.scheduler.strategy.SchedulingStrategyFactory; import org.apache.flink.runtime.shuffle.ShuffleMaster; @@ -142,12 +141,12 @@ public class DefaultScheduler extends SchedulerBase implements SchedulerOperatio this.executionVertexOperations = checkNotNull(executionVertexOperations); final FailoverStrategy failoverStrategy = failoverStrategyFactory.create( - getFailoverTopology(), + getSchedulingTopology(), getResultPartitionAvailabilityChecker()); log.info("Using failover strategy {} for {} ({}).", failoverStrategy, jobGraph.getName(), jobGraph.getJobID()); this.executionFailureHandler = new ExecutionFailureHandler( - getFailoverTopology(), + getSchedulingTopology(), failoverStrategy, restartBackoffTimeStrategy); this.schedulingStrategy = schedulingStrategyFactory.createInstance(this, getSchedulingTopology()); @@ -270,8 +269,8 @@ public class DefaultScheduler extends SchedulerBase implements SchedulerOperatio } @Override - protected void scheduleOrUpdateConsumersInternal(final ExecutionVertexID producerVertexId, final ResultPartitionID partitionId) { - schedulingStrategy.onPartitionConsumable(producerVertexId, partitionId); + protected void scheduleOrUpdateConsumersInternal(final IntermediateResultPartitionID partitionId) { + schedulingStrategy.onPartitionConsumable(partitionId); } // ------------------------------------------------------------------------ @@ -302,11 +301,7 @@ public class DefaultScheduler extends SchedulerBase implements SchedulerOperatio deploymentOptionsByVertex, slotExecutionVertexAssignments); - if (isDeployIndividually()) { - deployIndividually(deploymentHandles); - } else { - waitForAllSlotsAndDeploy(deploymentHandles); - } + waitForAllSlotsAndDeploy(deploymentHandles); } private void validateDeploymentOptions(final Collection deploymentOptions) { @@ -351,25 +346,6 @@ public class DefaultScheduler extends SchedulerBase implements SchedulerOperatio .collect(Collectors.toList()); } - /** - * HACK: See FLINK-14162 - * for details. - */ - private boolean isDeployIndividually() { - return schedulingStrategy instanceof LazyFromSourcesSchedulingStrategy; - } - - private void deployIndividually(final List deploymentHandles) { - for (final DeploymentHandle deploymentHandle : deploymentHandles) { - FutureUtils.assertNoException( - deploymentHandle - .getSlotExecutionVertexAssignment() - .getLogicalSlotFuture() - .handle(assignResourceOrHandleError(deploymentHandle)) - .handle(deployOrHandleError(deploymentHandle))); - } - } - private void waitForAllSlotsAndDeploy(final List deploymentHandles) { FutureUtils.assertNoException( assignAllResources(deploymentHandles).handle(deployAll(deploymentHandles))); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java index 4664f6fea92b0716f82320c588214b07800a85d8..aa38b951d319ebf20a93ad8ace52c4fc81089ecf 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java @@ -54,7 +54,6 @@ import org.apache.flink.runtime.executiongraph.JobStatusListener; import org.apache.flink.runtime.executiongraph.failover.FailoverStrategy; import org.apache.flink.runtime.executiongraph.failover.FailoverStrategyLoader; import org.apache.flink.runtime.executiongraph.failover.NoOpFailoverStrategy; -import org.apache.flink.runtime.executiongraph.failover.flip1.FailoverTopology; import org.apache.flink.runtime.executiongraph.failover.flip1.ResultPartitionAvailabilityChecker; import org.apache.flink.runtime.executiongraph.restart.RestartStrategy; import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory; @@ -62,6 +61,7 @@ import org.apache.flink.runtime.executiongraph.restart.RestartStrategyResolving; import org.apache.flink.runtime.io.network.partition.JobMasterPartitionTracker; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; +import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.OperatorID; @@ -132,9 +132,7 @@ public abstract class SchedulerBase implements SchedulerNG { private final ExecutionGraph executionGraph; - private final SchedulingTopology schedulingTopology; - - private final FailoverTopology failoverTopology; + private final SchedulingTopology schedulingTopology; private final InputsLocationsRetriever inputsLocationsRetriever; @@ -222,7 +220,6 @@ public abstract class SchedulerBase implements SchedulerNG { this.executionGraph = createAndRestoreExecutionGraph(jobManagerJobMetricGroup, checkNotNull(shuffleMaster), checkNotNull(partitionTracker)); this.schedulingTopology = executionGraph.getSchedulingTopology(); - this.failoverTopology = executionGraph.getFailoverTopology(); this.inputsLocationsRetriever = new ExecutionGraphToInputsLocationsRetrieverAdapter(executionGraph); } @@ -287,7 +284,6 @@ public abstract class SchedulerBase implements SchedulerNG { * execution graph and accessors should be preferred over direct access: *

Pattern OperationDescription模式操作描述
begin(#name) -

Defines a starting pattern:

+

定一个开始模式:

{% 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`,不同跳过策略之间的不同如下: - - - + + + @@ -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
- + @@ -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
- + - + @@ -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
- + @@ -1332,19 +1291,19 @@ For example, for a given pattern `b+ c` and a data stream `b1 b2 b3 c`, the diff b1 b2 b3 c
b3 c
- +
Skip StrategyResult Description跳过策略结果 描述
NO_SKIP After found matching b1 b2 b3 c, the match process will not discard any result.找到匹配b1 b2 b3 c之后,不会丢弃任何结果。
SKIP_TO_NEXT 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] 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.找到匹配b1 b2 b3 c之后,会尝试丢弃所有在b1之前开始的部分匹配,但没有这样的匹配,所以没有任何匹配被丢弃。
SKIP_TO_LAST[b] After found matching b1 b2 b3 c, the match process will try to discard all partial matches started before b3. There is one such match b2 b3 c找到匹配b1 b2 b3 c之后,会尝试丢弃所有在b3之前开始的部分匹配,有一个这样的b2 b3 c被丢弃。
-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`,结果将会是: - - - + + + @@ -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
- + @@ -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
- +
Skip StrategyResult Description跳过策略结果 描述
NO_SKIP 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*] 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`,结果将会是: - - - + + + @@ -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
- + - +
Skip StrategyResult Description跳过策略结果 描述
NO_SKIP 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.找到匹配a b1之后,会丢弃所有以a开始的部分匹配。这意味着不会产生a b1 b2a b1 b2 b3了。
-To specify which skip strategy to use, just create an `AfterMatchSkipStrategy` by calling: +想指定要使用的跳过策略,只需要调用下面的方法创建`AfterMatchSkipStrategy`: - - + + - + - + - + - + - +
FunctionDescription方法描述
AfterMatchSkipStrategy.noSkip()Create a NO_SKIP skip strategy 创建NO_SKIP策略
AfterMatchSkipStrategy.skipToNext()Create a SKIP_TO_NEXT skip strategy 创建SKIP_TO_NEXT策略
AfterMatchSkipStrategy.skipPastLastEvent()Create a SKIP_PAST_LAST_EVENT skip strategy 创建SKIP_PAST_LAST_EVENT策略
AfterMatchSkipStrategy.skipToFirst(patternName)Create a SKIP_TO_FIRST skip strategy with the referenced pattern name patternName创建引用模式名称为patternNameSKIP_TO_FIRST策略
AfterMatchSkipStrategy.skipToLast(patternName)Create a SKIP_TO_LAST skip strategy with the referenced pattern name patternName创建引用模式名称为patternNameSKIP_TO_LAST策略
-Then apply the skip strategy to a pattern by calling: +可以通过调用下面方法将跳过策略应用到模式上:
@@ -1438,9 +1397,9 @@ Pattern.begin("patternName", skipStrategy)
-{% 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策略,另外一个选项是抛出异常。 +可以使用如下的选项:
@@ -1455,18 +1414,18 @@ AfterMatchSkipStrategy.skipToFirst(patternName).throwExceptionOnMiss()
-## 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`:
{% highlight java %} DataStream input = ... Pattern pattern = ... -EventComparator comparator = ... // optional +EventComparator comparator = ... // 可选的 PatternStream patternStream = CEP.pattern(input, pattern, comparator); {% endhighlight %} @@ -1476,27 +1435,25 @@ PatternStream patternStream = CEP.pattern(input, pattern, comparator); {% highlight scala %} val input : DataStream[Event] = ... val pattern : Pattern[Event, _] = ... -var comparator : EventComparator[Event] = ... // optional +var comparator : EventComparator[Event] = ... // 可选的 val patternStream: PatternStream[Event] = CEP.pattern(input, pattern, comparator) {% endhighlight %}
-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 sql %} +CREATE CATALOG mypg WITH( + 'type'='jdbc', + 'default-database'='...', + 'username'='...', + 'password'='...', + 'base-url'='...' +); + +USE CATALOG 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 +### 数据库操作
@@ -172,7 +263,7 @@ catalog.listDatabases("mycatalog");
-### Table operations +### 表操作
@@ -201,7 +292,7 @@ catalog.listTables("mydb");
-### View operations +### 视图操作
@@ -231,7 +322,7 @@ catalog.listViews("mydb");
-### Partition operations +### 分区操作
@@ -272,7 +363,7 @@ catalog.listPartitions(new ObjectPath("mydb", "mytable"), Arrays.asList(epr1, ..
-### Function operations +### 函数操作
@@ -299,12 +390,12 @@ catalog.listFunctions("mydb");
-## 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 会话中。
@@ -314,8 +405,8 @@ tableEnv.registerCatalog(new CustomCatalog("myCatalog"));
-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 程序的通用结构。
@@ -123,30 +123,30 @@ table_env.execute("python_job")
-**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 中(默认行为),你应该明确地设置要在当前程序中使用的计划器。
@@ -288,53 +288,36 @@ b_b_t_env = BatchTableEnvironment.create(environment_settings=b_b_settings)
-**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 中创建:
@@ -377,18 +360,15 @@ table_env.register_table("projectedTable", proj_table)
-**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 或者常规的文件系统都可以通过这种方式来声明。
@@ -431,17 +411,15 @@ tableEnvironment.sqlUpdate("CREATE [TEMPORARY] TABLE MyTable (...) WITH (...)")
-### 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 保留关键字都必须转义。
@@ -508,18 +486,18 @@ tableEnv.createTemporaryView("other_catalog.other_database.exampleView", table)
-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` 对象返回。
@@ -658,7 +636,7 @@ revenue = table_env.sql_query(
-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` 表示原子类型。
@@ -1146,11 +1125,11 @@ val table: Table = tableEnv.fromDataStream(stream, 'myLong, 'myInt)
-**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. - -
- - - - - - - - - - - - - - - - - - - - - - - - - -
APIsDescription
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.

-{% highlight python %} -table_env.add_python_file(file_path) -{% endhighlight %} -
set_python_requirements(requirements_file_path, requirements_cache_dir=None) -

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.

-{% highlight python %} -# commands executed in shell -echo numpy==1.16.5 > requirements.txt -pip download -d cached_dir -r requirements.txt --no-binary :all: - -# python code -table_env.set_python_requirements("requirements.txt", "cached_dir") -{% endhighlight %} -

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".

-{% highlight python %} -table_env.add_python_archive("py_env.zip") -table_env.get_config().set_python_executable("py_env.zip/py_env/bin/python") -{% endhighlight %} -

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. - - - - - - - - - - - - - - - - - - - - - - - - - - - -
APIsDescription
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.

-{% highlight python %} -table_env.add_python_file(file_path) +table_env.sql_query("SELECT string, bigint, hash_code(bigint) FROM MyTable") {% endhighlight %} -
set_python_requirements(requirements_file_path, requirements_cache_dir=None) -

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.

-{% highlight python %} -# commands executed in shell -echo numpy==1.16.5 > requirements.txt -pip download -d cached_dir -r requirements.txt --no-binary :all: - -# python code -table_env.set_python_requirements("requirements.txt", "cached_dir") -{% endhighlight %} -

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".

-{% highlight python %} -table_env.add_python_archive("py_env.zip") -table_env.get_config().set_python_executable("py_env.zip/py_env/bin/python") -{% endhighlight %} -

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/hive_functions.zh.md b/docs/dev/table/hive/hive_functions.zh.md index e2b7ac51b6b6ca255f4f665ec3fefe721f65fd72..537d684aa37d71b4e15eaaf0fafc9a1e22652d51 100644 --- a/docs/dev/table/hive/hive_functions.zh.md +++ b/docs/dev/table/hive/hive_functions.zh.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版本为例:
@@ -374,7 +353,6 @@ catalogs: - name: myhive type: hive hive-conf-dir: /opt/hive-conf - hive-version: 2.3.4 {% endhighlight %}
@@ -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
diff --git a/docs/dev/table/modules.zh.md b/docs/dev/table/modules.zh.md index 8367865bed9ae11f7e08ecff4844480569a493eb..9cc84c688816b6eb70c86a52adb1a06333b18ba8 100644 --- a/docs/dev/table/modules.zh.md +++ b/docs/dev/table/modules.zh.md @@ -105,12 +105,11 @@ modules: type: core - name: myhive type: hive - hive-version: 1.2.1 {% endhighlight %}
-### 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. + + + + + + + + + + + + + + + + + + + + + + + + + + + +
APIsDescription
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.

+{% highlight python %} +table_env.add_python_file(file_path) +{% endhighlight %} +
set_python_requirements(requirements_file_path, requirements_cache_dir=None) +

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.

+{% highlight python %} +# commands executed in shell +echo numpy==1.16.5 > requirements.txt +pip download -d cached_dir -r requirements.txt --no-binary :all: + +# python code +table_env.set_python_requirements("requirements.txt", "cached_dir") +{% endhighlight %} +

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".

+{% highlight python %} +table_env.add_python_archive("py_env.zip") +table_env.get_config().set_python_executable("py_env.zip/py_env/bin/python") +{% endhighlight %} +

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. + + + + + + + + + + + + + + + + + + + + + + + + + + + +
APIsDescription
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.

+{% highlight python %} +table_env.add_python_file(file_path) +{% endhighlight %} +
set_python_requirements(requirements_file_path, requirements_cache_dir=None) +

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.

+{% highlight python %} +# commands executed in shell +echo numpy==1.16.5 > requirements.txt +pip download -d cached_dir -r requirements.txt --no-binary :all: + +# python code +table_env.set_python_requirements("requirements.txt", "cached_dir") +{% endhighlight %} +

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".

+{% highlight python %} +table_env.add_python_archive("py_env.zip") +table_env.get_config().set_python_executable("py_env.zip/py_env/bin/python") +{% endhighlight %} +

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`. + +
+
+{% highlight python %} +from pyflink.table.udf import ScalarFunction + +class MyUDF(ScalarFunction): + + def __init__(self): + self.counter = None + + def open(self, function_context): + self.counter = function_context.get_metric_group().counter("my_counter") + + def eval(self, i): + self.counter.inc(i) + return i + +{% endhighlight %} +
+ +
+ +#### 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. + +
+
+{% highlight python %} +from pyflink.table.udf import ScalarFunction + +class MyUDF(ScalarFunction): + + def __init__(self): + self.length = 0 + + def open(self, function_context): + function_context.get_metric_group().gauge("my_gauge", lambda : self.length) + + def eval(self, i): + self.length = i + return i - 1 +{% endhighlight %} +
+ +
+ +#### 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. + +
+
+{% highlight python %} +from pyflink.table.udf import ScalarFunction + +class MyUDF(ScalarFunction): + + def __init__(self): + self.distribution = None + + def open(self, function_context): + self.distribution = function_context.get_metric_group().distribution("my_distribution") + + def eval(self, i): + self.distribution.update(i) + return i - 1 +{% endhighlight %} +
+ +
+ +#### 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. + +
+
+{% highlight python %} + +function_context + .get_metric_group() + .add_group("my_metrics") + .counter("my_counter") + +function_context + .get_metric_group() + .add_group("my_metrics_key", "my_metrics_value") + .counter("my_counter") + +{% endhighlight %} +
+ +
+ +### 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. + +
+
+{% highlight python %} +function_context + .get_metric_group() + .add_group("my_metrics_key", "my_metrics_value") + .counter("my_counter") +{% endhighlight %} +
+ +
+ +## 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`. + +
+
+{% highlight python %} +from pyflink.table.udf import ScalarFunction + +class MyUDF(ScalarFunction): + + def __init__(self): + self.counter = None + + def open(self, function_context): + self.counter = function_context.get_metric_group().counter("my_counter") + + def eval(self, i): + self.counter.inc(i) + return i + +{% endhighlight %} +
+ +
+ +#### 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. + +
+
+{% highlight python %} +from pyflink.table.udf import ScalarFunction + +class MyUDF(ScalarFunction): + + def __init__(self): + self.length = 0 + + def open(self, function_context): + function_context.get_metric_group().gauge("my_gauge", lambda : self.length) + + def eval(self, i): + self.length = i + return i - 1 +{% endhighlight %} +
+ +
+ +#### 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. + +
+
+{% highlight python %} +from pyflink.table.udf import ScalarFunction + +class MyUDF(ScalarFunction): + + def __init__(self): + self.distribution = None + + def open(self, function_context): + self.distribution = function_context.get_metric_group().distribution("my_distribution") + + def eval(self, i): + self.distribution.update(i) + return i - 1 +{% endhighlight %} +
+ +
+ +#### 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. + +
+
+{% highlight python %} + +function_context + .get_metric_group() + .add_group("my_metrics") + .counter("my_counter") + +function_context + .get_metric_group() + .add_group("my_metrics_key", "my_metrics_value") + .counter("my_counter") + +{% endhighlight %} +
+ +
+ +### 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. + +
+
+{% highlight python %} +function_context + .get_metric_group() + .add_group("my_metrics_key", "my_metrics_value") + .counter("my_counter") +{% endhighlight %} +
+ +
+ +## 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 %}
@@ -461,7 +461,7 @@ AppendStreamTableSink implements TableSink { {% highlight scala %} AppendStreamTableSink[T] extends TableSink[T] { - def emitDataStream(dataStream: DataStream[T]): Unit + def consumeDataStream(dataStream: DataStream[T]): DataStreamSink[_] } {% endhighlight %}
@@ -484,7 +484,7 @@ RetractStreamTableSink implements TableSink> { public TypeInformation getRecordType(); - public void emitDataStream(DataStream> dataStream); + public DataStreamSink consumeDataStream(DataStream> dataStream); } {% endhighlight %}
@@ -495,7 +495,7 @@ RetractStreamTableSink[T] extends TableSink[Tuple2[Boolean, T]] { def getRecordType: TypeInformation[T] - def emitDataStream(dataStream: DataStream[Tuple2[Boolean, T]]): Unit + def consumeDataStream(dataStream: DataStream[Tuple2[Boolean, T]]): DataStreamSink[_] } {% endhighlight %}
@@ -522,7 +522,7 @@ UpsertStreamTableSink implements TableSink> { public TypeInformation getRecordType(); - public void emitDataStream(DataStream> dataStream); + public DataStreamSink consumeDataStream(DataStream> dataStream); } {% endhighlight %}
@@ -537,7 +537,7 @@ UpsertStreamTableSink[T] extends TableSink[Tuple2[Boolean, T]] { def getRecordType: TypeInformation[T] - def emitDataStream(dataStream: DataStream[Tuple2[Boolean, T]]): Unit + def consumeDataStream(dataStream: DataStream[Tuple2[Boolean, T]]): DataStreamSink[_] } {% endhighlight %}
diff --git a/docs/dev/table/sourceSinks.zh.md b/docs/dev/table/sourceSinks.zh.md index fab6930dbee41b4bc2dbcbb98e8a0814519d5d1f..e9f2c775959871ead976d165da5e765f5f257590 100644 --- a/docs/dev/table/sourceSinks.zh.md +++ b/docs/dev/table/sourceSinks.zh.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 %} @@ -461,7 +461,7 @@ AppendStreamTableSink implements TableSink { {% highlight scala %} AppendStreamTableSink[T] extends TableSink[T] { - def emitDataStream(dataStream: DataStream[T]): Unit + def consumeDataStream(dataStream: DataStream[T]): DataStreamSink[_] } {% endhighlight %} @@ -484,7 +484,7 @@ RetractStreamTableSink implements TableSink> { public TypeInformation getRecordType(); - public void emitDataStream(DataStream> dataStream); + public DataStreamSink consumeDataStream(DataStream> dataStream); } {% endhighlight %} @@ -495,7 +495,7 @@ RetractStreamTableSink[T] extends TableSink[Tuple2[Boolean, T]] { def getRecordType: TypeInformation[T] - def emitDataStream(dataStream: DataStream[Tuple2[Boolean, T]]): Unit + def consumeDataStream(dataStream: DataStream[Tuple2[Boolean, T]]): DataStreamSink[_] } {% endhighlight %} @@ -522,7 +522,7 @@ UpsertStreamTableSink implements TableSink> { public TypeInformation getRecordType(); - public void emitDataStream(DataStream> dataStream); + public DataStreamSink consumeDataStream(DataStream> dataStream); } {% endhighlight %} @@ -537,7 +537,7 @@ UpsertStreamTableSink[T] extends TableSink[Tuple2[Boolean, T]] { def getRecordType: TypeInformation[T] - def emitDataStream(dataStream: DataStream[Tuple2[Boolean, T]]): Unit + def consumeDataStream(dataStream: DataStream[Tuple2[Boolean, T]]): DataStreamSink[_] } {% 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
Grouping sets, Rollup, Cube
- Batch + Batch Streaming + Result Updating
{% highlight sql %} @@ -479,6 +480,7 @@ SELECT SUM(amount) FROM Orders GROUP BY GROUPING SETS ((user), (product)) {% endhighlight %} +

Note: Streaming mode Grouping sets, Rollup and Cube are only supported in Blink planner.

GroupBy 聚合
批处理 流处理
- 可自动更新结果 + 结果更新

注意: GroupBy 在流处理表中会产生更新结果(updating result)。详情请阅读 动态表流概念 。 @@ -459,7 +459,7 @@ WINDOW w AS (

Distinct
批处理 流处理
- 可自动更新结果 + 结果更新
{% highlight sql %} @@ -471,7 +471,8 @@ SELECT DISTINCT users FROM Orders
Grouping sets, Rollup, Cube
- 批处理 + 批处理 流处理 + 结果更新
{% highlight sql %} @@ -479,6 +480,7 @@ SELECT SUM(amount) FROM Orders GROUP BY GROUPING SETS ((user), (product)) {% endhighlight %} +

Note: 流式 Grouping sets、Rollup 以及 Cube 只在 Blink planner 中支持。

Outer Equi-join
批处理 流处理 - 可自动更新结果 + 结果更新

目前仅支持 equi-join ,即 join 的联合条件至少拥有一个相等谓词。不支持任何 cross join 和 theta join。

@@ -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 / SQLStream 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: +下图显示了流、动态表和连续查询之间的关系:
Dynamic tables
-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. +下图显示了单击事件流(左侧)如何转换为表(右侧)。当插入更多的单击流记录时,结果表将不断增长。
Append mode
-**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` 表被附加的行更新时,查询是如何被评估的。
Continuous Non-Windowed Query
-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)的窗口,后面会讨论)。同样,该图显示了不同时间点的输入和输出,以可视化动态表的变化特性。
Continuous Group-Window Query
-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 流的过程。
Dynamic tables


-* **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 流的过程。
Dynamic tables


-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 中指定时间属性:
@@ -53,7 +53,7 @@ final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEn env.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime); // default -// alternatively: +// 或者: // env.setStreamTimeCharacteristic(TimeCharacteristic.IngestionTime); // env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); {% endhighlight %} @@ -64,7 +64,7 @@ val env = StreamExecutionEnvironment.getExecutionEnvironment env.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime) // default -// alternatively: +// 或者: // env.setStreamTimeCharacteristic(TimeCharacteristic.IngestionTime) // env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) {% endhighlight %} @@ -75,30 +75,30 @@ env = StreamExecutionEnvironment.get_execution_environment() env.set_stream_time_characteristic(TimeCharacteristic.ProcessingTime) # default -# alternatively: +# 或者: # env.set_stream_time_characteristic(TimeCharacteristic.IngestionTime) # env.set_stream_time_characteristic(TimeCharacteristic.EventTime) {% endhighlight %}
-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** +**声明**
@@ -778,34 +721,27 @@ DataTypes.TIMESTAMP_WITH_TIME_ZONE(p)
-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** +**声明**
@@ -824,37 +760,32 @@ DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(p)
-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** +**声明**
@@ -878,40 +809,37 @@ DataTypes.INTERVAL(DataTypes.MONTH())
-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`。 + +**声明**
@@ -951,30 +879,25 @@ DataTypes.INTERVAL(DataTypes.SECOND(p2))
-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** +**声明**
@@ -1117,25 +1030,23 @@ DataTypes.ROW(DataTypes.FIELD(n0, t0, d0), DataTypes.FIELD(n1, t1, d1), ...)
-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 @@ + + + + + + + + + + + + + + Produced by OmniGraffle 7.9.3 + 2018-12-07 15:56:23 +0000 + + + Canvas 1 + + + Layer 1 + + + + + + + + + data in + + + + + + + data out + + + + + + + control stream + + + + + 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 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Produced by OmniGraffle 7.10.2 + 2019-07-13 20:06:51 +0000 + + + Canvas 1 + + Layer 1 + + path3065 + + + + + + + + + text3067 + + JobManager + + + + path3005 + + + + + text3007 + + TaskManager + + + + path3009 + + + + path3011 + + + + text3013 + + Task + + + + text3015 + + Slot + + + + path3017 + + + + path3019 + + + + text3021 + + Task + + + + text3023 + + Slot + + + + path3025 + + + + path3027 + + + + text3029 + + Task + + + + path3031 + + + + path3033 + + + + text3035 + + Task + + + + text3037 + + Slot + + + + path3039 + + + + path3041 + + + + text3043 + + Task + + + + path3147 + + + + + text3149 + + TaskManager + + + + path3151 + + + + path3153 + + + + text3155 + + Task + + + + text3157 + + Slot + + + + path3167 + + + + path3169 + + + + text3171 + + Task + + + + path3173 + + + + path3175 + + + + text3177 + + Task + + + + text3179 + + Slot + + + + path3181 + + + + path3183 + + + + text3185 + + Task + + + + path3245 + + + + text3249 + + Scheduler + + + + path3251 + + + + text3255 + + Checkpoint + + + + text3257 + + Coordinator + + + + + path2991 + + + + path2993 + + + + text2995 + + Flink Client Program + + + + path2997 + + + + path2999 + + + + text3001 + + Client + + + + path3237 + + + + path3239 + + + + text3241 + + Program + + + + text3243 + + code + + + + path3259 + + + + + text3265 + + Graph Builder + + + + path3267 + + + + + text3269 + + Job graph + + + + path3271 + + + + + text3273 + + Program + + + + text3275 + + Dataflow + + + + + + + + DataStreams + + + + + + + + + + Resource + Manager + + + + + + + Dispatcher + + + + text3149 + + Flink Master + + + + + + Submit job + + + + + + + + + + path3173 + + + + path3175 + + + + text3177 + + Task + + + + text3179 + + Slot + + + + path3181 + + + + text3171 + + Task + + + + path3183 + + + + + 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 @@ + + + + + + + + + + + + + + Produced by OmniGraffle 7.9.3 + 2018-12-07 15:56:11 +0000 + + + Canvas 1 + + Layer 1 + + + + + + + + + data in + + + + + + + data out + + + + + 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 @@ + + + + + + + + + + + + + + Produced by OmniGraffle 7.15 + 2018-12-06 15:14:03 +0000 + + + Canvas 1 + + events + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + windows + + + + + + + + + Tumbling count windows + + + + + Sliding count windows + + + + + Global window + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Tumbling time windows + + + + + Sliding time windows + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Session windows + + + + + + + + + + + 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 +
+
+
+
+
+
+ Yarn +
+
+ Deploy Flink on-top Apache Hadoop's resource manager +
Learn more +
+
+
+
+
+
+
+
+ Mesos +
+
+ 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 +
+
+
+
+
+
+ Yarn +
+
+ Deploy Flink on-top Apache Hadoop's resource manager +
Learn more +
+
+
+
+
+
+
+
+ Mesos +
+
+ 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 -
-
{% highlight bash %} $ ./bin/start-cluster.sh # Start Flink {% endhighlight %} -
-
-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 -
-
{% highlight bash %} $ ./bin/start-cluster.sh # Start Flink {% endhighlight %} -
-
-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:/// {% endhighlight %} -The endpoint can either be a single file or a directory, for example: +Endpoint 可以是一个文件或目录,例如: {% highlight java %} -// Read from S3 bucket +// 读取 S3 bucket env.readTextFile("s3:///"); -// Write to S3 bucket +// 写入 S3 bucket stream.writeAsText("s3:///"); -// Use S3 as FsStatebackend +// 使用 S3 作为 FsStatebackend env.setStateBackend(new FsStateBackend("s3:///")); {% endhighlight %} -Note that these examples are *not* exhaustive and you can use S3 in other places as well, including your [high availability setup](../jobmanager_high_availability.html) or the [RocksDBStateBackend]({{ site.baseurl }}/ops/state/state_backends.html#the-rocksdbstatebackend); everywhere that Flink expects a FileSystem URI. +注意这些例子并*不详尽*,S3 同样可以用在其他场景,包括 [JobManager 高可用配置](../jobmanager_high_availability.html) 或 [RocksDBStateBackend]({{ site.baseurl }}/zh/ops/state/state_backends.html#the-rocksdbstatebackend),以及所有 Flink 需要使用文件系统 URI 的位置。 -For most use cases, you may use one of our `flink-s3-fs-hadoop` and `flink-s3-fs-presto` S3 filesystem plugins which are self-contained and easy to set up. -For some cases, however, e.g., for using S3 as YARN's resource storage dir, it may be necessary to set up a specific Hadoop S3 filesystem implementation. +在大部分使用场景下,可使用 `flink-s3-fs-hadoop` 或 `flink-s3-fs-presto` 两个独立且易于设置的 S3 文件系统插件。然而在某些情况下,例如使用 S3 作为 YARN 的资源存储目录时,可能需要配置 Hadoop S3 文件系统。 -### Hadoop/Presto S3 File Systems plugins +### Hadoop/Presto S3 文件系统插件 -{% panel **Note:** You don't have to configure this manually if you are running [Flink on EMR](https://docs.aws.amazon.com/emr/latest/ReleaseGuide/emr-flink.html). %} +{% panel **注意:** 如果您在使用 [Flink on EMR](https://docs.aws.amazon.com/emr/latest/ReleaseGuide/emr-flink.html),您无需手动对此进行配置。 %} -Flink provides two file systems to talk to Amazon S3, `flink-s3-fs-presto` and `flink-s3-fs-hadoop`. -Both implementations are self-contained with no dependency footprint, so there is no need to add Hadoop to the classpath to use them. +Flink 提供两种文件系统用来与 S3 交互:`flink-s3-fs-presto` 和 `flink-s3-fs-hadoop`。两种实现都是独立的且没有依赖项,因此使用时无需将 Hadoop 添加至 classpath。 - - `flink-s3-fs-presto`, registered under the scheme *s3://* and *s3p://*, is based on code from the [Presto project](https://prestodb.io/). - You can configure it the same way you can [configure the Presto file system](https://prestodb.io/docs/0.187/connector/hive.html#amazon-s3-configuration) by placing adding the configurations to your `flink-conf.yaml`. Presto is the recommended file system for checkpointing to S3. + - `flink-s3-fs-presto`,通过 *s3://* 和 *s3p://* 两种 scheme 使用,基于 [Presto project](https://prestodb.io/)。 + 可以使用与[配置 Presto 文件系统](https://prestodb.io/docs/0.187/connector/hive.html#amazon-s3-configuration)相同的方法进行配置,即将配置添加到 `flink-conf.yaml` 文件中。推荐使用 Presto 文件系统来在 S3 中建立 checkpoint。 - - `flink-s3-fs-hadoop`, registered under *s3://* and *s3a://*, based on code from the [Hadoop Project](https://hadoop.apache.org/). - The file system can be [configured exactly like Hadoop's s3a](https://hadoop.apache.org/docs/stable/hadoop-aws/tools/hadoop-aws/index.html#S3A) by placing adding the configurations to your `flink-conf.yaml`. It is the only S3 file system with support for the [StreamingFileSink]({{ site.baseurl}}/dev/connectors/streamfile_sink.html). + - `flink-s3-fs-hadoop`,通过 *s3://* 和 *s3a://* 两种 scheme 使用, 基于 [Hadoop Project](https://hadoop.apache.org/)。 + 文件系统可以使用与 [Hadoop S3A 完全相同的配置方法](https://hadoop.apache.org/docs/stable/hadoop-aws/tools/hadoop-aws/index.html#S3A)进行配置,即将配置添加到 `flink-conf.yaml` 文件中。它是唯一一个支持 [StreamingFileSink]({{ site.baseurl}}/zh/dev/connectors/streamfile_sink.html) 的文件系统。 -Both `flink-s3-fs-hadoop` and `flink-s3-fs-presto` register default FileSystem -wrappers for URIs with the *s3://* scheme, `flink-s3-fs-hadoop` also registers -for *s3a://* and `flink-s3-fs-presto` also registers for *s3p://*, so you can -use this to use both at the same time. -For example, the job uses the [StreamingFileSink]({{ site.baseurl}}/dev/connectors/streamfile_sink.html) which only supports Hadoop, but uses Presto for checkpointing. -In this case, it is advised to explicitly use *s3a://* as a scheme for the sink (Hadoop) and *s3p://* for checkpointing (Presto). +`flink-s3-fs-hadoop` 和 `flink-s3-fs-presto` 都为 *s3://* scheme 注册了默认的文件系统包装器,`flink-s3-fs-hadoop` 另外注册了 *s3a://*,`flink-s3-fs-presto` 注册了 *s3p://*,因此二者可以同时使用。 +例如某作业使用了 [StreamingFileSink]({{ site.baseurl}}/zh/dev/connectors/streamfile_sink.html),它仅支持 Hadoop,但建立 checkpoint 使用 Presto。在这种情况下,建议明确地使用 *s3a://* 作为 sink (Hadoop) 的 scheme,checkpoint (Presto) 使用 *s3p://*。 -To use `flink-s3-fs-hadoop` or `flink-s3-fs-presto`, copy the respective JAR file from the `opt` directory to the `plugins` directory of your Flink distribution before starting Flink, e.g. +在启动 Flink 之前,将对应的 JAR 文件从 `opt` 复制到 Flink 发行版的 `plugins` 目录下,以使用 `flink-s3-fs-hadoop` 或 `flink-s3-fs-presto`。 {% highlight bash %} mkdir ./plugins/s3-fs-presto cp ./opt/flink-s3-fs-presto-{{ site.version }}.jar ./plugins/s3-fs-presto/ {% endhighlight %} -#### Configure Access Credentials +#### 配置访问凭据 -After setting up the S3 FileSystem wrapper, you need to make sure that Flink is allowed to access your S3 buckets. +在设置好 S3 文件系统包装器后,您需要确认 Flink 具有访问 S3 Bucket 的权限。 -##### Identity and Access Management (IAM) (Recommended) +##### Identity and Access Management (IAM)(推荐使用) -The recommended way of setting up credentials on AWS is via [Identity and Access Management (IAM)](http://docs.aws.amazon.com/IAM/latest/UserGuide/introduction.html). You can use IAM features to securely give Flink instances the credentials that they need to access S3 buckets. Details about how to do this are beyond the scope of this documentation. Please refer to the AWS user guide. What you are looking for are [IAM Roles](http://docs.aws.amazon.com/AWSEC2/latest/UserGuide/iam-roles-for-amazon-ec2.html). +建议通过 [Identity and Access Management (IAM)](http://docs.aws.amazon.com/IAM/latest/UserGuide/introduction.html) 来配置 AWS 凭据。可使用 IAM 功能为 Flink 实例安全地提供访问 S3 Bucket 所需的凭据。关于配置的细节超出了本文档的范围,请参考 AWS 用户手册中的 [IAM Roles](http://docs.aws.amazon.com/AWSEC2/latest/UserGuide/iam-roles-for-amazon-ec2.html) 部分。 -If you set this up correctly, you can manage access to S3 within AWS and don't need to distribute any access keys to Flink. +如果配置正确,则可在 AWS 中管理对 S3 的访问,而无需为 Flink 分发任何访问密钥(Access Key)。 -##### Access Keys (Discouraged) +##### 访问密钥(Access Key)(不推荐) -Access to S3 can be granted via your **access and secret key pair**. Please note that this is discouraged since the [introduction of IAM roles](https://blogs.aws.amazon.com/security/post/Tx1XG3FX6VMU6O5/A-safer-way-to-distribute-AWS-credentials-to-EC2). +可以通过**访问密钥对(access and secret key)**授予 S3 访问权限。请注意,根据 [Introduction of IAM roles](https://blogs.aws.amazon.com/security/post/Tx1XG3FX6VMU6O5/A-safer-way-to-distribute-AWS-credentials-to-EC2),不推荐使用该方法。 -You need to configure both `s3.access-key` and `s3.secret-key` in Flink's `flink-conf.yaml`: + `s3.access-key` 和 `s3.secret-key` 均需要在 Flink 的 `flink-conf.yaml` 中进行配置: {% highlight yaml %} s3.access-key: your-access-key s3.secret-key: your-secret-key {% endhighlight %} -## Configure Non-S3 Endpoint +## 配置非 S3 访问点 -The S3 Filesystems also support using S3 compliant object stores such as [IBM's Cloud Object Storage](https://www.ibm.com/cloud/object-storage) and [Minio](https://min.io/). -To do so, configure your endpoint in `flink-conf.yaml`. +S3 文件系统还支持兼容 S3 的对象存储服务,如 [IBM's Cloud Object Storage](https://www.ibm.com/cloud/object-storage) 和 [Minio](https://min.io/)。可在 `flink-conf.yaml` 中配置使用的访问点: {% highlight yaml %} s3.endpoint: your-endpoint-hostname {% endhighlight %} -## 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`. +某些兼容 S3 的对象存储服务可能没有默认启用虚拟主机样式的寻址。这种情况下需要在 `flink-conf.yaml` 中添加配置以启用路径样式的访问: {% highlight yaml %} s3.path.style.access: true {% endhighlight %} -## Entropy injection for S3 file systems +## S3 文件系统的熵注入 -The bundled S3 file systems (`flink-s3-fs-presto` and `flink-s3-fs-hadoop`) support entropy injection. Entropy injection is -a technique to improve the scalability of AWS S3 buckets through adding some random characters near the beginning of the key. +内置的 S3 文件系统 (`flink-s3-fs-presto` and `flink-s3-fs-hadoop`) 支持熵注入。熵注入是通过在关键字开头附近添加随机字符,以提高 AWS S3 bucket 可扩展性的技术。 -If entropy injection is activated, a configured substring in the path is replaced with random characters. For example, path -`s3://my-bucket/checkpoints/_entropy_/dashboard-job/` would be replaced by something like `s3://my-bucket/checkpoints/gf36ikvg/dashboard-job/`. -**This only happens when the file creation passes the option to inject entropy!** -Otherwise, the file path removes the entropy key substring entirely. See [FileSystem.create(Path, WriteOption)](https://ci.apache.org/projects/flink/flink-docs-release-1.6/api/java/org/apache/flink/core/fs/FileSystem.html#create-org.apache.flink.core.fs.Path-org.apache.flink.core.fs.FileSystem.WriteOptions-) -for details. +如果熵注入被启用,路径中配置好的字串将会被随机字符所替换。例如路径 `s3://my-bucket/checkpoints/_entropy_/dashboard-job/` 将会被替换成类似于 `s3://my-bucket/checkpoints/gf36ikvg/dashboard-job/` 的路径。 +**这仅在使用熵注入选项创建文件时启用!** +否则将完全删除文件路径中的 entropy key。更多细节请参见 [FileSystem.create(Path, WriteOption)](https://ci.apache.org/projects/flink/flink-docs-release-1.6/api/java/org/apache/flink/core/fs/FileSystem.html#create-org.apache.flink.core.fs.Path-org.apache.flink.core.fs.FileSystem.WriteOptions-)。 -{% panel **Note:** The Flink runtime currently passes the option to inject entropy only to checkpoint data files. All other files, including checkpoint metadata and external URI, do not inject entropy to keep checkpoint URIs predictable. %} +{% panel **注意:** 目前 Flink 运行时仅对 checkpoint 数据文件使用熵注入选项。所有其他文件包括 chekcpoint 元数据与外部 URI 都不使用熵注入,以保证 checkpoint URI 的可预测性。 %} -To enable entropy injection, configure the *entropy key* and the *entropy length* parameters. +配置 *entropy key* 与 *entropy length* 参数以启用熵注入: ``` s3.entropy.key: _entropy_ @@ -138,8 +128,8 @@ s3.entropy.length: 4 (default) ``` -The `s3.entropy.key` defines the string in paths that is replaced by the random characters. Paths that do not contain the entropy key are left unchanged. -If a file system operation does not pass the *"inject entropy"* write option, the entropy key substring is simply removed. -The `s3.entropy.length` defines the number of random alphanumeric characters used for entropy. +`s3.entropy.key` 定义了路径中被随机字符替换掉的字符串。不包含 entropy key 路径将保持不变。 +如果文件系统操作没有经过 *"熵注入"* 写入,entropy key 字串将被直接移除。 +`s3.entropy.length` 定义了用于熵注入的随机字母/数字字符的数量。 {% top %} diff --git a/docs/ops/index.md b/docs/ops/index.md index a2e33ad142d21d27b505d9e1e887f43bca238134..a968d134ee66bcb235b7de24e9b3db5d7f11b0d7 100644 --- a/docs/ops/index.md +++ b/docs/ops/index.md @@ -3,7 +3,7 @@ title: "Deployment & Operations" nav-id: ops nav-title: ' Deployment & Operations' nav-parent_id: root -nav-pos: 6 +nav-pos: 10 --- -This section gives a detailed description of all components in Flink’s memory model of task executor. -Check [memory configuration guide](mem_setup.html) for the basic memory setup. +本文将详细介绍 Flink TaskExecutor 内存模型中的所有组成部分。 +关于基本的内存配置方法,请参考[配置指南](mem_setup.html)。 * toc {:toc} -## Overview +## 概述
Simple memory model

- -The following table lists all memory components, depicted above, and references Flink configuration options -which affect the size of the respective components: - -|   **Component**   |   **Configuration options**   |   **Description**   | -| :-------------------------------------------------------------------------------- | :------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------ | :----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | -| [Framework Heap Memory](#framework-memory) | [`taskmanager.memory.framework.heap.size`](../config.html#taskmanager-memory-framework-heap-size) | JVM heap memory dedicated to Flink framework (advanced option) | -| [Task Heap Memory](mem_setup.html#task-operator-heap-memory) | [`taskmanager.memory.task.heap.size`](../config.html#taskmanager-memory-task-heap-size) | JVM heap memory dedicated to Flink application to run operators and user code | -| [Managed memory](mem_setup.html#managed-memory) | [`taskmanager.memory.managed.size`](../config.html#taskmanager-memory-managed-size)
[`taskmanager.memory.managed.fraction`](../config.html#taskmanager-memory-managed-fraction) | Native memory managed by Flink, reserved for sorting, hash tables, caching of intermediate results and RocksDB state backend | -| [Framework Off-heap Memory](#framework-memory) | [`taskmanager.memory.framework.off-heap.size`](../config.html#taskmanager-memory-framework-off-heap-size) | [Off-heap direct (or native) memory](mem_setup.html#configure-off-heap-memory-direct-or-native) dedicated to Flink framework (advanced option) | -| [Task Off-heap Memory](mem_setup.html#configure-off-heap-memory-direct-or-native) | [`taskmanager.memory.task.off-heap.size`](../config.html#taskmanager-memory-task-off-heap-size) | [Off-heap direct (or native) memory](mem_setup.html#configure-off-heap-memory-direct-or-native) dedicated to Flink application to run operators | -| Network Memory | [`taskmanager.memory.network.min`](../config.html#taskmanager-memory-network-min)
[`taskmanager.memory.network.max`](../config.html#taskmanager-memory-network-max)
[`taskmanager.memory.network.fraction`](../config.html#taskmanager-memory-network-fraction) | Direct memory reserved for data record exchange between tasks (e.g. buffering for the transfer over the network), it is a [capped fractionated component](#capped-fractionated-components) of the [total Flink memory](mem_setup.html#configure-total-memory) | -| [JVM metaspace](#jvm-parameters) | [`taskmanager.memory.jvm-metaspace.size`](../config.html#taskmanager-memory-jvm-metaspace-size) | Metaspace size of the Flink JVM process | -| JVM Overhead | [`taskmanager.memory.jvm-overhead.min`](../config.html#taskmanager-memory-jvm-overhead-min)
[`taskmanager.memory.jvm-overhead.max`](../config.html#taskmanager-memory-jvm-overhead-max)
[`taskmanager.memory.jvm-overhead.fraction`](../config.html#taskmanager-memory-jvm-overhead-fraction) | Native memory reserved for other JVM overhead: e.g. thread stacks, code cache, garbage collection space etc, it is a [capped fractionated component](#capped-fractionated-components) of the [total process memory](mem_setup.html#configure-total-memory) | +提示 任务堆外内存也包括了用户代码使用的本地内存(非直接内存)。 + +如上图所示,下表中列出了 Flink TaskExecutor 内存模型的所有组成部分,以影响其大小的相关配置参数。 + +|   **组成部分**   |   **配置参数**   |   **描述**   | +| :----------------------------------------------------------------------------- | :--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | :----------------------------------------------------------------------------------------------------------------------------------------- | +| [框架堆内存(Framework Heap Memory)](#框架内存) | [`taskmanager.memory.framework.heap.size`](../config.html#taskmanager-memory-framework-heap-size) | 用于 Flink 框架的 JVM 堆内存(进阶配置)。 | +| [任务堆内存(Task Heap Memory)](mem_setup.html#任务算子堆内存) | [`taskmanager.memory.task.heap.size`](../config.html#taskmanager-memory-task-heap-size) | 用于 Flink 应用的算子及用户代码的 JVM 堆内存。 | +| [托管内存(Managed memory)](mem_setup.html#托管内存) | [`taskmanager.memory.managed.size`](../config.html#taskmanager-memory-managed-size)
[`taskmanager.memory.managed.fraction`](../config.html#taskmanager-memory-managed-fraction) | 由 Flink 管理的用于排序、哈希表、缓存中间结果及 RocksDB State Backend 的本地内存。 | +| [框架堆外内存(Framework Off-heap Memory)](#框架内存) | [`taskmanager.memory.framework.off-heap.size`](../config.html#taskmanager-memory-framework-off-heap-size) | 用于 Flink 框架的[堆外内存(直接内存或本地内存)](mem_setup.html#配置堆外内存直接内存或本地内存)(进阶配置)。 | +| [任务堆外内存(Task Off-heap Memory)](mem_setup.html#配置堆外内存直接内存或本地内存) | [`taskmanager.memory.task.off-heap.size`](../config.html#taskmanager-memory-task-off-heap-size) | 用于 Flink 应用的算计及用户代码的[堆外内存(直接内存或本地内存)](mem_setup.html#配置堆外内存直接内存或本地内存)。 | +| 网络内存(Network Memory) | [`taskmanager.memory.network.min`](../config.html#taskmanager-memory-network-min)
[`taskmanager.memory.network.max`](../config.html#taskmanager-memory-network-max)
[`taskmanager.memory.network.fraction`](../config.html#taskmanager-memory-network-fraction) | 用于任务之间数据传输的直接内存(例如网络传输缓冲)。该内存部分为基于 [Flink 总内存](mem_setup.html#配置总内存)的[受限的等比内存部分](#受限的等比内存部分)。 | +| [JVM Metaspace](#jvm-参数) | [`taskmanager.memory.jvm-metaspace.size`](../config.html#taskmanager-memory-jvm-metaspace-size) | Flink JVM 进程的 Metaspace。 | +| JVM 开销 | [`taskmanager.memory.jvm-overhead.min`](../config.html#taskmanager-memory-jvm-overhead-min)
[`taskmanager.memory.jvm-overhead.max`](../config.html#taskmanager-memory-jvm-overhead-max)
[`taskmanager.memory.jvm-overhead.fraction`](../config.html#taskmanager-memory-jvm-overhead-fraction) | 用于其他 JVM 开销的本地内存,例如栈空间、垃圾回收空间等。该内存部分为基于[进程总内存](mem_setup.html#配置总内存)的[受限的等比内存部分](#受限的等比内存部分)。 | {:.table-bordered}
-As you can see, the size of some memory components can be simply set by the respective option. -Other components can be tuned using multiple options. - -## Framework Memory +我们可以看到,有些内存部分的大小可以直接通过一个配置参数进行设置,有些则需要根据多个参数进行调整。 -The *framework heap memory* and *framework off-heap memory* options are not supposed to be changed without a good reason. -Adjust them only if you are sure that Flink needs more memory for some internal data structures or operations. -It can be related to a particular deployment environment or job structure, like high parallelism. -In addition, Flink dependencies, such as Hadoop may consume more direct or native memory in certain setups. +## 框架内存 -Note Neither heap nor off-heap versions of framework and task memory are currently isolated within Flink. -The separation of framework and task memory can be used in future releases for further optimizations. +通常情况下,不建议对*框架堆内存*和*框架堆外内存*进行调整。 +除非你非常肯定 Flink 的内部数据结构及操作需要更多的内存。 +这可能与具体的部署环境及作业结构有关,例如非常高的并发度。 +此外,Flink 的部分依赖(例如 Hadoop)在某些特定的情况下也可能会需要更多的直接内存或本地内存。 -## Capped Fractionated Components +提示 不管是堆内存还是堆外内存,Flink 中的框架内存和任务内存之间目前是没有隔离的。 +对框架和任务内存的区分,主要是为了在后续版本中做进一步优化。 -This section describes the configuration details of the following options which can be a fraction of a certain -[total memory](mem_setup.html#configure-total-memory): +## 受限的等比内存部分 -* *Network memory* can be a fraction of the *total Flink memory* -* *JVM overhead* can be a fraction of the *total process memory* +本节介绍下列内存部分的配置方法,它们都可以通过指定在[总内存](mem_setup.html#配置总内存)中所占比例的方式进行配置。 +* *网络内存*:可以配置占用 *Flink 总内存*的固定比例 +* *JVM 开销*:可以配置占用*进程总内存*的固定比例 -See also [detailed memory model](#overview). +请同时参考[概述部分](#概述)。 -The size of those components always has to be between its maximum and minimum value, otherwise Flink startup will fail. -The maximum and minimum values have defaults or can be explicitly set by corresponding configuration options. -For example, if only the following memory options are set: -- total Flink memory = 1000Mb, -- network min = 64Mb, -- network max = 128Mb, -- network fraction = 0.1 +这些内存部分的大小必须在相应的最大值、最小值范围内,否则 Flink 将无法启动。 +最大值、最小值具有默认值,也可以通过相应的配置参数进行设置。 +例如,如果仅配置下列参数: +- Flink 总内存 = 1000Mb +- 网络内存最小值 = 64Mb +- 网络内存最大值 = 128Mb +- 网络内存占比 = 0.1 -then the network memory will be 1000Mb x 0.1 = 100Mb which is within the range 64-128Mb. +那么网络内存的实际大小将会是 1000Mb x 0.1 = 100Mb,在 64-128Mb 的范围内。 -Notice if you configure the same maximum and minimum value it effectively means that its size is fixed to that value. +如果将最大值、最小值设置成相同大小,那相当于明确指定了该内存部分的大小。 -If the component memory is not explicitly configured, then Flink will use the fraction to calculate the memory size -based on the total memory. The calculated value is capped by its corresponding min/max options. -For example, if only the following memory options are set: -- total Flink memory = 1000Mb, -- network min = 128Mb, -- network max = 256Mb, -- network fraction = 0.1 +如果没有明确指定内存部分的大小,Flink 会根据总内存和占比计算出该内存部分的大小。 +计算得到的内存大小将受限于相应的最大值、最小。 +例如,如果仅配置下列参数: +- Flink 总内存 = 1000Mb +- 网络内存最小值 = 128Mb +- 网络内存最大值 = 256Mb +- 网络内存占比 = 0.1 -then the network memory will be 128Mb because the size derived from fraction is 100Mb and it is less than the minimum. +那么网络内存的实际大小将会是 128Mb,因为根据总内存和占比计算得到的内存大小 100Mb 小于最小值。 -It can also happen that the fraction is ignored if the sizes of the total memory and its other components are defined. -In this case, the network memory is the rest of the total memory. The derived value still has to be within its min/max -range otherwise the configuration fails. For example, suppose only the following memory options are set: -- total Flink memory = 1000Mb, -- task heap = 100Mb, -- network min = 64Mb, -- network max = 256Mb, -- network fraction = 0.1 +如果配置了总内存和其他内存部分的大小,那么 Flink 也有可能会忽略给定的占比。 +这种情况下,受限的等比内存部分的实际大小是总内存减去其他所有内存部分后剩余的部分。 +这样推导得出的内存大小必须符合最大值、最小值范围,否则 Flink 将无法启动。 +例如,如果仅配置下列参数: +- Flink 总内存 = 1000Mb, +- 任务堆内存 = 100Mb, +- 网络内存最小值 = 64Mb +- 网络内存最大值 = 256Mb +- 网络内存占比 = 0.1 -All other components of the total Flink memory have default values, including the default managed memory fraction. -Then the network memory is not the fraction (1000Mb x 0.1 = 100Mb) but the rest of the total Flink memory -which will either be within the range 64-256Mb or fail. +Flink 总内存中所有其他内存部分均有默认大小(包括托管内存的默认占比)。 +因此,网络内存的实际大小不是根据占比算出的大小(1000Mb x 0.1 = 100Mb),而是 Flink 总内存中剩余的部分。 +这个剩余部分的大小必须在 64-256Mb 的范围内,否则将会启动失败。 -## JVM Parameters +## JVM 参数 -Flink explicitly adds the following memory related JVM arguments while starting the task executor process, -based on the configured or derived memory component sizes: +Flink 启动 TaskExecutor 进程时,会根据配置的和自动推导出的各内存部分大小,显式地设置以下 JVM 参数: -|   **JVM Arguments**   |   **Value**   | -| :---------------------------------------- | :----------------------------------------- | -| *-Xmx* and *-Xms* | Framework + Task Heap Memory | -| *-XX:MaxDirectMemorySize* | Framework + Task Off-Heap + Network Memory | -| *-XX:MaxMetaspaceSize* | JVM Metaspace | +|   **JVM 参数**   |   **值**   | +| :---------------------------------- | :------------------------------- | +| *-Xmx* 和 *-Xms* | 框架堆内存 + 任务堆内存 | +| *-XX:MaxDirectMemorySize* | 框架堆外内存 + 任务堆外内存 + 网络内存 | +| *-XX:MaxMetaspaceSize* | JVM Metaspace | {:.table-bordered}
-See also [detailed memory model](#overview). +请同时参考[概述部分](#概述)。 -## Local Execution -If you start Flink locally on your machine as a single java program without creating a cluster (e.g. from your IDE) -then all components are ignored except for the following: +## 本地执行 +如果你是将 Flink 作为一个单独的 Java 程序运行在你的电脑本地而非创建一个集群(例如在 IDE 中),那么只有下列配置会生效,其他配置参数则不会起到任何效果: -|   **Memory component**   |   **Relevant options**   |   **Default value for the local execution**   | -| :------------------------------------------- | :-------------------------------------------------------------------------------------------- | :---------------------------------------------------------------------------- | -| Task heap | [`taskmanager.memory.task.heap.size`](../config.html#taskmanager-memory-task-heap-size) | infinite | -| Task off-heap | [`taskmanager.memory.task.off-heap.size`](../config.html#taskmanager-memory-task-off-heap-size) | infinite | -| Managed memory | [`taskmanager.memory.managed.size`](../config.html#taskmanager-memory-managed-size) | 128Mb | -| Network memory | [`taskmanager.memory.network.min`](../config.html#taskmanager-memory-network-min)
[`taskmanager.memory.network.max`](../config.html#taskmanager-memory-network-max) | 64Mb | +|   **组成部分**   |   **配置参数**   |   **本地执行时的默认值**   | +| :---------------------------------- | :------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | :------------------------------------------ | +| 任务堆内存 | [`taskmanager.memory.task.heap.size`](../config.html#taskmanager-memory-task-heap-size) | 无穷大 | +| 任务堆外内存 | [`taskmanager.memory.task.off-heap.size`](../config.html#taskmanager-memory-task-off-heap-size) | 无穷大 | +| 托管内存 | [`taskmanager.memory.managed.size`](../config.html#taskmanager-memory-managed-size) | 128Mb | +| 网络内存 | [`taskmanager.memory.network.min`](../config.html#taskmanager-memory-network-min)
[`taskmanager.memory.network.max`](../config.html#taskmanager-memory-network-max) | 64Mb | {:.table-bordered}
-All of the components listed above can be but do not have to be explicitly configured for the local execution. -If they are not configured they are set to their default values. [Task heap memory](mem_setup.html#task-operator-heap-memory) and -*task off-heap memory* are considered to be infinite (*Long.MAX_VALUE* bytes) and [managed memory](mem_setup.html#managed-memory) -has a default value of 128Mb only for the local execution mode. +本地执行模式下,上面列出的所有内存部分均可以但不是必须进行配置。 +如果未配置,则会采用默认值。 +其中,[*任务堆内存*](mem_setup.html#任务算子堆内存)和*任务堆外内存*的默认值无穷大(*Long.MAX_VALUE*字节),以及[托管内存](mem_setup.html#托管内存)的默认值 128Mb 均只针对本地执行模式。 -Note The task heap size is not related in any way to the real heap size in this case. -It can become relevant for future optimizations coming with next releases. The actual JVM heap size of the started -local process is not controlled by Flink and depends on how you start the process. -If you want to control the JVM heap size you have to explicitly pass the corresponding JVM arguments, e.g. *-Xmx*, *-Xms*. +提示 这种情况下,任务堆内存的大小与实际的堆空间大小无关。 +该配置参数可能与后续版本中的进一步优化相关。 +本地执行模式下,JVM 堆空间的实际大小不受 Flink 掌控,而是取决于本地执行进程是如何启动的。 +如果希望控制 JVM 的堆空间大小,可以在启动进程时明确地指定相关的 JVM 参数,即 *-Xmx* 和 *-Xms*. diff --git a/docs/ops/memory/mem_migration.md b/docs/ops/memory/mem_migration.md index dff7508b000e94e24d02d2d0f45f69b09b824bac..eb2db4fa5fd2724f87395df12e44a5f85db11946 100644 --- a/docs/ops/memory/mem_migration.md +++ b/docs/ops/memory/mem_migration.md @@ -123,11 +123,10 @@ The following options are deprecated but if they are still used they will be int Although, the network memory configuration has not changed too much it is recommended to verify its configuration. It can change if other memory components have new sizes, e.g. the total memory which the network can be a fraction of. -See also [new detailed memory model](mem_setup.html#detailed-memory-model). +See also [new detailed memory model](mem_detail.html). -The container cut-off configuration options, [`containerized.heap-cutoff-ratio`](config.html#containerized-heap-cutoff-ratio) -and [`containerized.heap-cutoff-min`](config.html#containerized-heap-cutoff-min), have no effect for task manager processes anymore -but they still have the same semantics for the job manager process. See also [how to migrate container cut-off](#container-cut-off-memory). +The container cut-off configuration options, `containerized.heap-cutoff-ratio` and `containerized.heap-cutoff-min`, +have no effect anymore. See also [how to migrate container cut-off](#container-cut-off-memory). ## Total Memory (Previously Heap Memory) @@ -158,7 +157,7 @@ Additionally, you can now have more direct control over the JVM heap assigned to ([`taskmanager.memory.task.heap.size`](../config.html#taskmanager-memory-task-heap-size)), see also [Task (Operator) Heap Memory](mem_setup.html#task-operator-heap-memory). The JVM heap memory is also used by the heap state backends ([MemoryStateBackend](../state/state_backends.html#the-memorystatebackend) -or [FsStateBackend](../state/state_backends.html#the-fsstatebackend) if it is chosen for streaming jobs. +or [FsStateBackend](../state/state_backends.html#the-fsstatebackend)) if it is chosen for streaming jobs. A part of the JVM heap is now always reserved for Flink framework ([`taskmanager.memory.framework.heap.size`](../config.html#taskmanager-memory-framework-heap-size)). @@ -217,7 +216,7 @@ The other direct or native off-heap memory consumers can now be addressed by the * Task off-heap memory ([`taskmanager.memory.task.off-heap.size`](../config.html#taskmanager-memory-task-off-heap-size)) * Framework off-heap memory ([`taskmanager.memory.framework.off-heap.size`](../config.html#taskmanager-memory-framework-off-heap-size)) * JVM metaspace ([`taskmanager.memory.jvm-metaspace.size`](../config.html#taskmanager-memory-jvm-metaspace-size)) -* JVM overhead (see also [detailed new memory model](mem_setup.html#detailed-memory-model)) +* JVM overhead (see also [detailed new memory model](mem_detail.html)) Note The job manager still has container cut-off memory configuration options. The mentioned configuration options remain valid for the job manager in the same way as before. diff --git a/docs/ops/memory/mem_migration.zh.md b/docs/ops/memory/mem_migration.zh.md index dff7508b000e94e24d02d2d0f45f69b09b824bac..72d2b01fed95f8ec7f034c0868a35481348c2b73 100644 --- a/docs/ops/memory/mem_migration.zh.md +++ b/docs/ops/memory/mem_migration.zh.md @@ -1,5 +1,5 @@ --- -title: "Migration Guide" +title: "升级指南" nav-parent_id: ops_mem nav-pos: 5 --- @@ -22,67 +22,63 @@ specific language governing permissions and limitations under the License. --> -The [memory setup of task managers](mem_setup.html) has changed a lot with the 1.10 release. Many configuration options -were removed or their semantics changed. This guide will help you to migrate the memory configuration from Flink -[<= *1.9*](https://ci.apache.org/projects/flink/flink-docs-release-1.9/ops/mem_setup.html) to >= *1.10*. +在 1.10 版本中,Flink 的 [TaskExecutor 内存配置方法](mem_setup.html)发生了较大的变化。 +部分配置参数被移除了,或是语义上发生了变化。 +本篇升级指南将介绍如何将 [*Flink 1.9 及以前版本*](https://ci.apache.org/projects/flink/flink-docs-release-1.9/ops/mem_setup.html)的内存配置升级到 *Flink 1.10 及以后版本*。 * toc {:toc}
- Warning: It is important to review this guide because the legacy and new memory configuration can - result in different sizes of memory components. If you try to reuse your Flink configuration from older versions - before 1.10, it can result in changes to the behavior, performance or even configuration failures of your application. + 注意: 请仔细阅读本篇升级指南。 + 使用原本的和新的内存配制方法可能会使内存组成部分具有截然不同的大小。 + 未经调整直接沿用 Flink 1.10 以前版本的配置文件,可能导致应用的行为、性能发生变化,甚至造成应用执行失败。
-Note Before version *1.10*, Flink did not require that memory related options are set at all -as they all had default values. The [new memory configuration](mem_setup.html#configure-total-memory) requires -that at least one subset of the following options is configured explicitly, otherwise the configuration will fail: +提示 在 *1.10* 版本之前,Flink 不要求用户一定要配置内存相关的参数,因为这些参数都具有默认值。 +[新的内存配置](mem_setup.html#配置总内存)要求用户至少指定下列配置参数(或参数组合)的其中之一,否则 Flink 将无法启动。 * [`taskmanager.memory.flink.size`](../config.html#taskmanager-memory-flink-size) * [`taskmanager.memory.process.size`](../config.html#taskmanager-memory-process-size) -* [`taskmanager.memory.task.heap.size`](../config.html#taskmanager-memory-task-heap-size) and [`taskmanager.memory.managed.size`](../config.html#taskmanager-memory-managed-size) +* [`taskmanager.memory.task.heap.size`](../config.html#taskmanager-memory-task-heap-size) 和 [`taskmanager.memory.managed.size`](../config.html#taskmanager-memory-managed-size) -The [default `flink-conf.yaml`](#default-configuration-in-flink-confyaml) shipped with Flink sets [`taskmanager.memory.process.size`](../config.html#taskmanager-memory-process-size) -to make the default memory configuration consistent. +Flink 自带的[默认 flink-conf.yaml](#flink-confyaml-中的默认配置) 文件指定了 [`taskmanager.memory.process.size`](../config.html#taskmanager-memory-process-size),以便与此前的行为保持一致。 -This [spreadsheet](https://docs.google.com/spreadsheets/d/1mJaMkMPfDJJ-w6nMXALYmTc4XxiV30P5U7DzgwLkSoE) can also help -to evaluate and compare the results of the legacy and new memory computations. +可以使用这张[电子表格](https://docs.google.com/spreadsheets/d/1mJaMkMPfDJJ-w6nMXALYmTc4XxiV30P5U7DzgwLkSoE)来估算和比较原本的和新的内存配置下的计算结果。 -## Changes in Configuration Options +## 配置参数变化 -This chapter shortly lists all changes to Flink's memory configuration options introduced with the *1.10* release. -It also references other chapters for more details about migrating to the new configuration options. +本节简要列出了 *Flink 1.10* 引入的配置参数变化,并援引其他章节中关于如何升级到新配置参数的相关描述。 -The following options are completely removed. If they are still used, they will be ignored. +下列配置参数已被彻底移除,配置它们将不会产生任何效果。 - - + + - + - +
Removed optionNote移除的配置参数备注
taskmanager.memory.fraction
- Check the description of the new option taskmanager.memory.managed.fraction. - The new option has different semantics and the value of the deprecated option usually has to be adjusted. - See also how to migrate managed memory. + 请参考新配置参数 taskmanager.memory.managed.fraction 的相关描述。 + 新的配置参数与被移除的配置参数在语义上有所差别,因此其配置值通常也需要做出适当调整。 + 请参考如何升级托管内存
taskmanager.memory.off-heap
On-heap managed memory is no longer supported. See also how to migrate managed memory.Flink 不再支持堆上的(On-Heap)托管内存。请参考如何升级托管内存
taskmanager.memory.preallocate
Pre-allocation is no longer supported and managed memory is always allocated lazily. See also how to migrate managed memory.Flink 不再支持内存预分配,今后托管内存将都是惰性分配的。请参考如何升级托管内存
-The following options are deprecated but if they are still used they will be interpreted as new options for backwards compatibility: +下列配置参数将被弃用,出于向后兼容性考虑,配置它们将被解读成对应的新配置参数。 @@ -96,15 +92,15 @@ The following options are deprecated but if they are still used they will be int - + @@ -121,116 +117,102 @@ The following options are deprecated but if they are still used they will be int
taskmanager.heap.size
- See also how to migrate total memory. + 请参考如何升级总内存
taskmanager.memory.size
taskmanager.memory.managed.size, see also how to migrate managed memory.taskmanager.memory.managed.size。请参考如何升级托管内存
taskmanager.network.memory.min
-Although, the network memory configuration has not changed too much it is recommended to verify its configuration. -It can change if other memory components have new sizes, e.g. the total memory which the network can be a fraction of. -See also [new detailed memory model](mem_setup.html#detailed-memory-model). +尽管网络内存的配置参数没有发生太多变化,我们仍建议您检查其配置结果。 +网络内存的大小可能会受到其他内存部分大小变化的影响,例如总内存变化时,根据占比计算出的网络内存也可能发生变化。 +请参考[内存模型详解](mem_detail.html)。 -The container cut-off configuration options, [`containerized.heap-cutoff-ratio`](config.html#containerized-heap-cutoff-ratio) -and [`containerized.heap-cutoff-min`](config.html#containerized-heap-cutoff-min), have no effect for task manager processes anymore -but they still have the same semantics for the job manager process. See also [how to migrate container cut-off](#container-cut-off-memory). +容器切除(Cut-Off)内存相关的配置参数(`containerized.heap-cutoff-ratio` 和 `containerized.heap-cutoff-min`)将不再对进程生效。 -## Total Memory (Previously Heap Memory) +## 总内存(原堆内存) -The previous options which were responsible for the total memory used by Flink are `taskmanager.heap.size` or `taskmanager.heap.mb`. -Despite their naming, they included not only JVM heap but also other off-heap memory components. The options have been deprecated. +在原本的内存配置方法中,用于指定用于 Flink 的总内存的配置参数是 `taskmanager.heap.size` 或 `taskmanager.heap.mb`。 +尽管这两个参数以“堆(Heap)”命名,实际上它们指定的内存既包含了 JVM 堆内存,也包含了其他堆外内存部分。 +这两个配置参数目前已被弃用。 -The Mesos integration also had a separate option with the same semantics: `mesos.resourcemanager.tasks.mem` which has also been removed. +Flink 在 Mesos 上还有另一个具有同样语义的配置参数 `mesos.resourcemanager.tasks.mem`,目前也已经被弃用。 -If the mentioned legacy options are used without specifying the corresponding new options, -they will be directly translated into the following new options: -* Total Flink memory ([`taskmanager.memory.flink.size`](../config.html#taskmanager-memory-flink-size)) for standalone deployments -* Total process memory ([`taskmanager.memory.process.size`](../config.html#taskmanager-memory-process-size)) for containerized deployments (Yarn or Mesos) +如果配置了上述弃用的参数,同时又没有配置与之对应的新配置参数,那它们将按如下规则对应到新的配置参数。 +* 独立部署模式(Standalone Deployment)下:Flink 总内存([`taskmanager.memory.flink.size`](../config.html#taskmanager-memory-flink-size)) +* 容器化部署模式(Containerized Deployement)下(Yarn、Mesos):进程总内存([`taskmanager.memory.process.size`](../config.html#taskmanager-memory-process-size)) -It is also recommended to use these new options instead of the legacy ones as they might be completely removed in the following releases. +建议您尽早使用新的配置参数取代启用的配置参数,它们在今后的版本中可能会被彻底移除。 -See also [how to configure total memory now](mem_setup.html#configure-total-memory). +请参考[如何配置总内存](mem_setup.html#配置总内存). -## JVM Heap Memory +## JVM 堆内存 -JVM heap memory previously consisted of the managed memory (if configured to be on-heap) and the rest -which included any other usages of heap memory. This rest was always implicitly derived as the remaining part of the total memory, -see also [how to migrate managed memory](#managed-memory). +此前,JVM 堆空间由托管内存(仅在配置为堆上时)及 Flink 用到的所有其他堆内存组成。 +这里的其他堆内存是由总内存减去所有其他非堆内存得到的。 +请参考[如何升级托管内存](#托管内存)。 -Now, if only *total Flink memory* or *total process memory* is configured, then the JVM heap is also derived as the rest of -what is left after subtracting all other components from the total memory, see also [how to configure total memory](mem_setup.html#configure-total-memory). +现在,如果仅配置了*Flink总内存*或*进程总内存*,JVM 的堆空间依然是根据总内存减去所有其他非堆内存得到的。 +请参考[如何配置总内存](mem_setup.html#配置总内存)。 -Additionally, you can now have more direct control over the JVM heap assigned to the operator tasks -([`taskmanager.memory.task.heap.size`](../config.html#taskmanager-memory-task-heap-size)), -see also [Task (Operator) Heap Memory](mem_setup.html#task-operator-heap-memory). -The JVM heap memory is also used by the heap state backends ([MemoryStateBackend](../state/state_backends.html#the-memorystatebackend) -or [FsStateBackend](../state/state_backends.html#the-fsstatebackend) if it is chosen for streaming jobs. +此外,你现在可以更直接地控制用于任务和算子的 JVM 的堆内存([`taskmanager.memory.task.heap.size`](../config.html#taskmanager-memory-task-heap-size)),详见[任务堆内存](mem_setup.html#任务算子堆内存)。 +如果流处理作业选择使用 Heap State Backend([MemoryStateBackend](../state/state_backends.html#memorystatebackend) +或 [FsStateBackend](../state/state_backends.html#fsstatebackend)),那么它同样需要使用 JVM 堆内存。 -A part of the JVM heap is now always reserved for Flink framework -([`taskmanager.memory.framework.heap.size`](../config.html#taskmanager-memory-framework-heap-size)). -See also [Framework memory](mem_detail.html#framework-memory). +Flink 现在总是会预留一部分 JVM 堆内存供框架使用([`taskmanager.memory.framework.heap.size`](../config.html#taskmanager-memory-framework-heap-size))。 +请参考[框架内存](mem_detail.html#框架内存)。 -## Managed Memory +## 托管内存 -See also [how to configure managed memory now](mem_setup.html#managed-memory). +请参考[如何配置托管内存](mem_setup.html#托管内存)。 -### Explicit Size +### 明确的大小 -The previous option to configure managed memory size (`taskmanager.memory.size`) was renamed to -[`taskmanager.memory.managed.size`](../config.html#taskmanager-memory-managed-size) and deprecated. -It is recommended to use the new option because the legacy one can be removed in future releases. +原本用于指定明确的托管内存大小的配置参数(`taskmanager.memory.size`)已被弃用,与它具有相同语义的新配置参数为 [`taskmanager.memory.managed.size`](../config.html#taskmanager-memory-managed-size)。 +建议使用新的配置参数,原本的配置参数在今后的版本中可能会被彻底移除。 -### Fraction +### 占比 -If not set explicitly, the managed memory could be previously specified as a fraction (`taskmanager.memory.fraction`) -of the total memory minus network memory and container cut-off (only for [Yarn](../deployment/yarn_setup.html) and -[Mesos](../deployment/mesos.html) deployments). This option has been completely removed and will have no effect if still used. -Please, use the new option [`taskmanager.memory.managed.fraction`](../config.html#taskmanager-memory-managed-fraction) instead. -This new option will set the [managed memory](mem_setup.html#managed-memory) to the specified fraction of the -[total Flink memory](mem_setup.html#configure-total-memory) if its size is not set explicitly by -[`taskmanager.memory.managed.size`](../config.html#taskmanager-memory-managed-size). +此前,如果不指定明确的大小,也可以将托管内存配置为占用总内存减去网络内存和容器切除内存(仅在 [Yarn](../deployment/yarn_setup.html) 和 +[Mesos](../deployment/mesos.html) 上)之后剩余部分的固定比例(`taskmanager.memory.fraction`)。 +该配置参数已经被彻底移除,配置它不会产生任何效果。 +请使用新的配置参数 [`taskmanager.memory.managed.fraction`](../config.html#taskmanager-memory-managed-fraction)。 +在未通过 [`taskmanager.memory.managed.size`](../config.html#taskmanager-memory-managed-size) 指定明确大小的情况下,新的配置参数将指定[托管内存](mem_setup.html#托管内存)在 [Flink 总内存](mem_setup.html#配置总内存)中的所占比例。 -### RocksDB state +### RocksDB State Backend -If the [RocksDBStateBackend](../state/state_backends.html#the-rocksdbstatebackend) is chosen for a streaming job, -its native memory consumption should now be accounted for in [managed memory](mem_setup.html#managed-memory). -The RocksDB memory allocation is limited by the [managed memory](mem_setup.html#managed-memory) size. -This should prevent the killing of containers on [Yarn](../deployment/yarn_setup.html) or [Mesos](../deployment/mesos.html). -You can disable the RocksDB memory control by setting [state.backend.rocksdb.memory.managed](../config.html#state-backend-rocksdb-memory-managed) -to `false`. See also [how to migrate container cut-off](#container-cut-off-memory). +流处理作业如果选择使用 [RocksDBStateBackend](../state/state_backends.html#rocksdbstatebackend),它使用的本地内存现在也被归为[托管内存](mem_setup.html#托管内存)。 +默认情况下,RocksDB 将限制其内存用量不超过[托管内存](mem_setup.html#托管内存)大小,以避免在 [Yarn](../deployment/yarn_setup.html) 或 [Mesos](../deployment/mesos.html) 上容器被杀。你也可以通过设置 [state.backend.rocksdb.memory.managed](../config.html#state-backend-rocksdb-memory-managed) 来关闭 RocksDB 的内存控制。 +请参考[如何升级容器切除内存](#容器切除cut-off内存)。 -### Other changes +### 其他变化 -Additionally, the following changes have been made: -* The [managed memory](mem_setup.html#managed-memory) is always off-heap now. The configuration option `taskmanager.memory.off-heap` is removed and will have no effect anymore. -* The [managed memory](mem_setup.html#managed-memory) now uses native memory which is not direct memory. It means that the managed memory is no longer accounted for in the JVM direct memory limit. -* The [managed memory](mem_setup.html#managed-memory) is always lazily allocated now. The configuration option `taskmanager.memory.preallocate` is removed and will have no effect anymore. +此外,Flink 1.10 对托管内存还引入了下列变化: +* [托管内存](mem_setup.html#托管内存)现在总是在堆外。配置参数 `taskmanager.memory.off-heap` 已被彻底移除,配置它不会产生任何效果。 +* [托管内存](mem_setup.html#托管内存)现在使用本地内存而非直接内存。这意味着托管内存将不在 JVM 直接内存限制的范围内。 +* [托管内存](mem_setup.html#托管内存)现在总是惰性分配的。配置参数 `taskmanager.memory.preallocate` 已被彻底移除,配置它不会产生任何效果。 -## Container Cut-Off Memory +## 容器切除(Cut-Off)内存 -For containerized deployments, you could previously specify a cut-off memory. This memory could accommodate for unaccounted memory allocations. -Dependencies which were not directly controlled by Flink were the main source of those allocations, e.g. RocksDB, internals of JVM, etc. -This is no longer available and the related configuration options (`containerized.heap-cutoff-ratio` and `containerized.heap-cutoff-min`) -will have no effect on the task manager process anymore. The new memory model introduced more specific memory components, -described further, to address these concerns. +在容器化部署模式(Containerized Deployment)下,此前你可以指定切除内存。 +这部分内存将预留给所有未被 Flink 计算在内的内存开销。 +其主要来源是不受 Flink 直接管理的依赖使用的内存,例如 RocksDB、JVM 内部开销等。 +相应的配置参数(`containerized.heap-cutoff-ratio` 和 `containerized.heap-cutoff-min`)现在不再对 TaskExecutor 生效。 +新的内存配置方法引入了新的内存组成部分来具体描述这些内存用量。 -In streaming jobs which use [RocksDBStateBackend](../state/state_backends.html#the-rocksdbstatebackend), the RocksDB -native memory consumption should be accounted for as a part of the [managed memory](mem_setup.html#managed-memory) now. -The RocksDB memory allocation is also limited by the configured size of the [managed memory](mem_setup.html#managed-memory). -See also [migrating managed memory](#managed-memory) and [how to configure managed memory now](mem_setup.html#managed-memory). +流处理作业如果使用了 [RocksDBStateBackend](../state/state_backends.html#the-rocksdbstatebackend),RocksDB 使用的本地内存现在将被归为[托管内存](mem_setup.html#托管内存)。 +默认情况下,RocksDB 将限制其内存用量不超过[托管内存](mem_setup.html#托管内存)大小。 +请同时参考[如何升级托管内存](#托管内存)以及[如何配置托管内存](mem_setup.html#托管内存)。 -The other direct or native off-heap memory consumers can now be addressed by the following new configuration options: -* Task off-heap memory ([`taskmanager.memory.task.off-heap.size`](../config.html#taskmanager-memory-task-off-heap-size)) -* Framework off-heap memory ([`taskmanager.memory.framework.off-heap.size`](../config.html#taskmanager-memory-framework-off-heap-size)) -* JVM metaspace ([`taskmanager.memory.jvm-metaspace.size`](../config.html#taskmanager-memory-jvm-metaspace-size)) -* JVM overhead (see also [detailed new memory model](mem_setup.html#detailed-memory-model)) +其他直接内存或堆外内存开销,现在可以通过下列配置参数进行设置: +* 任务堆外内存([`taskmanager.memory.task.off-heap.size`](../config.html#taskmanager-memory-task-off-heap-size)) +* 框架堆外内存([`taskmanager.memory.framework.off-heap.size`](../config.html#taskmanager-memory-framework-off-heap-size)) +* JVM Metaspace([`taskmanager.memory.jvm-metaspace.size`](../config.html#taskmanager-memory-jvm-metaspace-size)) +* JVM 开销(请参考[内存模型详解](mem_detail.html)) -Note The job manager still has container cut-off memory configuration options. -The mentioned configuration options remain valid for the job manager in the same way as before. +提示 JobManager 进程仍保留了容器切除内存,相关配置项和此前一样仍对 JobManager 生效。 -## Default Configuration in flink-conf.yaml +## flink-conf.yaml 中的默认配置 -This section describes the changes of the default `flink-conf.yaml` shipped with Flink. +本节描述 Flink 自带的默认 `flink-conf.yaml` 文件中的变化。 -The total memory (`taskmanager.heap.size`) is replaced by [`taskmanager.memory.process.size`](../config.html#taskmanager-memory-process-size) -in the default `flink-conf.yaml`. The value is also increased from 1024Mb to 1568Mb. -See also [how to configure total memory now](mem_setup.html#configure-total-memory). +在默认 `flink-conf.yaml` 文件中,原本的总内存(`taskmanager.heap.size`)被新的配置项 [`taskmanager.memory.process.size`](../config.html#taskmanager-memory-process-size) 所取代。 +默认值从 1024Mb 增加到了 1568Mb。 +请参考[如何配置总内存](mem_setup.html#配置总内存)。
- Warning: If you use the new default `flink-conf.yaml` it can result in different sizes of - the memory components and can lead to performance changes. + 注意: 使用新的默认 `flink-conf.yaml` 可能会造成各内存部分的大小发生变化,从而产生性能变化。
diff --git a/docs/ops/memory/mem_setup.md b/docs/ops/memory/mem_setup.md index 7e27ee2cf4dd3943d02ffb4ad1c9f57cfb108b07..32cfd9c554754e76410120a630709e177732f0b6 100644 --- a/docs/ops/memory/mem_setup.md +++ b/docs/ops/memory/mem_setup.md @@ -47,7 +47,7 @@ and by the JVM to run the process. The *total Flink memory* consumption includes
-If you run FIink locally (e.g. from your IDE) without creating a cluster, then only a subset of the memory configuration +If you run Flink locally (e.g. from your IDE) without creating a cluster, then only a subset of the memory configuration options are relevant, see also [local execution](mem_detail.html#local-execution) for more details. Otherwise, the simplest way to setup memory in Flink is to configure either of the two following options: @@ -55,7 +55,7 @@ Otherwise, the simplest way to setup memory in Flink is to configure either of t * Total process memory ([`taskmanager.memory.process.size`](../config.html#taskmanager-memory-process-size)) The rest of the memory components will be adjusted automatically, based on default values or additionally configured options. -[Here](mem_detail.html#detailed-memory-model) are more details about the other memory components. +[Here](mem_detail.html) are more details about the other memory components. Configuring *total Flink memory* is better suited for standalone deployments where you want to declare how much memory is given to Flink itself. The *total Flink memory* splits up into JVM heap, [managed memory size](#managed-memory) @@ -87,7 +87,7 @@ to specify explicitly both [task heap](#task-operator-heap-memory) and [managed It gives more control over the available JVM heap to Flink’s tasks and its [managed memory](#managed-memory). The rest of the memory components will be adjusted automatically, based on default values or additionally configured options. -[Here](mem_detail.html#detailed-memory-model) are more details about the other memory components. +[Here](mem_detail.html) are more details about the other memory components. Note If you have configured the task heap and managed memory explicitly, it is recommended to set neither *total process memory* nor *total Flink memory*. Otherwise, it may easily lead to memory configuration conflicts. @@ -130,4 +130,4 @@ see also [JVM parameters](mem_detail.html#jvm-parameters). Note The *network memory* is also part of JVM *direct memory* but it is managed by Flink and guaranteed to never exceed its configured size. Therefore, resizing the *network memory* will not help in this situation. -See also [the detailed memory model](mem_detail.html#detailed-memory-model). +See also [the detailed memory model](mem_detail.html). diff --git a/docs/ops/memory/mem_setup.zh.md b/docs/ops/memory/mem_setup.zh.md index 7e27ee2cf4dd3943d02ffb4ad1c9f57cfb108b07..0e30b53744676fdc60010d47694613937b272ecf 100644 --- a/docs/ops/memory/mem_setup.zh.md +++ b/docs/ops/memory/mem_setup.zh.md @@ -1,5 +1,5 @@ --- -title: "Set up Task Executor Memory" +title: "配置 TaskExecutor 内存" nav-parent_id: ops_mem nav-pos: 1 --- @@ -22,112 +22,98 @@ specific language governing permissions and limitations under the License. --> -Apache Flink provides efficient workloads on top of the JVM by tightly controlling the memory usage of its various components. -While the community strives to offer sensible defaults to all configurations, the full breadth of applications -that users deploy on Flink means this isn't always possible. To provide the most production value to our users, -Flink allows both high level and fine-grained tuning of memory allocation within clusters. +Apache Flink 基于 JVM 的高效处理能力,依赖于其对各组件内存用量的细致掌控。 +考虑到用户在 Flink 上运行的应用的多样性,尽管社区已经努力为所有配置项提供合理的默认值,仍无法满足所有情况下的需求。 +为了给用户生产提供最大化的价值, Flink 允许用户在整体上以及细粒度上对集群的内存分配进行调整。 * toc {:toc} -The further described memory configuration is applicable starting with the release version *1.10*. If you upgrade Flink -from earlier versions, check the [migration guide](mem_migration.html) because many changes were introduced with the *1.10* release. +本文接下来介绍的内存配置方法适用于 *1.10* 及以上版本。 +Flink 在 1.10 版本中对内存配置部分进行了较大幅度的改动,从早期版本升级的用户请参考[升级指南](mem_migration.html)。 -Note This memory setup guide is relevant only for task executors! -Check [job manager related configuration options](../config.html#jobmanager-heap-size) for the memory setup of job manager. +提示 本篇内存配置文档仅针对 TaskExecutor!关于 JobManager 的内存配置请参考 [JobManager 相关配置参数](../config.html#jobmanager-heap-size)。 -## Configure Total Memory +## 配置总内存 -The *total process memory* of Flink JVM processes consists of memory consumed by Flink application (*total Flink memory*) -and by the JVM to run the process. The *total Flink memory* consumption includes usage of JVM heap, -*managed memory* (managed by Flink) and other direct (or native) memory. +Flink JVM 进程的*进程总内存(Total Process Memory)*包含了由 Flink 应用使用的内存(*Flink 总内存*)以及由运行 Flink 的 JVM 使用的内存。 +其中,*Flink 总内存(Total Flink Memory)*包括 JVM 堆内存(Heap Memory)、*托管内存(Managed Memory)*以及其他直接内存(Direct Memory)或本地内存(Native Memory)。
Simple memory model

-If you run FIink locally (e.g. from your IDE) without creating a cluster, then only a subset of the memory configuration -options are relevant, see also [local execution](mem_detail.html#local-execution) for more details. +如果你是在本地运行 Flink(例如在 IDE 中)而非创建一个集群,那么本文介绍的配置并非所有都是适用的,详情请参考[本地执行](mem_detail.html#本地执行)。 -Otherwise, the simplest way to setup memory in Flink is to configure either of the two following options: -* Total Flink memory ([`taskmanager.memory.flink.size`](../config.html#taskmanager-memory-flink-size)) -* Total process memory ([`taskmanager.memory.process.size`](../config.html#taskmanager-memory-process-size)) +其他情况下,配置 Flink 内存最简单的方法就是配置下列两个参数中的任意一个。 +* Flink 总内存([`taskmanager.memory.flink.size`](../config.html#taskmanager-memory-flink-size)) +* 进程总内存([`taskmanager.memory.process.size`](../config.html#taskmanager-memory-process-size)) -The rest of the memory components will be adjusted automatically, based on default values or additionally configured options. -[Here](mem_detail.html#detailed-memory-model) are more details about the other memory components. +Flink 会根据默认值或其他配置参数自动调整剩余内存部分的大小。关于各内存部分的更多细节,请参考[相关文档](mem_detail.html)。 -Configuring *total Flink memory* is better suited for standalone deployments where you want to declare how much memory -is given to Flink itself. The *total Flink memory* splits up into JVM heap, [managed memory size](#managed-memory) -and *direct memory*. +对于独立部署模式(Standalone Deployment),如果你希望指定由 Flink 应用本身使用的内存大小,最好选择配置 *Flink 总内存*。 +*Flink 总内存*会进一步划分为 JVM 堆内存、[托管内存](#托管内存)和*直接内存*。 -If you configure *total process memory* you declare how much memory in total should be assigned to the Flink *JVM process*. -For the containerized deployments it corresponds to the size of the requested container, see also -[how to configure memory for containers](mem_tuning.html#configure-memory-for-containers) -([Kubernetes](../deployment/kubernetes.html), [Yarn](../deployment/yarn_setup.html) or [Mesos](../deployment/mesos.html)). +通过配置*进程总内存*可以指定由 Flink *JVM 进程*使用的总内存大小。 +对于容器化部署模式(Containerized Deployment),这相当于申请的容器(Container)大小,详情请参考[如何配置容器内存](mem_tuning.html#容器container的内存配置)([Kubernetes](../deployment/kubernetes.html)、[Yarn](../deployment/yarn_setup.html) 或 [Mesos](../deployment/mesos.html))。 -Another way to setup the memory is to set [task heap](#task-operator-heap-memory) and [managed memory](#managed-memory) -([`taskmanager.memory.task.heap.size`](../config.html#taskmanager-memory-task-heap-size) and [`taskmanager.memory.managed.size`](../config.html#taskmanager-memory-managed-size)). -This more fine-grained approach is described in more detail [here](#configure-heap-and-managed-memory). +此外,还可以通过设置[任务堆内存(Task Heap Memory)](#任务算子堆内存)和[托管内存](#托管内存)的方式进行内存配置([`taskmanager.memory.task.heap.size`](../config.html#taskmanager-memory-task-heap-size) 和 [`taskmanager.memory.managed.size`](../config.html#taskmanager-memory-managed-size))。 +这是一种更细粒度的配置方式,更多细节请参考[相关文档](#配置堆内存和托管内存)。 -Note One of the three mentioned ways has to be used to configure Flink’s memory (except for local execution), or the Flink startup will fail. -This means that one of the following option subsets, which do not have default values, have to be configured explicitly: +提示 以上三种方式中,用户需要至少选择其中一种进行配置(本地运行除外),否则 Flink 将无法启动。 +这意味着,用户需要从以下无默认值的配置参数(或参数组合)中选择一个给出明确的配置: * [`taskmanager.memory.flink.size`](../config.html#taskmanager-memory-flink-size) * [`taskmanager.memory.process.size`](../config.html#taskmanager-memory-process-size) -* [`taskmanager.memory.task.heap.size`](../config.html#taskmanager-memory-task-heap-size) and [`taskmanager.memory.managed.size`](../config.html#taskmanager-memory-managed-size) +* [`taskmanager.memory.task.heap.size`](../config.html#taskmanager-memory-task-heap-size) 和 [`taskmanager.memory.managed.size`](../config.html#taskmanager-memory-managed-size) -Note Explicitly configuring both *total process memory* and *total Flink memory* is not recommended. -It may lead to deployment failures due to potential memory configuration conflicts. Additional configuration -of other memory components also requires caution as it can produce further configuration conflicts. +提示 不建议同时设置*进程总内存*和 *Flink 总内存*。 +这可能会造成内存配置冲突,从而导致部署失败。 +额外配置其他内存部分时,同样需要注意可能产生的配置冲突。 -## Configure Heap and Managed Memory +## 配置堆内存和托管内存 -As mentioned before in [total memory description](#configure-total-memory), another way to setup memory in Flink is -to specify explicitly both [task heap](#task-operator-heap-memory) and [managed memory](#managed-memory). -It gives more control over the available JVM heap to Flink’s tasks and its [managed memory](#managed-memory). +如[配置总内存](#配置总内存)中所述,另一种配置 Flink 内存的方式是同时设置[任务堆内存](#任务算子堆内存)和[托管内存](#托管内存)。 +通过这种方式,用户可以更好地掌控用于 Flink 任务的 JVM 堆内存及 Flink 的[托管内存](#托管内存)大小。 -The rest of the memory components will be adjusted automatically, based on default values or additionally configured options. -[Here](mem_detail.html#detailed-memory-model) are more details about the other memory components. +Flink 会根据默认值或其他配置参数自动调整剩余内存部分的大小。关于各内存部分的更多细节,请参考[相关文档](mem_detail.html)。 -Note If you have configured the task heap and managed memory explicitly, it is recommended to set neither -*total process memory* nor *total Flink memory*. Otherwise, it may easily lead to memory configuration conflicts. +提示 如果已经明确设置了任务堆内存和托管内存,建议不要再设置*进程总内存*或 *Flink 总内存*,否则可能会造成内存配置冲突。 -### Task (Operator) Heap Memory +### 任务(算子)堆内存 -If you want to guarantee that a certain amount of JVM heap is available for your user code, you can set the *task heap memory* -explicitly ([`taskmanager.memory.task.heap.size`](../config.html#taskmanager-memory-task-heap-size)). -It will be added to the JVM heap size and will be dedicated to Flink’s operators running the user code. +如果希望确保指定大小的 JVM 堆内存给用户代码使用,可以明确指定*任务堆内存*([`taskmanager.memory.task.heap.size`](../config.html#taskmanager-memory-task-heap-size))。 +指定的内存将被包含在总的 JVM 堆空间中,专门用于 Flink 算子及用户代码的执行。 -### Managed Memory +### 托管内存 -*Managed memory* is managed by Flink and is allocated as native memory (off-heap). The following workloads use *managed memory*: -* Streaming jobs can use it for [RocksDB state backend](../state/state_backends.html#the-rocksdbstatebackend). -* [Batch jobs](../../dev/batch) can use it for sorting, hash tables, caching of intermediate results. +*托管内存*是由 Flink 负责分配和管理的本地(堆外)内存。 +以下场景需要使用*托管内存*: +* 流处理作业中用于 [RocksDB State Backend](../state/state_backends.html#the-rocksdbstatebackend)。 +* [批处理作业](../../dev/batch)中用于排序、哈希表及缓存中间结果。 -The size of *managed memory* can be -* either configured explicitly via [`taskmanager.memory.managed.size`](../config.html#taskmanager-memory-managed-size) -* or computed as a fraction of *total Flink memory* via [`taskmanager.memory.managed.fraction`](../config.html#taskmanager-memory-managed-fraction). +可以通过以下两种范式指定*托管内存*的大小: +* 通过 [`taskmanager.memory.managed.size`](../config.html#taskmanager-memory-managed-size) 明确指定其大小。 +* 通过 [`taskmanager.memory.managed.fraction`](../config.html#taskmanager-memory-managed-fraction) 指定在*Flink 总内存*中的占比。 -*Size* will override *fraction*, if both are set. -If neither *size* nor *fraction* is explicitly configured, the [default fraction](../config.html#taskmanager-memory-managed-fraction) will be used. +当同时指定二者时,会优先采用指定的大小(Size)。 +若二者均未指定,会根据[默认占比](../config.html#taskmanager-memory-managed-fraction)进行计算。 -See also [how to configure memory for state backends](mem_tuning.html#configure-memory-for-state-backends) and [batch jobs](mem_tuning.html#configure-memory-for-batch-jobs). +请同时参考[如何配置 State Backend 内存](mem_tuning.html#state-backend-的内存配置)以及[如何配置批处理作业内存](mem_tuning.html#批处理作业的内存配置)。 -## Configure Off-Heap Memory (direct or native) +## 配置堆外内存(直接内存或本地内存) -The off-heap memory which is allocated by user code should be accounted for in *task off-heap memory* -([`taskmanager.memory.task.off-heap.size`](../config.html#taskmanager-memory-task-off-heap-size)). +用户代码中分配的堆外内存被归为*任务堆外内存(Task Off-Heap Memory),可以通过 [`taskmanager.memory.task.off-heap.size`](../config.html#taskmanager-memory-task-off-heap-size) 指定。 -Note You can also adjust the [framework off-heap memory](mem_detail.html#framework-memory). This option is advanced -and only recommended to be changed if you are sure that the Flink framework needs more memory. +提示 你也可以调整[框架推外内存(Framework Off-Heap Memory)](mem_detail.html#框架内存)。 +这是一个进阶配置,建议仅在确定 Flink 框架需要更多的内存时调整该配置。 -Flink includes the *framework off-heap memory* and *task off-heap memory* into the *direct memory* limit of the JVM, -see also [JVM parameters](mem_detail.html#jvm-parameters). +Flink 将*框架堆外内存*和*任务堆外内存*都计算在 JVM 的*直接内存*限制中,请参考 [JVM 参数](mem_detail.html#jvm-参数)。 -Note Although, native non-direct memory usage can be accounted for as a part of the -*framework off-heap memory* or *task off-heap memory*, it will result in a higher JVM's *direct memory* limit in this case. +提示 本地内存(非直接内存)也可以被归在*框架堆外内存*或*任务推外内存*中,在这种情况下 JVM 的*直接内存*限制可能会高于实际需求。 -Note The *network memory* is also part of JVM *direct memory* but it is managed by Flink and guaranteed -to never exceed its configured size. Therefore, resizing the *network memory* will not help in this situation. +提示 *网络内存(Network Memory)*同样被计算在 JVM *直接内存*中。 +Flink 会负责管理网络内存,保证其实际用量不会超过配置大小。 +因此,调整*网络内存*的大小不会对其他堆外内存有实质上的影响。 -See also [the detailed memory model](mem_detail.html#detailed-memory-model). +请参考[内存模型详解](mem_detail.html)。 diff --git a/docs/ops/memory/mem_trouble.md b/docs/ops/memory/mem_trouble.md index cd6463bc1b88d073699e8fb38363d34b5e84d84f..f3e6b137d7c24f24e89bf7a5c2bc4bd5ee3ccaf2 100644 --- a/docs/ops/memory/mem_trouble.md +++ b/docs/ops/memory/mem_trouble.md @@ -29,7 +29,7 @@ under the License. If you see an *IllegalConfigurationException* thrown from *TaskExecutorProcessUtils*, it usually indicates that there is either an invalid configuration value (e.g. negative memory size, fraction that is greater than 1, etc.) -or configuration conflicts. Check the documentation chapters related to the [memory components](mem_setup.html#detailed-memory-model) +or configuration conflicts. Check the documentation chapters related to the [memory components](mem_detail.html) mentioned in the exception message. ## OutOfMemoryError: Java heap space @@ -44,7 +44,7 @@ is advanced and should only be changed if you are sure that the Flink framework The exception usually indicates that the JVM *direct memory* limit is too small or that there is a *direct memory leak*. Check whether user code or other external dependencies use the JVM *direct memory* and that it is properly accounted for. -You can try to increase its limit by adjusting [direct off-heap memory](mem_setup.html#detailed-memory-model). +You can try to increase its limit by adjusting [direct off-heap memory](mem_detail.html). See also [how to configure off-heap memory](mem_setup.html#configure-off-heap-memory-direct-or-native) and the [JVM arguments](mem_detail.html#jvm-parameters) which Flink sets. @@ -55,7 +55,7 @@ You can try to increase the [JVM metaspace option](../config.html#taskmanager-me ## IOException: Insufficient number of network buffers -The exception usually indicates that the size of the configured [network memory](mem_setup.html#detailed-memory-model) +The exception usually indicates that the size of the configured [network memory](mem_detail.html) is not big enough. You can try to increase the *network memory* by adjusting the following options: * [`taskmanager.memory.network.min`](../config.html#taskmanager-memory-network-min) * [`taskmanager.memory.network.max`](../config.html#taskmanager-memory-network-max) @@ -70,5 +70,5 @@ monitoring system or from the error messages when a container gets killed by the If [RocksDBStateBackend](../state/state_backends.html#the-rocksdbstatebackend) is used and the memory controlling is disabled, you can try to increase the [managed memory](mem_setup.html#managed-memory). -Alternatively, you can increase the [JVM overhead](mem_setup.html#detailed-memory-model). +Alternatively, you can increase the [JVM overhead](mem_detail.html). See also [how to configure memory for containers](mem_tuning.html#configure-memory-for-containers). diff --git a/docs/ops/memory/mem_trouble.zh.md b/docs/ops/memory/mem_trouble.zh.md index cd6463bc1b88d073699e8fb38363d34b5e84d84f..2e3377c041da076924d472cbdc85d0f9a89c6f1d 100644 --- a/docs/ops/memory/mem_trouble.zh.md +++ b/docs/ops/memory/mem_trouble.zh.md @@ -1,5 +1,5 @@ --- -title: "Troubleshooting" +title: "常见问题" nav-parent_id: ops_mem nav-pos: 4 --- @@ -27,48 +27,43 @@ under the License. ## IllegalConfigurationException -If you see an *IllegalConfigurationException* thrown from *TaskExecutorProcessUtils*, it usually indicates -that there is either an invalid configuration value (e.g. negative memory size, fraction that is greater than 1, etc.) -or configuration conflicts. Check the documentation chapters related to the [memory components](mem_setup.html#detailed-memory-model) -mentioned in the exception message. +如果遇到从 *TaskExecutorProcessUtils* 抛出的 *IllegalConfigurationException* 异常,这通常说明您的配置参数中存在无效值(例如内存大小为负数、占比大于 1 等)或者配置冲突。 +请根据异常信息,确认[内存模型详解](mem_detail.html)中与出错的内存部分对应章节的内容。 ## OutOfMemoryError: Java heap space -The exception usually indicates that the JVM heap is too small. You can try to increase the JVM heap size -by increasing [total memory](mem_setup.html#configure-total-memory) or [task heap memory](mem_setup.html#task-operator-heap-memory). +该异常说明 JVM 的堆空间过小。 +可以通过增大[总内存](mem_setup.html#配置总内存)或[任务堆内存](mem_setup.html#任务算子堆内存)的方法来增大 JVM 堆空间。 -Note You can also increase the [framework heap memory](mem_detail.html#framework-memory) but this option -is advanced and should only be changed if you are sure that the Flink framework itself needs more memory. +提示 也可以增大[框架堆内存](mem_detail.html#框架内存)。这是一个进阶配置,只有在确认是 Flink 框架自身需要更多内存时才应该去调整。 ## OutOfMemoryError: Direct buffer memory -The exception usually indicates that the JVM *direct memory* limit is too small or that there is a *direct memory leak*. -Check whether user code or other external dependencies use the JVM *direct memory* and that it is properly accounted for. -You can try to increase its limit by adjusting [direct off-heap memory](mem_setup.html#detailed-memory-model). -See also [how to configure off-heap memory](mem_setup.html#configure-off-heap-memory-direct-or-native) and -the [JVM arguments](mem_detail.html#jvm-parameters) which Flink sets. +该异常通常说明 JVM 的*直接内存*限制过小,或者存在*直接内存泄漏(Direct Memory Leak)*。 +请确认用户代码及外部依赖中是否使用了 JVM *直接内存*,以及如果使用了直接内存,是否配置了足够的内存空间。 +可以通过调整[堆外内存](mem_detail.html)来增大直接内存限制。 +请同时参考[如何配置堆外内存](mem_setup.html#配置堆外内存直接内存或本地内存))以及 Flink 设置的 [JVM 参数](mem_detail.html#jvm-参数)。 ## OutOfMemoryError: Metaspace -The exception usually indicates that [JVM metaspace limit](mem_detail.html#jvm-parameters) is configured too small. -You can try to increase the [JVM metaspace option](../config.html#taskmanager-memory-jvm-metaspace-size). +该异常说明 [JVM Metaspace 限制](mem_detail.html#jvm-参数)过小。 +可以尝试调整 [JVM Metaspace 参数](../config.html#taskmanager-memory-jvm-metaspace-size)。 ## IOException: Insufficient number of network buffers -The exception usually indicates that the size of the configured [network memory](mem_setup.html#detailed-memory-model) -is not big enough. You can try to increase the *network memory* by adjusting the following options: +该异常通常说明[网络内存](mem_detail.html)过小。 +可以通过调整以下配置参数增大*网络内存*: * [`taskmanager.memory.network.min`](../config.html#taskmanager-memory-network-min) * [`taskmanager.memory.network.max`](../config.html#taskmanager-memory-network-max) * [`taskmanager.memory.network.fraction`](../config.html#taskmanager-memory-network-fraction) -## Container Memory Exceeded +## 容器(Container)内存超用 -If a task executor container tries to allocate memory beyond its requested size (Yarn, Mesos or Kubernetes), -this usually indicates that Flink has not reserved enough native memory. You can observe this either by using an external -monitoring system or from the error messages when a container gets killed by the deployment environment. +如果 TaskExecutor 容器尝试分配超过其申请大小的内存(Yarn、Mesos 或 Kubernetes),这通常说明 Flink 没有预留出足够的本地内存。 +可以通过外部监控系统或者容器被部署环境杀掉时的错误信息判断是否存在容器内存超用。 -If [RocksDBStateBackend](../state/state_backends.html#the-rocksdbstatebackend) is used and the memory controlling is disabled, -you can try to increase the [managed memory](mem_setup.html#managed-memory). +如果使用了 [RocksDBStateBackend](../state/state_backends.html#rocksdbstatebackend) 且没有开启内存控制,也可以尝试增大[托管内存](mem_setup.html#托管内存)。 -Alternatively, you can increase the [JVM overhead](mem_setup.html#detailed-memory-model). -See also [how to configure memory for containers](mem_tuning.html#configure-memory-for-containers). +此外,还可以尝试增大 [JVM 开销](mem_detail.html)。 + +请参考[如何配置容器内存](mem_tuning.html#容器container的内存配置)。 diff --git a/docs/ops/memory/mem_tuning.md b/docs/ops/memory/mem_tuning.md index 9d83d00d8dc772ba1ac2c3fb1a4b2d3dadf6fa7f..de5451341154a9dfbe700b6b9278e93f7c8a8c77 100644 --- a/docs/ops/memory/mem_tuning.md +++ b/docs/ops/memory/mem_tuning.md @@ -31,7 +31,7 @@ depending on the use case and which options are important in which case. ## Configure memory for standalone deployment It is recommended to configure [total Flink memory](mem_setup.html#configure-total-memory) -([`taskmanager.memory.flink.size`](../config.html#taskmanager-memory-flink-size)) or its [components](mem_setup.html#detailed-memory-model) +([`taskmanager.memory.flink.size`](../config.html#taskmanager-memory-flink-size)) or its [components](mem_detail.html) for [standalone deployment](../deployment/cluster_setup.html) where you want to declare how much memory is given to Flink itself. Additionally, you can adjust *JVM metaspace* if it causes [problems](mem_trouble.html#outofmemoryerror-metaspace). @@ -47,7 +47,7 @@ It declares how much memory in total should be assigned to the Flink *JVM proces Note If you configure the *total Flink memory* Flink will implicitly add JVM memory components to derive the *total process memory* and request a container with the memory of that derived size, -see also [detailed Memory Model](mem_setup.html#detailed-memory-model). +see also [detailed Memory Model](mem_detail.html).
Warning: If Flink or user code allocates unmanaged off-heap (native) memory beyond the container size @@ -63,7 +63,7 @@ will dictate the optimal memory configurations of your cluster. ### Heap state backend When running a stateless job or using a heap state backend ([MemoryStateBackend](../state/state_backends.html#the-memorystatebackend) -or [FsStateBackend](../state/state_backends.html#the-fsstatebackend), set [managed memory](mem_setup.html#managed-memory) to zero. +or [FsStateBackend](../state/state_backends.html#the-fsstatebackend)), set [managed memory](mem_setup.html#managed-memory) to zero. This will ensure that the maximum amount of memory is allocated for user code on the JVM. ### RocksDB state backend diff --git a/docs/ops/memory/mem_tuning.zh.md b/docs/ops/memory/mem_tuning.zh.md index 9d83d00d8dc772ba1ac2c3fb1a4b2d3dadf6fa7f..f1e3a8c23af57c661212cfbfc49d3813610f0a49 100644 --- a/docs/ops/memory/mem_tuning.zh.md +++ b/docs/ops/memory/mem_tuning.zh.md @@ -1,5 +1,5 @@ --- -title: "Memory tuning guide" +title: "调优指南" nav-parent_id: ops_mem nav-pos: 3 --- @@ -22,66 +22,55 @@ specific language governing permissions and limitations under the License. --> -In addition to the [main memory setup guide](mem_setup.html), this section explains how to setup memory of task executors -depending on the use case and which options are important in which case. +本文在的基本的[配置指南](mem_setup.html)的基础上,介绍如何根据具体的使用场景调整 TaskExecutor 的内存配置,以及在不同使用场景下分别需要重点关注哪些配置参数。 * toc {:toc} -## Configure memory for standalone deployment +## 独立部署模式(Standalone Deployment)下的内存配置 -It is recommended to configure [total Flink memory](mem_setup.html#configure-total-memory) -([`taskmanager.memory.flink.size`](../config.html#taskmanager-memory-flink-size)) or its [components](mem_setup.html#detailed-memory-model) -for [standalone deployment](../deployment/cluster_setup.html) where you want to declare how much memory is given to Flink itself. -Additionally, you can adjust *JVM metaspace* if it causes [problems](mem_trouble.html#outofmemoryerror-metaspace). +[独立部署模式](../deployment/cluster_setup.html),我们通常更关注 Flink 应用本身使用的内存大小。 +建议配置 [Flink 总内存](mem_setup.html#配置总内存)([`taskmanager.memory.flink.size`](../config.html#taskmanager-memory-flink-size))或者它的[组成部分](mem_detail.html)。 +此外,如果出现 [Metaspace 不足的问题](mem_trouble.html#outofmemoryerror-metaspace),可以调整 *JVM Metaspace* 的大小。 -The *total Process memory* is not relevant because *JVM overhead* is not controlled by Flink or deployment environment, -only physical resources of the executing machine matter in this case. +这种情况下通常无需配置*进程总内存*,因为不管是 Flink 还是部署环境都不会对 *JVM 开销* 进行限制,它只与机器的物理资源相关。 -## Configure memory for containers +## 容器(Container)的内存配置 -It is recommended to configure [total process memory](mem_setup.html#configure-total-memory) -([`taskmanager.memory.process.size`](../config.html#taskmanager-memory-process-size)) for the containerized deployments -([Kubernetes](../deployment/kubernetes.html), [Yarn](../deployment/yarn_setup.html) or [Mesos](../deployment/mesos.html)). -It declares how much memory in total should be assigned to the Flink *JVM process* and corresponds to the size of the requested container. +在容器化部署模式(Containerized Deployment)下([Kubernetes](../deployment/kubernetes.html)、[Yarn](../deployment/yarn_setup.html) 或 [Mesos](../deployment/mesos.html)),建议配置[进程总内存](mem_setup.html#配置总内存)([`taskmanager.memory.process.size`](../config.html#taskmanager-memory-process-size))。 +该配置参数用于指定分配给 Flink *JVM 进程*的总内存,也就是需要申请的容器大小。 -Note If you configure the *total Flink memory* Flink will implicitly add JVM memory components -to derive the *total process memory* and request a container with the memory of that derived size, -see also [detailed Memory Model](mem_setup.html#detailed-memory-model). +提示 如果配置了 *Flink 总内存*,Flink 会自动加上 JVM 相关的内存部分,根据推算出的*进程总内存*大小申请容器。 +请参考[内存模型详解](mem_detail.html)。
- Warning: If Flink or user code allocates unmanaged off-heap (native) memory beyond the container size - the job can fail because the deployment environment can kill the offending containers. + 注意: 如果 Flink 或者用户代码分配超过容器大小的非托管的堆外(本地)内存,部署环境可能会杀掉超用内存的容器,造成作业执行失败。
-See also description of [container memory exceeded](mem_trouble.html#container-memory-exceeded) failure. +请参考[容器内存超用](mem_trouble.html#容器container内存超用)中的相关描述。 -## Configure memory for state backends +## State Backend 的内存配置 -When deploying a Flink streaming application, the type of [state backend](../state/state_backends.html) used -will dictate the optimal memory configurations of your cluster. +在部署 Flink 流处理应用时,可以根据 [State Backend](../state/state_backends.html) 的类型对集群的配置进行优化。 -### Heap state backend +### Heap State Backend -When running a stateless job or using a heap state backend ([MemoryStateBackend](../state/state_backends.html#the-memorystatebackend) -or [FsStateBackend](../state/state_backends.html#the-fsstatebackend), set [managed memory](mem_setup.html#managed-memory) to zero. -This will ensure that the maximum amount of memory is allocated for user code on the JVM. +执行无状态作业或者使用 Heap State Backend([MemoryStateBackend](../state/state_backends.html#memorystatebackend) +或 [FsStateBackend](../state/state_backends.html#fsstatebackend))时,建议将[托管内存](mem_setup.html#托管内存)设置为 0。 +这样能够最大化分配给 JVM 上用户代码的内存。 -### RocksDB state backend +### RocksDB State Backend -The [RocksDBStateBackend](../state/state_backends.html#the-rocksdbstatebackend) uses native memory. By default, -RocksDB is setup to limit native memory allocation to the size of the [managed memory](mem_setup.html#managed-memory). -Therefore, it is important to reserve enough *managed memory* for your state use case. If you disable the default RocksDB memory control, -task executors can be killed in containerized deployments if RocksDB allocates memory above the limit of the requested container size -(the [total process memory](mem_setup.html#configure-total-memory)). -See also [how to tune RocksDB memory](../state/large_state_tuning.html#tuning-rocksdb-memory) -and [state.backend.rocksdb.memory.managed](../config.html#state-backend-rocksdb-memory-managed). +[RocksDBStateBackend](../state/state_backends.html#rocksdbstatebackend) 使用本地内存。 +默认情况下,RocksDB 会限制其内存用量不超过用户配置的[*托管内存*](mem_setup.html#托管内存)。 +因此,使用这种方式存储状态时,配置足够多的*托管内存*是十分重要的。 +如果你关闭了 RocksDB 的内存控制,那么在容器化部署模式下如果 RocksDB 分配的内存超出了申请容器的大小([进程总内存](mem_setup.html#配置总内存)),可能会造成 TaskExecutor 被部署环境杀掉。 +请同时参考[如何调整 RocksDB 内存](../state/large_state_tuning.html#tuning-rocksdb-memory)以及 [state.backend.rocksdb.memory.managed](../config.html#state-backend-rocksdb-memory-managed)。 -## Configure memory for batch jobs +## 批处理作业的内存配置 -Flink's batch operators leverage [managed memory](../memory/mem_setup.html#managed-memory) to run more efficiently. -In doing so, some operations can be performed directly on raw data without having to be deserialized into Java objects. -This means that [managed memory](../memory/mem_setup.html#managed-memory) configurations have practical effects -on the performance of your applications. Flink will attempt to allocate and use as much [managed memory](../memory/mem_setup.html#managed-memory) -as configured for batch jobs but not go beyond its limits. This prevents `OutOfMemoryError`'s because Flink knows precisely -how much memory it has to leverage. If the [managed memory](../memory/mem_setup.html#managed-memory) is not sufficient, -Flink will gracefully spill to disk. +Flink 批处理算子使用[托管内存](../memory/mem_setup.html#托管内存)来提高处理效率。 +算子运行时,部分操作可以直接在原始数据上进行,而无需将数据反序列化成 Java 对象。 +这意味着[托管内存](../memory/mem_setup.html#托管内存)对应用的性能具有实质上的影响。 +因此 Flink 会在不超过其配置限额的前提下,尽可能分配更多的[托管内存](../memory/mem_setup.html#托管内存)。 +Flink 明确知道可以使用的内存大小,因此可以有效避免 `OutOfMemoryError` 的发生。 +当[托管内存](../memory/mem_setup.html#托管内存)不足时,Flink 会优雅的将数据落盘。 diff --git a/docs/ops/plugins.md b/docs/ops/plugins.md index 759d5dcf35fcce9c32c21f803d14b147440bc873..ea203ff418ea1e3dd465e9ac36214f9ddaa3ad08 100644 --- a/docs/ops/plugins.md +++ b/docs/ops/plugins.md @@ -70,7 +70,7 @@ possible across Flink core, plugins, and user code. ## File Systems -All [file systems](filesystems/index) **except MapR** are pluggable. That means they can and should +All [file systems](filesystems) **except MapR** are pluggable. That means they can and should be used as plugins. To use a pluggable file system, copy the corresponding JAR file from the `opt` directory to a directory under `plugins` directory of your Flink distribution before starting Flink, e.g. diff --git a/docs/ops/plugins.zh.md b/docs/ops/plugins.zh.md index 759d5dcf35fcce9c32c21f803d14b147440bc873..ea203ff418ea1e3dd465e9ac36214f9ddaa3ad08 100644 --- a/docs/ops/plugins.zh.md +++ b/docs/ops/plugins.zh.md @@ -70,7 +70,7 @@ possible across Flink core, plugins, and user code. ## File Systems -All [file systems](filesystems/index) **except MapR** are pluggable. That means they can and should +All [file systems](filesystems) **except MapR** are pluggable. That means they can and should be used as plugins. To use a pluggable file system, copy the corresponding JAR file from the `opt` directory to a directory under `plugins` directory of your Flink distribution before starting Flink, e.g. diff --git a/docs/ops/python_shell.md b/docs/ops/python_shell.md index a2faea246d26fa18f7aef0342086bb4a72f6ac16..ebcb7291df2056a975fcb7432d3aab2d97aa8925 100644 --- a/docs/ops/python_shell.md +++ b/docs/ops/python_shell.md @@ -27,27 +27,18 @@ It can be used in a local setup as well as in a cluster setup. See the [local setup page](deployment/local.html) for more information about how to setup a local Flink. You can also [build a local setup from source](../flinkDev/building.html). -Note The Python Shell will run the command “python”. Please run the following command to confirm that the command “python” in current environment points to Python 3.5+: +Note The Python Shell will run the command “python”. Note The Python Shell will run the command “python”. Please refer to the Python Table API [installation guide]({{ site.baseurl }}/dev/table/python/installation.html) on how to set up the Python execution environments. -{% highlight bash %} -$ python --version -# the version printed here must be 3.5+ -{% endhighlight %} - -Note Using Python UDF in Python Shell requires apache-beam 2.19.0. Run the following command to confirm that it meets the requirements before run the Shell in local mode: - -{% highlight bash %} -$ python -m pip install apache-beam==2.19.0 -{% endhighlight %} - -To use the shell with an integrated Flink cluster just execute: +To use the shell with an integrated Flink cluster, you can simply install PyFlink with PyPi and execute the shell directly: {% highlight bash %} -bin/pyflink-shell.sh local +# install PyFlink +$ python -m pip install apache-flink +# execute the shell +$ pyflink-shell.sh local {% endhighlight %} -in the root directory of your binary Flink directory. To run the Shell on a -cluster, please see the Setup section below. +To run the shell on a cluster, please see the Setup section below. ## Usage @@ -130,7 +121,7 @@ The example below is a simple program in the Python shell: To get an overview of what options the Python Shell provides, please use {% highlight bash %} -bin/pyflink-shell.sh --help +pyflink-shell.sh --help {% endhighlight %} ### Local @@ -138,7 +129,7 @@ bin/pyflink-shell.sh --help To use the shell with an integrated Flink cluster just execute: {% highlight bash %} -bin/pyflink-shell.sh local +pyflink-shell.sh local {% endhighlight %} @@ -148,7 +139,7 @@ To use it with a running cluster, please start the Python shell with the keyword and supply the host and port of the JobManager with: {% highlight bash %} -bin/pyflink-shell.sh remote +pyflink-shell.sh remote {% endhighlight %} ### Yarn Python Shell cluster @@ -163,7 +154,7 @@ For example, to start a Yarn cluster for the Python Shell with two TaskManagers use the following: {% highlight bash %} -bin/pyflink-shell.sh yarn -n 2 +pyflink-shell.sh yarn -n 2 {% endhighlight %} For all other options, see the full reference at the bottom. @@ -175,7 +166,7 @@ If you have previously deployed a Flink cluster using the Flink Yarn Session, the Python shell can connect with it using the following command: {% highlight bash %} -bin/pyflink-shell.sh yarn +pyflink-shell.sh yarn {% endhighlight %} diff --git a/docs/ops/python_shell.zh.md b/docs/ops/python_shell.zh.md index 09dbccc51498af99a695d75a14e0212d4c2aebc4..e5f2a6c6093c2522ab0126643e956b30a5277760 100644 --- a/docs/ops/python_shell.zh.md +++ b/docs/ops/python_shell.zh.md @@ -27,23 +27,15 @@ Flink附带了一个集成的交互式Python Shell。 本地安装Flink,请看[本地安装](deployment/local.html)页面。 您也可以从源码安装Flink,请看[从源码构建 Flink](../flinkDev/building.html)页面。 -注意 Python Shell会调用“python”命令。请执行以下命令以确认当前环境下的指令“python”指向Python 3.5及以上版本: +注意 Python Shell会调用“python”命令。关于Python执行环境的要求,请参考Python Table API[环境安装]({{ site.baseurl }}/dev/dev/table/python/installation.html)。 -{% highlight bash %} -$ python --version -# the version printed here must be 3.5+ -{% endhighlight %} - -注意 在Python Shell中使用Python UDF依赖apache-beam 2.19.0。 在以本地模式执行之前,执行以下命令以确认环境满足需求: - -{% highlight bash %} -$ python -m pip install apache-beam==2.19.0 -{% endhighlight %} - -为了使用Flink的Python Shell,你只需要在Flink的binary目录下执行: +你可以通过PyPi安装PyFlink,然后使用Python Shell: {% highlight bash %} -bin/pyflink-shell.sh local +# 安装 PyFlink +$ python -m pip install apache-flink +# 执行脚本 +$ pyflink-shell.sh local {% endhighlight %} 关于如何在一个Cluster集群上运行Python shell,可以参考启动章节介绍。 @@ -129,7 +121,7 @@ bin/pyflink-shell.sh local 查看Python Shell提供的可选参数,可以使用: {% highlight bash %} -bin/pyflink-shell.sh --help +pyflink-shell.sh --help {% endhighlight %} ### Local @@ -137,7 +129,7 @@ bin/pyflink-shell.sh --help Python Shell运行在local模式下,只需要执行: {% highlight bash %} -bin/pyflink-shell.sh local +pyflink-shell.sh local {% endhighlight %} @@ -147,7 +139,7 @@ Python Shell运行在一个指定的JobManager上,通过关键字`remote`和 的地址和端口号来进行指定: {% highlight bash %} -bin/pyflink-shell.sh remote +pyflink-shell.sh remote {% endhighlight %} ### Yarn Python Shell cluster @@ -157,7 +149,7 @@ Python Shell可以运行在YARN集群之上。Python shell在Yarn上部署一个 例如,在一个部署了两个TaskManager的Yarn集群上运行Python Shell: {% highlight bash %} -bin/pyflink-shell.sh yarn -n 2 +pyflink-shell.sh yarn -n 2 {% endhighlight %} 关于所有可选的参数,可以查看本页面底部的完整说明。 @@ -168,7 +160,7 @@ bin/pyflink-shell.sh yarn -n 2 如果你已经通过Flink Yarn Session部署了一个Flink集群,能够通过以下的命令连接到这个集群: {% highlight bash %} -bin/pyflink-shell.sh yarn +pyflink-shell.sh yarn {% endhighlight %} diff --git a/docs/redirects/basic_api_concepts.md b/docs/redirects/basic_api_concepts.md index df04ce182927a87864ed148fd9a28b3c6c68dc60..e51661e376f67a9313dbcc3350adaea8193073ed 100644 --- a/docs/redirects/basic_api_concepts.md +++ b/docs/redirects/basic_api_concepts.md @@ -1,7 +1,7 @@ --- title: "Basic API Concepts" layout: redirect -redirect: /dev/api_concepts.html +redirect: /dev/datastream_api.html permalink: /apis/common/index.html --- + +The focus of this tutorial is to broadly cover the DataStream API well enough that you will be +able to get started writing streaming applications. + +* This will be replaced by the TOC +{:toc} + +## What can be Streamed? + +Flink's DataStream APIs for Java and Scala will let you stream anything they can serialize. Flink's +own serializer is used for + +- basic types, i.e., String, Long, Integer, Boolean, Array +- composite types: Tuples, POJOs, and Scala case classes + +and Flink falls back to Kryo for other types. It is also possible to use other serializers with +Flink. Avro, in particular, is well supported. + +### Java tuples and POJOs + +Flink's native serializer can operate efficiently on tuples and POJOs. + +#### Tuples + +For Java, Flink defines its own `Tuple0` thru `Tuple25` types. + +{% highlight java %} +Tuple2 person = Tuple2.of("Fred", 35); + +// zero based index! +String name = person.f0; +Integer age = person.f1; +{% endhighlight %} + +#### POJOs + +Flink recognizes a data type as a POJO type (and allows “by-name” field referencing) if the following conditions are fulfilled: + +- The class is public and standalone (no non-static inner class) +- The class has a public no-argument constructor +- All non-static, non-transient fields in the class (and all superclasses) are either public (and + non-final) or have public getter- and setter- methods that follow the Java beans naming + conventions for getters and setters. + +Example: + +{% highlight java %} +public class Person { + public String name; + public Integer age; + public Person() {}; + public Person(String name, Integer age) { + . . . + }; +} + +Person person = new Person("Fred Flintstone", 35); +{% endhighlight %} + +Flink's serializer [supports schema evolution for POJO types]({% link dev/stream/state/schema_evolution.md %}#pojo-types). + +### Scala tuples and case classes + +These work just as you'd expect. + +{% top %} + +## A Complete Example + +This example takes a stream of records about people as input, and filters it to only include the adults. + +{% highlight java %} +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.api.common.functions.FilterFunction; + +public class Example { + + public static void main(String[] args) throws Exception { + final StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream flintstones = env.fromElements( + new Person("Fred", 35), + new Person("Wilma", 35), + new Person("Pebbles", 2)); + + DataStream adults = flintstones.filter(new FilterFunction() { + @Override + public boolean filter(Person person) throws Exception { + return person.age >= 18; + } + }); + + adults.print(); + + env.execute(); + } + + public static class Person { + public String name; + public Integer age; + public Person() {}; + + public Person(String name, Integer age) { + this.name = name; + this.age = age; + }; + + public String toString() { + return this.name.toString() + ": age " + this.age.toString(); + }; + } +} +{% endhighlight %} + +### Stream execution environment + +Every Flink application needs an execution environment, `env` in this example. Streaming +applications need to use a `StreamExecutionEnvironment`. + +The DataStream API calls made in your application build a job graph that is attached to the +`StreamExecutionEnvironment`. When `env.execute()` is called this graph is packaged up and sent to +the Flink Master, which parallelizes the job and distributes slices of it to the Task Managers for +execution. Each parallel slice of your job will be executed in a *task slot*. + +Note that if you don't call execute(), your application won't be run. + +Flink runtime: client, job manager, task managers + +This distributed runtime depends on your application being serializable. It also requires that all +dependencies are available to each node in the cluster. + +### Basic stream sources + +The example above constructs a `DataStream` using `env.fromElements(...)`. This is a +convenient way to throw together a simple stream for use in a prototype or test. There is also a +`fromCollection(Collection)` method on `StreamExecutionEnvironment`. So instead, you could do this: + +{% highlight java %} +List people = new ArrayList(); + +people.add(new Person("Fred", 35)); +people.add(new Person("Wilma", 35)); +people.add(new Person("Pebbles", 2)); + +DataStream flintstones = env.fromCollection(people); +{% endhighlight %} + +Another convenient way to get some data into a stream while prototyping is to use a socket + +{% highlight java %} +DataStream lines = env.socketTextStream("localhost", 9999) +{% endhighlight %} + +or a file + +{% highlight java %} +DataStream lines = env.readTextFile("file:///path"); +{% endhighlight %} + +In real applications the most commonly used data sources are those that support low-latency, high +throughput parallel reads in combination with rewind and replay -- the prerequisites for high +performance and fault tolerance -- such as Apache Kafka, Kinesis, and various filesystems. REST APIs +and databases are also frequently used for stream enrichment. + +### Basic stream sinks + +The example above uses `adults.print()` to print its results to the task manager logs (which will +appear in your IDE's console, when running in an IDE). This will call `toString()` on each element +of the stream. + +The output looks something like this + + 1> Fred: age 35 + 2> Wilma: age 35 + +where 1> and 2> indicate which sub-task (i.e., thread) produced the output. + +In production, commonly used sinks include the StreamingFileSink, various databases, +and several pub-sub systems. + +### Debugging + +In production, your application will run in a remote cluster or set of containers. And if it fails, +it will fail remotely. The Flink Master and Task Manager logs can be very helpful in debugging such +failures, but it is much easier to do local debugging inside an IDE, which is something that Flink +supports. You can set breakpoints, examine local variables, and step through your code. You can also +step into Flink's code, which can be a great way to learn more about its internals if you are +curious to see how Flink works. + +{% top %} + +## Hands-on + +At this point you know enough to get started coding and running a simple DataStream application. +Clone the [flink-training repo](https://github.com/apache/flink-training/tree/{% if site.is_stable %}release-{{ site.version_title }}{% else %}master{% endif %}), and after following the +instructions in the README, do the first exercise: +[Filtering a Stream (Ride Cleansing)](https://github.com/apache/flink-training/tree/{% if site.is_stable %}release-{{ site.version_title }}{% else %}master{% endif %}/ride-cleansing). + +{% top %} + +## Further Reading + +- [Flink Serialization Tuning Vol. 1: Choosing your Serializer — if you can](https://flink.apache.org/news/2020/04/15/flink-serialization-tuning-vol-1.html) +- [Anatomy of a Flink Program]({% link dev/datastream_api.md %}#anatomy-of-a-flink-program) +- [Data Sources]({% link dev/datastream_api.md %}#data-sources) +- [Data Sinks]({% link dev/datastream_api.md %}#data-sinks) +- [DataStream Connectors]({% link dev/connectors/index.md %}) + +{% top %} diff --git a/docs/tutorials/datastream_api.zh.md b/docs/tutorials/datastream_api.zh.md new file mode 100644 index 0000000000000000000000000000000000000000..e07a5a9c237ae6771673ca6374212cf72162e593 --- /dev/null +++ b/docs/tutorials/datastream_api.zh.md @@ -0,0 +1,237 @@ +--- +title: Intro to the DataStream API +nav-id: datastream-api +nav-pos: 2 +nav-title: Intro to the DataStream API +nav-parent_id: tutorials +permalink: /tutorials/datastream_api.html +--- + + +The focus of this tutorial is to broadly cover the DataStream API well enough that you will be +able to get started writing streaming applications. + +* This will be replaced by the TOC +{:toc} + +## What can be Streamed? + +Flink's DataStream APIs for Java and Scala will let you stream anything they can serialize. Flink's +own serializer is used for + +- basic types, i.e., String, Long, Integer, Boolean, Array +- composite types: Tuples, POJOs, and Scala case classes + +and Flink falls back to Kryo for other types. It is also possible to use other serializers with +Flink. Avro, in particular, is well supported. + +### Java tuples and POJOs + +Flink's native serializer can operate efficiently on tuples and POJOs. + +#### Tuples + +For Java, Flink defines its own `Tuple0` thru `Tuple25` types. + +{% highlight java %} +Tuple2 person = Tuple2.of("Fred", 35); + +// zero based index! +String name = person.f0; +Integer age = person.f1; +{% endhighlight %} + +#### POJOs + +Flink recognizes a data type as a POJO type (and allows “by-name” field referencing) if the following conditions are fulfilled: + +- The class is public and standalone (no non-static inner class) +- The class has a public no-argument constructor +- All non-static, non-transient fields in the class (and all superclasses) are either public (and + non-final) or have public getter- and setter- methods that follow the Java beans naming + conventions for getters and setters. + +Example: + +{% highlight java %} +public class Person { + public String name; + public Integer age; + public Person() {}; + public Person(String name, Integer age) { + . . . + }; +} + +Person person = new Person("Fred Flintstone", 35); +{% endhighlight %} + +Flink's serializer [supports schema evolution for POJO types]({% link dev/stream/state/schema_evolution.zh.md %}#pojo-types). + +### Scala tuples and case classes + +These work just as you'd expect. + +{% top %} + +## A Complete Example + +This example takes a stream of records about people as input, and filters it to only include the adults. + +{% highlight java %} +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.api.common.functions.FilterFunction; + +public class Example { + + public static void main(String[] args) throws Exception { + final StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream flintstones = env.fromElements( + new Person("Fred", 35), + new Person("Wilma", 35), + new Person("Pebbles", 2)); + + DataStream adults = flintstones.filter(new FilterFunction() { + @Override + public boolean filter(Person person) throws Exception { + return person.age >= 18; + } + }); + + adults.print(); + + env.execute(); + } + + public static class Person { + public String name; + public Integer age; + public Person() {}; + + public Person(String name, Integer age) { + this.name = name; + this.age = age; + }; + + public String toString() { + return this.name.toString() + ": age " + this.age.toString(); + }; + } +} +{% endhighlight %} + +### Stream execution environment + +Every Flink application needs an execution environment, `env` in this example. Streaming +applications need to use a `StreamExecutionEnvironment`. + +The DataStream API calls made in your application build a job graph that is attached to the +`StreamExecutionEnvironment`. When `env.execute()` is called this graph is packaged up and sent to +the Flink Master, which parallelizes the job and distributes slices of it to the Task Managers for +execution. Each parallel slice of your job will be executed in a *task slot*. + +Note that if you don't call execute(), your application won't be run. + +Flink runtime: client, job manager, task managers + +This distributed runtime depends on your application being serializable. It also requires that all +dependencies are available to each node in the cluster. + +### Basic stream sources + +The example above constructs a `DataStream` using `env.fromElements(...)`. This is a +convenient way to throw together a simple stream for use in a prototype or test. There is also a +`fromCollection(Collection)` method on `StreamExecutionEnvironment`. So instead, you could do this: + +{% highlight java %} +List people = new ArrayList(); + +people.add(new Person("Fred", 35)); +people.add(new Person("Wilma", 35)); +people.add(new Person("Pebbles", 2)); + +DataStream flintstones = env.fromCollection(people); +{% endhighlight %} + +Another convenient way to get some data into a stream while prototyping is to use a socket + +{% highlight java %} +DataStream lines = env.socketTextStream("localhost", 9999) +{% endhighlight %} + +or a file + +{% highlight java %} +DataStream lines = env.readTextFile("file:///path"); +{% endhighlight %} + +In real applications the most commonly used data sources are those that support low-latency, high +throughput parallel reads in combination with rewind and replay -- the prerequisites for high +performance and fault tolerance -- such as Apache Kafka, Kinesis, and various filesystems. REST APIs +and databases are also frequently used for stream enrichment. + +### Basic stream sinks + +The example above uses `adults.print()` to print its results to the task manager logs (which will +appear in your IDE's console, when running in an IDE). This will call `toString()` on each element +of the stream. + +The output looks something like this + + 1> Fred: age 35 + 2> Wilma: age 35 + +where 1> and 2> indicate which sub-task (i.e., thread) produced the output. + +In production, commonly used sinks include the StreamingFileSink, various databases, +and several pub-sub systems. + +### Debugging + +In production, your application will run in a remote cluster or set of containers. And if it fails, +it will fail remotely. The Flink Master and Task Manager logs can be very helpful in debugging such +failures, but it is much easier to do local debugging inside an IDE, which is something that Flink +supports. You can set breakpoints, examine local variables, and step through your code. You can also +step into Flink's code, which can be a great way to learn more about its internals if you are +curious to see how Flink works. + +{% top %} + +## Hands-on + +At this point you know enough to get started coding and running a simple DataStream application. +Clone the [flink-training repo](https://github.com/apache/flink-training/tree/{% if site.is_stable %}release-{{ site.version_title }}{% else %}master{% endif %}), and after following the +instructions in the README, do the first exercise: +[Filtering a Stream (Ride Cleansing)](https://github.com/apache/flink-training/tree/{% if site.is_stable %}release-{{ site.version_title }}{% else %}master{% endif %}/ride-cleansing). + +{% top %} + +## Further Reading + +- [Flink Serialization Tuning Vol. 1: Choosing your Serializer — if you can](https://flink.apache.org/news/2020/04/15/flink-serialization-tuning-vol-1.html) +- [Anatomy of a Flink Program]({% link dev/api_concepts.zh.md %}#anatomy-of-a-flink-program) +- [Data Sources]({% link dev/datastream_api.zh.md %}#data-sources) +- [Data Sinks]({% link dev/datastream_api.zh.md %}#data-sinks) +- [DataStream Connectors]({% link dev/connectors/index.zh.md %}) + +{% top %} diff --git a/docs/tutorials/etl.md b/docs/tutorials/etl.md new file mode 100644 index 0000000000000000000000000000000000000000..59dfa1b41ffdd60847625550f24e7c329ff0e693 --- /dev/null +++ b/docs/tutorials/etl.md @@ -0,0 +1,536 @@ +--- +title: Data Pipelines & ETL +nav-id: etl +nav-pos: 3 +nav-title: Data Pipelines & ETL +nav-parent_id: tutorials +--- + + +One very common use case for Apache Flink is to implement ETL (extract, transform, load) pipelines +that take data from one or more sources, perform some transformations and/or enrichments, and +then store the results somewhere. In this tutorial we are going to look at how to use Flink's +DataStream API to implement this kind of application. + +Note that Flink's [Table and SQL APIs]({% link dev/table/index.md %}) +are well suited for many ETL use cases. But regardless of whether you ultimately use +the DataStream API directly, or not, having a solid understanding the basics presented here will +prove valuable. + +* This will be replaced by the TOC +{:toc} + +## Stateless Transformations + +This section covers `map()` and `flatmap()`, the basic operations used to implement +stateless transformations. The examples in this section assume you are familiar with the +Taxi Ride data used in the hands-on exercises in the +[flink-training repo](https://github.com/apache/flink-training/tree/{% if site.is_stable %}release-{{ site.version_title }}{% else %}master{% endif %}). + +### `map()` + +In the first exercise you filtered a stream of taxi ride events. In that same code base there's a +`GeoUtils` class that provides a static method `GeoUtils.mapToGridCell(float lon, float lat)` which +maps a location (longitude, latitude) to a grid cell that refers to an area that is approximately +100x100 meters in size. + +Now let's enrich our stream of taxi ride objects by adding `startCell` and `endCell` fields to each +event. You can create an `EnrichedRide` object that extends `TaxiRide`, adding these fields: + +{% highlight java %} +public static class EnrichedRide extends TaxiRide { + public int startCell; + public int endCell; + + public EnrichedRide() {} + + public EnrichedRide(TaxiRide ride) { + this.rideId = ride.rideId; + this.isStart = ride.isStart; + ... + this.startCell = GeoUtils.mapToGridCell(ride.startLon, ride.startLat); + this.endCell = GeoUtils.mapToGridCell(ride.endLon, ride.endLat); + } + + public String toString() { + return super.toString() + "," + + Integer.toString(this.startCell) + "," + + Integer.toString(this.endCell); + } +} +{% endhighlight %} + +You can then create an application that transforms the stream + +{% highlight java %} +DataStream rides = env.addSource(new TaxiRideSource(...)); + +DataStream enrichedNYCRides = rides + .filter(new RideCleansingSolution.NYCFilter()) + .map(new Enrichment()); + +enrichedNYCRides.print(); +{% endhighlight %} + +with this `MapFunction`: + +{% highlight java %} +public static class Enrichment implements MapFunction { + + @Override + public EnrichedRide map(TaxiRide taxiRide) throws Exception { + return new EnrichedRide(taxiRide); + } +} +{% endhighlight %} + +### `flatmap()` + +A `MapFunction` is suitable only when performing a one-to-one transformation: for each and every +stream element coming in, `map()` will emit one transformed element. Otherwise, you will want to use +`flatmap()` + +{% highlight java %} +DataStream rides = env.addSource(new TaxiRideSource(...)); + +DataStream enrichedNYCRides = rides + .flatMap(new NYCEnrichment()); + +enrichedNYCRides.print(); +{% endhighlight %} + +together with a `FlatMapFunction`: + +{% highlight java %} +public static class NYCEnrichment implements FlatMapFunction { + + @Override + public void flatMap(TaxiRide taxiRide, Collector out) throws Exception { + FilterFunction valid = new RideCleansing.NYCFilter(); + if (valid.filter(taxiRide)) { + out.collect(new EnrichedRide(taxiRide)); + } + } +} +{% endhighlight %} + +With the `Collector` provided in this interface, the `flatmap()` method can emit as many stream +elements as you like, including none at all. + +{% top %} + +## Keyed Streams + +### `keyBy()` + +It is often very useful to be able to partition a stream around one of its attributes, so that all +events with the same value of that attribute are grouped together. For example, suppose you wanted +to find the longest taxi rides starting in each of the grid cells. Thinking in terms of a SQL query, +this would mean doing some sort of GROUP BY with the `startCell`, while in Flink this is done with +`keyBy(KeySelector)` + +{% highlight java %} +rides + .flatMap(new NYCEnrichment()) + .keyBy("startCell") +{% endhighlight %} + +Every `keyBy` causes a network shuffle that repartitions the stream. In general this is pretty +expensive, since it involves network communication along with serialization and deserialization. + +keyBy and network shuffle + +In the example above, the key has been specified by a field name, "startCell". This style of key +selection has the drawback that the compiler is unable to infer the type of the field being used for +keying, and so Flink will pass around the key values as Tuples, which can be awkward. It is +better to use a properly typed KeySelector, e.g., + +{% highlight java %} +rides + .flatMap(new NYCEnrichment()) + .keyBy( + new KeySelector() { + + @Override + public int getKey(EnrichedRide enrichedRide) throws Exception { + return enrichedRide.startCell; + } + }) +{% endhighlight %} + +which can be more succinctly expressed with a lambda: + +{% highlight java %} +rides + .flatMap(new NYCEnrichment()) + .keyBy(enrichedRide -> enrichedRide.startCell) +{% endhighlight %} + +### Keys are computed + +KeySelectors aren't limited to extracting a key from your events. They can, instead, +compute the key in whatever way you want, so long as the resulting key is deterministic, +and has valid implementations of `hashCode()` and `equals()`. This restriction rules out +KeySelectors that generate random numbers, or that return Arrays or Enums, but you +can have composite keys using Tuples or POJOs, for example, so long as their elements +follow these same rules. + +The keys must be produced in a deterministic way, because they are recomputed whenever they +are needed, rather than being attached to the stream records. + +For example, rather than creating a new `EnrichedRide` class with a `startCell` field that we then use +as a key via + +{% highlight java %} +keyBy(enrichedRide -> enrichedRide.startCell) +{% endhighlight %} + +we could do this, instead: + +{% highlight java %} +keyBy(ride -> GeoUtils.mapToGridCell(ride.startLon, ride.startLat)) +{% endhighlight %} + +### Aggregations on Keyed Streams + +This bit of code creates a new stream of tuples containing the `startCell` and duration (in minutes) +for each end-of-ride event: + +{% highlight java %} +import org.joda.time.Interval; + +DataStream> minutesByStartCell = enrichedNYCRides + .flatMap(new FlatMapFunction>() { + + @Override + public void flatMap(EnrichedRide ride, + Collector> out) throws Exception { + if (!ride.isStart) { + Interval rideInterval = new Interval(ride.startTime, ride.endTime); + Minutes duration = rideInterval.toDuration().toStandardMinutes(); + out.collect(new Tuple2<>(ride.startCell, duration)); + } + } + }); +{% endhighlight %} + +Now it is possible to produce a stream that contains only those rides that are the longest rides +ever seen (to that point) for each `startCell`. + +There are a variety of ways that the field to use as the key can be expressed. Earlier you saw an +example with an `EnrichedRide` POJO, where the field to use as the key was specified with its name. +This case involves `Tuple2` objects, and the index within the tuple (starting from 0) is used to +specify the key. + +{% highlight java %} +minutesByStartCell + .keyBy(0) // startCell + .maxBy(1) // duration + .print(); +{% endhighlight %} + +The output stream now contains a record for each key every time the duration reaches a new maximum -- as shown here with cell 50797: + + ... + 4> (64549,5M) + 4> (46298,18M) + 1> (51549,14M) + 1> (53043,13M) + 1> (56031,22M) + 1> (50797,6M) + ... + 1> (50797,8M) + ... + 1> (50797,11M) + ... + 1> (50797,12M) + +### (Implicit) State + +This is the first example in these tutorials that involves stateful streaming. Though the state is +being handled transparently, Flink has to keep track of the maximum duration for each distinct +key. + +Whenever state gets involved in your application, you should think about how large the state might +become. Whenever the key space is unbounded, then so is the amount of state Flink will need. + +When working with streams, it generally makes more sense to think in terms of aggregations over +finite windows, rather than over the entire stream. + +### `reduce()` and other aggregators + +`maxBy()`, used above, is just one example of a number of aggregator functions available on Flink's +`KeyedStream`s. There is also a more general purpose `reduce()` function that you can use to +implement your own custom aggregations. + +{% top %} + +## Stateful Transformations + +### Why is Flink Involved in Managing State? + +Your applications are certainly capable of using state without getting Flink involved in managing it +-- but Flink offers some compelling features for the state it manages: + +* **local**: Flink state is kept local to the machine that processes it, and can be accessed at memory speed +* **durable**: Flink state is fault-tolerant, i.e., it is automatically checkpointed at regular intervals, and is restored upon failure +* **vertically scalable**: Flink state can be kept in embedded RocksDB instances that scale by adding more local disk +* **horizontally scalable**: Flink state is redistributed as your cluster grows and shrinks +* **queryable**: Flink state can be queried externally via the [Queryable State API]({% link dev/stream/state/queryable_state.md %}). + +In this section you will learn how to work with Flink's APIs that manage keyed state. + +### Rich Functions + +At this point you have already seen several of Flink's function interfaces, including +`FilterFunction`, `MapFunction`, and `FlatMapFunction`. These are all examples of the Single +Abstract Method pattern. + +For each of these interfaces, Flink also provides a so-called "rich" variant, e.g., +`RichFlatMapFunction`, which has some additional methods, including: + +- `open(Configuration c)` +- `close()` +- `getRuntimeContext()` + +`open()` is called once, during operator initialization. This is an opportunity to load some static +data, or to open a connection to an external service, for example. + +`getRuntimeContext()` provides access to a whole suite of potentially interesting things, but most +notably it is how you can create and access state managed by Flink. + +### An Example with Keyed State + +In this example, imagine you have a stream of events that you want to de-duplicate, so that you only +keep the first event with each key. Here's an application that does that, using a +`RichFlatMapFunction` called `Deduplicator`: + +{% highlight java %} +private static class Event { + public final String key; + public final long timestamp; + ... +} + +public static void main(String[] args) throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + env.addSource(new EventSource()) + .keyBy(e -> e.key) + .flatMap(new Deduplicator()) + .print(); + + env.execute(); +} +{% endhighlight %} + +To accomplish this, `Deduplicator` will need to somehow remember, for each key, whether or not there +has already been an event for that key. It will do so using Flink's _keyed state_ interface. + +When you are working with a keyed stream like this one, Flink will maintain a key/value store for +each item of state being managed. + +Flink supports several different types of keyed state, and this example uses the simplest one, +namely `ValueState`. This means that _for each key_, Flink will store a single object -- in this +case, an object of type `Boolean`. + +Our `Deduplicator` class has two methods: `open()` and `flatMap()`. The open method establishes the +use of managed state by defining a `ValueStateDescriptor`. The arguments to the constructor +specify a name for this item of keyed state ("keyHasBeenSeen"), and provide information that can be +used to serialize these objects (in this case, `Types.BOOLEAN`). + +{% highlight java %} +public static class Deduplicator extends RichFlatMapFunction { + ValueState keyHasBeenSeen; + + @Override + public void open(Configuration conf) { + ValueStateDescriptor desc = new ValueStateDescriptor<>("keyHasBeenSeen", Types.BOOLEAN); + keyHasBeenSeen = getRuntimeContext().getState(desc); + } + + @Override + public void flatMap(Event event, Collector out) throws Exception { + if (keyHasBeenSeen.value() == null) { + out.collect(event); + keyHasBeenSeen.update(true); + } + } +} +{% endhighlight %} + +When the flatMap method calls `keyHasBeenSeen.value()`, Flink's runtime looks up the value of this +piece of state _for the key in context_, and only if it is `null` does it go ahead and collect the +event to the output. It also updates `keyHasBeenSeen` to `true` in this case. + +This mechanism for accessing and updating key-partitioned state may seem rather magical, since the +key is not explicitly visible in the implementation of our `Deduplicator`. When Flink's runtime +calls the `open` method of our `RichFlatMapFunction`, there is no event, and thus no key in context +at that moment. But when it calls the `flatMap` method, the key for the event being processed is +available to the runtime, and is used behind the scenes to determine which entry in Flink's state +backend is being operated on. + +When deployed to a distributed cluster, there will be many instances of this `Deduplicator`, each of +which will responsible for a disjoint subset of the entire keyspace. Thus, when you see a single +item of `ValueState`, such as + +{% highlight java %} +ValueState keyHasBeenSeen; +{% endhighlight %} + +understand that this represents not just a single Boolean, but rather a distributed, sharded, key/value store. + +### Clearing State + +There's a potential problem with the example above: What will happen if the key space is unbounded? +Flink is storing somewhere an instance of `Boolean` for every distinct key that is used. If there's +a bounded set of keys then this will be fine, but in applications where the set of keys is growing +in an unbounded way, it's necessary to clear the state for keys that are no longer needed. This is +done by calling `clear()` on the state object, as in: + +{% highlight java %} +keyHasBeenSeen.clear() +{% endhighlight %} + +You might want to do this, for example, after a period of inactivity for a given key. You'll see how +to use Timers to do this when you learn about `ProcessFunction`s in the tutorial on event-driven +applications. + +There's also a [State Time-to-Live (TTL)]({% link dev/stream/state/state.md +%}#state-time-to-live-ttl) option that you can configure with the state descriptor that specifies +when you want the state for stale keys to be automatically cleared. + +### Non-keyed State + +It is also possible to work with managed state in non-keyed contexts. This is sometimes called +[operator state]({% link dev/stream/state/state.md %}#operator-state). The +interfaces involved are somewhat different, and since it is unusual for user-defined functions to +need non-keyed state, it is not covered here. This feature is most often used in the implementation +of sources and sinks. + +{% top %} + +## Connected Streams + +Sometimes instead of applying a pre-defined transformation like this: + +simple transformation + +you want to be able to dynamically alter some aspects of the transformation -- by streaming in +thresholds, or rules, or other parameters. The pattern in Flink that supports this is something +called _connected streams_, wherein a single operator has two input streams, like this: + +connected streams + +Connected streams can also be used to implement streaming joins. + +### Example + +In this example, a control stream is used to specify words which must be filtered out of the +`streamOfWords`. A `RichCoFlatMapFunction` called `ControlFunction` is applied to the connected +streams to get this done. + +{% highlight java %} +public static void main(String[] args) throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream control = env.fromElements("DROP", "IGNORE").keyBy(x -> x); + DataStream streamOfWords = env.fromElements("Apache", "DROP", "Flink", "IGNORE").keyBy(x -> x); + + control + .connect(datastreamOfWords) + .flatMap(new ControlFunction()) + .print(); + + env.execute(); +} +{% endhighlight %} + +Note that the two streams being connected must be keyed in compatible ways. +The role of a `keyBy` is to partition a stream's data, and when keyed streams are connected, they +must be partitioned in the same way. This ensures that all of the events from both streams with the +same key are sent to the same instance. This makes it possible, then, to join the two streams on +that key, for example. + +In this case the streams are both of type `DataStream`, and both streams are keyed by the +string. As you will see below, this `RichCoFlatMapFunction` is storing a Boolean value in keyed +state, and this Boolean is shared by the two streams. + +{% highlight java %} +public static class ControlFunction extends RichCoFlatMapFunction { + private ValueState blocked; + + @Override + public void open(Configuration config) { + blocked = getRuntimeContext().getState(new ValueStateDescriptor<>("blocked", Boolean.class)); + } + + @Override + public void flatMap1(String control_value, Collector out) throws Exception { + blocked.update(Boolean.TRUE); + } + + @Override + public void flatMap2(String data_value, Collector out) throws Exception { + if (blocked.value() == null) { + out.collect(data_value); + } + } +} +{% endhighlight %} + +A `RichCoFlatMapFunction` is a kind of `FlatMapFunction` that can be applied to a pair of connected +streams, and it has access to the rich function interface. This means that it can be made stateful. + +The `blocked` Boolean is being used to remember the keys (words, in this case) that have been +mentioned on the `control` stream, and those words are being filtered out of the `streamOfWords` +stream. This is _keyed_ state, and it is shared between the two streams, which is why the two +streams have to share the same keyspace. + +`flatMap1` and `flatMap2` are called by the Flink runtime with elements from each of the two +connected streams -- in our case, elements from the `control` stream are passed into `flatMap1`, and +elements from `streamOfWords` are passed into `flatMap2`. This was determined by the order in which +the two streams are connected with `control.connect(datastreamOfWords)`. + +It is important to recognize that you have no control over the order in which the `flatMap1` and +`flatMap2` callbacks are called. These two input streams are racing against each other, and the +Flink runtime will do what it wants to regarding consuming events from one stream or the other. In +cases where timing and/or ordering matter, you may find it necessary to buffer events in managed +Flink state until your application is ready to process them. (Note: if you are truly desperate, it +is possible to exert some limited control over the order in which a two-input operator consumes its +inputs by using a custom Operator that implements the +[InputSelectable]({{ site.javadocs_baseurl }}/api/java/org/apache/flink/streaming/api/operators/InputSelectable.html) +interface.) + +{% top %} + +## Hands-on + +The hands-on exercise that goes with this section is the [Rides and Fares +Exercise](https://github.com/apache/flink-training/tree/{% if site.is_stable %}release-{{ site.version_title }}{% else %}master{% endif %}/rides-and-fares). + +{% top %} + +## Further Reading + +- [DataStream Transformations]({% link dev/stream/operators/index.md %}#datastream-transformations) +- [Stateful Stream Processing]({% link concepts/stateful-stream-processing.md %}) + +{% top %} diff --git a/docs/tutorials/etl.zh.md b/docs/tutorials/etl.zh.md new file mode 100644 index 0000000000000000000000000000000000000000..79a4d620c4b8fe05aec2b0cb93867953f12e69bd --- /dev/null +++ b/docs/tutorials/etl.zh.md @@ -0,0 +1,536 @@ +--- +title: Data Pipelines & ETL +nav-id: etl +nav-pos: 3 +nav-title: Data Pipelines & ETL +nav-parent_id: tutorials +--- + + +One very common use case for Apache Flink is to implement ETL (extract, transform, load) pipelines +that take data from one or more sources, perform some transformations and/or enrichments, and +then store the results somewhere. In this tutorial we are going to look at how to use Flink's +DataStream API to implement this kind of application. + +Note that Flink's [Table and SQL APIs]({% link dev/table/index.zh.md %}) +are well suited for many ETL use cases. But regardless of whether you ultimately use +the DataStream API directly, or not, having a solid understanding the basics presented here will +prove valuable. + +* This will be replaced by the TOC +{:toc} + +## Stateless Transformations + +This section covers `map()` and `flatmap()`, the basic operations used to implement +stateless transformations. The examples in this section assume you are familiar with the +Taxi Ride data used in the hands-on exercises in the +[flink-training repo](https://github.com/apache/flink-training/tree/{% if site.is_stable %}release-{{ site.version_title }}{% else %}master{% endif %}). + +### `map()` + +In the first exercise you filtered a stream of taxi ride events. In that same code base there's a +`GeoUtils` class that provides a static method `GeoUtils.mapToGridCell(float lon, float lat)` which +maps a location (longitude, latitude) to a grid cell that refers to an area that is approximately +100x100 meters in size. + +Now let's enrich our stream of taxi ride objects by adding `startCell` and `endCell` fields to each +event. You can create an `EnrichedRide` object that extends `TaxiRide`, adding these fields: + +{% highlight java %} +public static class EnrichedRide extends TaxiRide { + public int startCell; + public int endCell; + + public EnrichedRide() {} + + public EnrichedRide(TaxiRide ride) { + this.rideId = ride.rideId; + this.isStart = ride.isStart; + ... + this.startCell = GeoUtils.mapToGridCell(ride.startLon, ride.startLat); + this.endCell = GeoUtils.mapToGridCell(ride.endLon, ride.endLat); + } + + public String toString() { + return super.toString() + "," + + Integer.toString(this.startCell) + "," + + Integer.toString(this.endCell); + } +} +{% endhighlight %} + +You can then create an application that transforms the stream + +{% highlight java %} +DataStream rides = env.addSource(new TaxiRideSource(...)); + +DataStream enrichedNYCRides = rides + .filter(new RideCleansingSolution.NYCFilter()) + .map(new Enrichment()); + +enrichedNYCRides.print(); +{% endhighlight %} + +with this `MapFunction`: + +{% highlight java %} +public static class Enrichment implements MapFunction { + + @Override + public EnrichedRide map(TaxiRide taxiRide) throws Exception { + return new EnrichedRide(taxiRide); + } +} +{% endhighlight %} + +### `flatmap()` + +A `MapFunction` is suitable only when performing a one-to-one transformation: for each and every +stream element coming in, `map()` will emit one transformed element. Otherwise, you will want to use +`flatmap()` + +{% highlight java %} +DataStream rides = env.addSource(new TaxiRideSource(...)); + +DataStream enrichedNYCRides = rides + .flatMap(new NYCEnrichment()); + +enrichedNYCRides.print(); +{% endhighlight %} + +together with a `FlatMapFunction`: + +{% highlight java %} +public static class NYCEnrichment implements FlatMapFunction { + + @Override + public void flatMap(TaxiRide taxiRide, Collector out) throws Exception { + FilterFunction valid = new RideCleansing.NYCFilter(); + if (valid.filter(taxiRide)) { + out.collect(new EnrichedRide(taxiRide)); + } + } +} +{% endhighlight %} + +With the `Collector` provided in this interface, the `flatmap()` method can emit as many stream +elements as you like, including none at all. + +{% top %} + +## Keyed Streams + +### `keyBy()` + +It is often very useful to be able to partition a stream around one of its attributes, so that all +events with the same value of that attribute are grouped together. For example, suppose you wanted +to find the longest taxi rides starting in each of the grid cells. Thinking in terms of a SQL query, +this would mean doing some sort of GROUP BY with the `startCell`, while in Flink this is done with +`keyBy(KeySelector)` + +{% highlight java %} +rides + .flatMap(new NYCEnrichment()) + .keyBy("startCell") +{% endhighlight %} + +Every `keyBy` causes a network shuffle that repartitions the stream. In general this is pretty +expensive, since it involves network communication along with serialization and deserialization. + +keyBy and network shuffle + +In the example above, the key has been specified by a field name, "startCell". This style of key +selection has the drawback that the compiler is unable to infer the type of the field being used for +keying, and so Flink will pass around the key values as Tuples, which can be awkward. It is +better to use a properly typed KeySelector, e.g., + +{% highlight java %} +rides + .flatMap(new NYCEnrichment()) + .keyBy( + new KeySelector() { + + @Override + public int getKey(EnrichedRide enrichedRide) throws Exception { + return enrichedRide.startCell; + } + }) +{% endhighlight %} + +which can be more succinctly expressed with a lambda: + +{% highlight java %} +rides + .flatMap(new NYCEnrichment()) + .keyBy(enrichedRide -> enrichedRide.startCell) +{% endhighlight %} + +### Keys are computed + +KeySelectors aren't limited to extracting a key from your events. They can, instead, +compute the key in whatever way you want, so long as the resulting key is deterministic, +and has valid implementations of `hashCode()` and `equals()`. This restriction rules out +KeySelectors that generate random numbers, or that return Arrays or Enums, but you +can have composite keys using Tuples or POJOs, for example, so long as their elements +follow these same rules. + +The keys must be produced in a deterministic way, because they are recomputed whenever they +are needed, rather than being attached to the stream records. + +For example, rather than creating a new `EnrichedRide` class with a `startCell` field that we then use +as a key via + +{% highlight java %} +keyBy(enrichedRide -> enrichedRide.startCell) +{% endhighlight %} + +we could do this, instead: + +{% highlight java %} +keyBy(ride -> GeoUtils.mapToGridCell(ride.startLon, ride.startLat)) +{% endhighlight %} + +### Aggregations on Keyed Streams + +This bit of code creates a new stream of tuples containing the `startCell` and duration (in minutes) +for each end-of-ride event: + +{% highlight java %} +import org.joda.time.Interval; + +DataStream> minutesByStartCell = enrichedNYCRides + .flatMap(new FlatMapFunction>() { + + @Override + public void flatMap(EnrichedRide ride, + Collector> out) throws Exception { + if (!ride.isStart) { + Interval rideInterval = new Interval(ride.startTime, ride.endTime); + Minutes duration = rideInterval.toDuration().toStandardMinutes(); + out.collect(new Tuple2<>(ride.startCell, duration)); + } + } + }); +{% endhighlight %} + +Now it is possible to produce a stream that contains only those rides that are the longest rides +ever seen (to that point) for each `startCell`. + +There are a variety of ways that the field to use as the key can be expressed. Earlier you saw an +example with an `EnrichedRide` POJO, where the field to use as the key was specified with its name. +This case involves `Tuple2` objects, and the index within the tuple (starting from 0) is used to +specify the key. + +{% highlight java %} +minutesByStartCell + .keyBy(0) // startCell + .maxBy(1) // duration + .print(); +{% endhighlight %} + +The output stream now contains a record for each key every time the duration reaches a new maximum -- as shown here with cell 50797: + + ... + 4> (64549,5M) + 4> (46298,18M) + 1> (51549,14M) + 1> (53043,13M) + 1> (56031,22M) + 1> (50797,6M) + ... + 1> (50797,8M) + ... + 1> (50797,11M) + ... + 1> (50797,12M) + +### (Implicit) State + +This is the first example in these tutorials that involves stateful streaming. Though the state is +being handled transparently, Flink has to keep track of the maximum duration for each distinct +key. + +Whenever state gets involved in your application, you should think about how large the state might +become. Whenever the key space is unbounded, then so is the amount of state Flink will need. + +When working with streams, it generally makes more sense to think in terms of aggregations over +finite windows, rather than over the entire stream. + +### `reduce()` and other aggregators + +`maxBy()`, used above, is just one example of a number of aggregator functions available on Flink's +`KeyedStream`s. There is also a more general purpose `reduce()` function that you can use to +implement your own custom aggregations. + +{% top %} + +## Stateful Transformations + +### Why is Flink Involved in Managing State? + +Your applications are certainly capable of using state without getting Flink involved in managing it +-- but Flink offers some compelling features for the state it manages: + +* **local**: Flink state is kept local to the machine that processes it, and can be accessed at memory speed +* **durable**: Flink state is fault-tolerant, i.e., it is automatically checkpointed at regular intervals, and is restored upon failure +* **vertically scalable**: Flink state can be kept in embedded RocksDB instances that scale by adding more local disk +* **horizontally scalable**: Flink state is redistributed as your cluster grows and shrinks +* **queryable**: Flink state can be queried externally via the [Queryable State API]({% link dev/stream/state/queryable_state.zh.md %}). + +In this section you will learn how to work with Flink's APIs that manage keyed state. + +### Rich Functions + +At this point you have already seen several of Flink's function interfaces, including +`FilterFunction`, `MapFunction`, and `FlatMapFunction`. These are all examples of the Single +Abstract Method pattern. + +For each of these interfaces, Flink also provides a so-called "rich" variant, e.g., +`RichFlatMapFunction`, which has some additional methods, including: + +- `open(Configuration c)` +- `close()` +- `getRuntimeContext()` + +`open()` is called once, during operator initialization. This is an opportunity to load some static +data, or to open a connection to an external service, for example. + +`getRuntimeContext()` provides access to a whole suite of potentially interesting things, but most +notably it is how you can create and access state managed by Flink. + +### An Example with Keyed State + +In this example, imagine you have a stream of events that you want to de-duplicate, so that you only +keep the first event with each key. Here's an application that does that, using a +`RichFlatMapFunction` called `Deduplicator`: + +{% highlight java %} +private static class Event { + public final String key; + public final long timestamp; + ... +} + +public static void main(String[] args) throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + env.addSource(new EventSource()) + .keyBy(e -> e.key) + .flatMap(new Deduplicator()) + .print(); + + env.execute(); +} +{% endhighlight %} + +To accomplish this, `Deduplicator` will need to somehow remember, for each key, whether or not there +has already been an event for that key. It will do so using Flink's _keyed state_ interface. + +When you are working with a keyed stream like this one, Flink will maintain a key/value store for +each item of state being managed. + +Flink supports several different types of keyed state, and this example uses the simplest one, +namely `ValueState`. This means that _for each key_, Flink will store a single object -- in this +case, an object of type `Boolean`. + +Our `Deduplicator` class has two methods: `open()` and `flatMap()`. The open method establishes the +use of managed state by defining a `ValueStateDescriptor`. The arguments to the constructor +specify a name for this item of keyed state ("keyHasBeenSeen"), and provide information that can be +used to serialize these objects (in this case, `Types.BOOLEAN`). + +{% highlight java %} +public static class Deduplicator extends RichFlatMapFunction { + ValueState keyHasBeenSeen; + + @Override + public void open(Configuration conf) { + ValueStateDescriptor desc = new ValueStateDescriptor<>("keyHasBeenSeen", Types.BOOLEAN); + keyHasBeenSeen = getRuntimeContext().getState(desc); + } + + @Override + public void flatMap(Event event, Collector out) throws Exception { + if (keyHasBeenSeen.value() == null) { + out.collect(event); + keyHasBeenSeen.update(true); + } + } +} +{% endhighlight %} + +When the flatMap method calls `keyHasBeenSeen.value()`, Flink's runtime looks up the value of this +piece of state _for the key in context_, and only if it is `null` does it go ahead and collect the +event to the output. It also updates `keyHasBeenSeen` to `true` in this case. + +This mechanism for accessing and updating key-partitioned state may seem rather magical, since the +key is not explicitly visible in the implementation of our `Deduplicator`. When Flink's runtime +calls the `open` method of our `RichFlatMapFunction`, there is no event, and thus no key in context +at that moment. But when it calls the `flatMap` method, the key for the event being processed is +available to the runtime, and is used behind the scenes to determine which entry in Flink's state +backend is being operated on. + +When deployed to a distributed cluster, there will be many instances of this `Deduplicator`, each of +which will responsible for a disjoint subset of the entire keyspace. Thus, when you see a single +item of `ValueState`, such as + +{% highlight java %} +ValueState keyHasBeenSeen; +{% endhighlight %} + +understand that this represents not just a single Boolean, but rather a distributed, sharded, key/value store. + +### Clearing State + +There's a potential problem with the example above: What will happen if the key space is unbounded? +Flink is storing somewhere an instance of `Boolean` for every distinct key that is used. If there's +a bounded set of keys then this will be fine, but in applications where the set of keys is growing +in an unbounded way, it's necessary to clear the state for keys that are no longer needed. This is +done by calling `clear()` on the state object, as in: + +{% highlight java %} +keyHasBeenSeen.clear() +{% endhighlight %} + +You might want to do this, for example, after a period of inactivity for a given key. You'll see how +to use Timers to do this when you learn about `ProcessFunction`s in the tutorial on event-driven +applications. + +There's also a [State Time-to-Live (TTL)]({% link dev/stream/state/state.zh.md +%}#state-time-to-live-ttl) option that you can configure with the state descriptor that specifies +when you want the state for stale keys to be automatically cleared. + +### Non-keyed State + +It is also possible to work with managed state in non-keyed contexts. This is sometimes called +[operator state]({% link dev/stream/state/state.zh.md %}#operator-state). The +interfaces involved are somewhat different, and since it is unusual for user-defined functions to +need non-keyed state, it is not covered here. This feature is most often used in the implementation +of sources and sinks. + +{% top %} + +## Connected Streams + +Sometimes instead of applying a pre-defined transformation like this: + +simple transformation + +you want to be able to dynamically alter some aspects of the transformation -- by streaming in +thresholds, or rules, or other parameters. The pattern in Flink that supports this is something +called _connected streams_, wherein a single operator has two input streams, like this: + +connected streams + +Connected streams can also be used to implement streaming joins. + +### Example + +In this example, a control stream is used to specify words which must be filtered out of the +`streamOfWords`. A `RichCoFlatMapFunction` called `ControlFunction` is applied to the connected +streams to get this done. + +{% highlight java %} +public static void main(String[] args) throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream control = env.fromElements("DROP", "IGNORE").keyBy(x -> x); + DataStream streamOfWords = env.fromElements("Apache", "DROP", "Flink", "IGNORE").keyBy(x -> x); + + control + .connect(datastreamOfWords) + .flatMap(new ControlFunction()) + .print(); + + env.execute(); +} +{% endhighlight %} + +Note that the two streams being connected must be keyed in compatible ways. +The role of a `keyBy` is to partition a stream's data, and when keyed streams are connected, they +must be partitioned in the same way. This ensures that all of the events from both streams with the +same key are sent to the same instance. This makes it possible, then, to join the two streams on +that key, for example. + +In this case the streams are both of type `DataStream`, and both streams are keyed by the +string. As you will see below, this `RichCoFlatMapFunction` is storing a Boolean value in keyed +state, and this Boolean is shared by the two streams. + +{% highlight java %} +public static class ControlFunction extends RichCoFlatMapFunction { + private ValueState blocked; + + @Override + public void open(Configuration config) { + blocked = getRuntimeContext().getState(new ValueStateDescriptor<>("blocked", Boolean.class)); + } + + @Override + public void flatMap1(String control_value, Collector out) throws Exception { + blocked.update(Boolean.TRUE); + } + + @Override + public void flatMap2(String data_value, Collector out) throws Exception { + if (blocked.value() == null) { + out.collect(data_value); + } + } +} +{% endhighlight %} + +A `RichCoFlatMapFunction` is a kind of `FlatMapFunction` that can be applied to a pair of connected +streams, and it has access to the rich function interface. This means that it can be made stateful. + +The `blocked` Boolean is being used to remember the keys (words, in this case) that have been +mentioned on the `control` stream, and those words are being filtered out of the `streamOfWords` +stream. This is _keyed_ state, and it is shared between the two streams, which is why the two +streams have to share the same keyspace. + +`flatMap1` and `flatMap2` are called by the Flink runtime with elements from each of the two +connected streams -- in our case, elements from the `control` stream are passed into `flatMap1`, and +elements from `streamOfWords` are passed into `flatMap2`. This was determined by the order in which +the two streams are connected with `control.connect(datastreamOfWords)`. + +It is important to recognize that you have no control over the order in which the `flatMap1` and +`flatMap2` callbacks are called. These two input streams are racing against each other, and the +Flink runtime will do what it wants to regarding consuming events from one stream or the other. In +cases where timing and/or ordering matter, you may find it necessary to buffer events in managed +Flink state until your application is ready to process them. (Note: if you are truly desperate, it +is possible to exert some limited control over the order in which a two-input operator consumes its +inputs by using a custom Operator that implements the +[InputSelectable]({{ site.javadocs_baseurl }}/api/java/org/apache/flink/streaming/api/operators/InputSelectable.html) +interface.) + +{% top %} + +## Hands-on + +The hands-on exercise that goes with this section is the [Rides and Fares +Exercise](https://github.com/apache/flink-training/tree/{% if site.is_stable %}release-{{ site.version_title }}{% else %}master{% endif %}/rides-and-fares). + +{% top %} + +## Further Reading + +- [DataStream Transformations]({% link dev/stream/operators/index.zh.md %}#datastream-transformations) +- [Stateful Stream Processing]({% link concepts/stateful-stream-processing.zh.md %}) + +{% top %} diff --git a/docs/tutorials/event_driven.md b/docs/tutorials/event_driven.md new file mode 100644 index 0000000000000000000000000000000000000000..89e41252858c40da1216bfbdba214db7030cf31e --- /dev/null +++ b/docs/tutorials/event_driven.md @@ -0,0 +1,308 @@ +--- +title: Event-driven Applications +nav-id: event-driven +nav-pos: 5 +nav-title: Event-driven Applications +nav-parent_id: tutorials +--- + + +* This will be replaced by the TOC +{:toc} + +## Process Functions + +### Introduction + +A `ProcessFunction` combines event processing with timers and state, making it a powerful building +block for stream processing applications. This is the basis for creating event-driven applications +with Flink. It is very similar to a `RichFlatMapFunction`, but with the addition of timers. + +### Example + +If you've done the +[hands-on exercise]({% link tutorials/streaming_analytics.md %}#hands-on) +in the [Streaming Analytics tutorial]({% link tutorials/streaming_analytics.md %}), +you will recall that it uses a `TumblingEventTimeWindow` to compute the sum of the tips for +each driver during each hour, like this: + +{% highlight java %} +// compute the sum of the tips per hour for each driver +DataStream> hourlyTips = fares + .keyBy((TaxiFare fare) -> fare.driverId) + .window(TumblingEventTimeWindows.of(Time.hours(1))) + .process(new AddTips()); +{% endhighlight %} + +It is reasonably straightforward, and educational, to do the same thing with a +`KeyedProcessFunction`. Let us begin by replacing the code above with this: + +{% highlight java %} +// compute the sum of the tips per hour for each driver +DataStream> hourlyTips = fares + .keyBy((TaxiFare fare) -> fare.driverId) + .process(new PseudoWindow(Time.hours(1))); +{% endhighlight %} + +In this code snippet a `KeyedProcessFunction` called `PseudoWindow` is being applied to a keyed +stream, the result of which is a `DataStream>` (the same kind of stream +produced by the implementation that uses Flink's built-in time windows). + +The overall outline of `PseudoWindow` has this shape: + +{% highlight java %} +// Compute the sum of the tips for each driver in hour-long windows. +// The keys are driverIds. +public static class PseudoWindow extends + KeyedProcessFunction> { + + private final long durationMsec; + + public PseudoWindow(Time duration) { + this.durationMsec = duration.toMilliseconds(); + } + + @Override + // Called once during initialization. + public void open(Configuration conf) { + . . . + } + + @Override + // Called as each fare arrives to be processed. + public void processElement( + TaxiFare fare, + Context ctx, + Collector> out) throws Exception { + + . . . + } + + @Override + // Called when the current watermark indicates that a window is now complete. + public void onTimer(long timestamp, + OnTimerContext context, + Collector> out) throws Exception { + + . . . + } +} +{% endhighlight %} + +Things to be aware of: + +* There are several types of ProcessFunctions -- this is a `KeyedProcessFunction`, but there are also + `CoProcessFunctions`, `BroadcastProcessFunctions`, etc. + +* A `KeyedProcessFunction` is a kind of `RichFunction`. Being a `RichFunction`, it has access to the `open` + and `getRuntimeContext` methods needed for working with managed keyed state. + +* There are two callbacks to implement: `processElement` and `onTimer`. `processElement` is called + with each incoming event; `onTimer` is called when timers fire. These can be either event time or + processing time timers. Both `processElement` and `onTimer` are provided with a context object + that can be used to interact with a `TimerService` (among other things). Both callbacks are also + passed a `Collector` that can be used to emit results. + +#### The `open()` method + +{% highlight java %} +// Keyed, managed state, with an entry for each window, keyed by the window's end time. +// There is a separate MapState object for each driver. +private transient MapState sumOfTips; + +@Override +public void open(Configuration conf) { + + MapStateDescriptor sumDesc = + new MapStateDescriptor<>("sumOfTips", Long.class, Float.class); + sumOfTips = getRuntimeContext().getMapState(sumDesc); +} +{% endhighlight %} + +Because the fare events can arrive out of order, it will sometimes be necessary to process events +for one hour before having finished computing the results for the previous hour. In fact, if the +watermarking delay is much longer than the window length, then there may be many windows open +simultaneously, rather than just two. This implementation supports this by using a `MapState` that +maps the timestamp for the end of each window to the sum of the tips for that window. + +#### The `processElement()` method + +{% highlight java %} +public void processElement( + TaxiFare fare, + Context ctx, + Collector> out) throws Exception { + + long eventTime = fare.getEventTime(); + TimerService timerService = ctx.timerService(); + + if (eventTime <= timerService.currentWatermark()) { + // This event is late; its window has already been triggered. + } else { + // Round up eventTime to the end of the window containing this event. + long endOfWindow = (eventTime - (eventTime % durationMsec) + durationMsec - 1); + + // Schedule a callback for when the window has been completed. + timerService.registerEventTimeTimer(endOfWindow); + + // Add this fare's tip to the running total for that window. + Float sum = sumOfTips.get(endOfWindow); + if (sum == null) { + sum = 0.0F; + } + sum += fare.tip; + sumOfTips.put(endOfWindow, sum); + } +} +{% endhighlight %} + +Things to consider: + +* What happens with late events? Events that are behind the watermark (i.e., late) are being + dropped. If you want to do something better than this, consider using a side output, which is + explained in the [next section]({% link tutorials/event_driven.md + %}#side-outputs). + +* This example uses a `MapState` where the keys are timestamps, and sets a `Timer` for that same + timestamp. This is a common pattern; it makes it easy and efficient to lookup relevant information + when the timer fires. + +#### The `onTimer()` method + +{% highlight java %} +public void onTimer( + long timestamp, + OnTimerContext context, + Collector> out) throws Exception { + + long driverId = context.getCurrentKey(); + // Look up the result for the hour that just ended. + Float sumOfTips = this.sumOfTips.get(timestamp); + + Tuple3 result = Tuple3.of(driverId, timestamp, sumOfTips); + out.collect(result); + this.sumOfTips.remove(timestamp); +} +{% endhighlight %} + +Observations: + +* The `OnTimerContext context` passed in to `onTimer` can be used to determine the current key. + +* Our pseudo-windows are being triggered when the current watermark reaches the end of each hour, at + which point `onTimer` is called. This onTimer method removes the related entry from `sumOfTips`, + which has the effect of making it impossible to accommodate late events. This is the equivalent of + setting the allowedLateness to zero when working with Flink's time windows. + +### Performance Considerations + +Flink provides `MapState` and `ListState` types that are optimized for RocksDB. Where possible, +these should be used instead of a `ValueState` object holding some sort of collection. The RocksDB +state backend can append to `ListState` without going through (de)serialization, and for `MapState`, each +key/value pair is a separate RocksDB object, so `MapState` can be efficiently accessed and updated. + +{% top %} + +## Side Outputs + +### Introduction + +There are several good reasons to want to have more than one output stream from a Flink operator, such as reporting: + +* exceptions +* malformed events +* late events +* operational alerts, such as timed-out connections to external services + +Side outputs are a convenient way to do this. Beyond error reporting, side outputs are also +a good way to implement an n-way split of a stream. + +### Example + +You are now in a position to do something with the late events that were ignored in the previous +section. + +A side output channel is associated with an `OutputTag`. These tags have generic types that +correspond to the type of the side output's `DataStream`, and they have names. + +{% highlight java %} +private static final OutputTag lateFares = new OutputTag("lateFares") {}; +{% endhighlight %} + +Shown above is a static `OutputTag` that can be referenced both when emitting +late events in the `processElement` method of the `PseudoWindow`: + +{% highlight java %} +if (eventTime <= timerService.currentWatermark()) { + // This event is late; its window has already been triggered. + ctx.output(lateFares, fare); +} else { + . . . +} +{% endhighlight %} + +and when accessing the stream from this side output in the `main` method of the job: + +{% highlight java %} +// compute the sum of the tips per hour for each driver +SingleOutputStreamOperator hourlyTips = fares + .keyBy((TaxiFare fare) -> fare.driverId) + .process(new PseudoWindow(Time.hours(1))); + +hourlyTips.getSideOutput(lateFares).print(); +{% endhighlight %} + +Alternatively, you can use two OutputTags with the +same name to refer to the same side output, but if you do, they must have the same type. + +{% top %} + +## Closing Remarks + +In this example you have seen how a `ProcessFunction` can be used to reimplement a straightforward time +window. Of course, if Flink's built-in windowing API meets your needs, by all means, go ahead and +use it. But if you find yourself considering doing something contorted with Flink's windows, don't +be afraid to roll your own. + +Also, `ProcessFunctions` are useful for many other use cases beyond computing analytics. The hands-on +exercise below provides an example of something completely different. + +Another common use case for ProcessFunctions is for expiring stale state. If you think back to the +[Rides and Fares Exercise](https://github.com/apache/flink-training/tree/{% if site.is_stable %}release-{{ site.version_title }}{% else %}master{% endif %}/rides-and-fares), +where a `RichCoFlatMapFunction` is used to compute a simple join, the sample solution assumes that +the TaxiRides and TaxiFares are perfectly matched, one-to-one for each `rideId`. If an event is lost, +the other event for the same `rideId` will be held in state forever. This could instead be implemented +as a `KeyedCoProcessFunction`, and a timer could be used to detect and clear any stale state. + +{% top %} + +## Hands-on + +The hands-on exercise that goes with this section is the [Long Ride Alerts +Exercise](https://github.com/apache/flink-training/tree/{% if site.is_stable %}release-{{ site.version_title }}{% else %}master{% endif %}/long-ride-alerts). + +{% top %} + +## Further Reading + +- [ProcessFunction]({% link dev/stream/operators/process_function.md %}) +- [Side Outputs]({% link dev/stream/side_output.md %}) + +{% top %} diff --git a/docs/tutorials/event_driven.zh.md b/docs/tutorials/event_driven.zh.md new file mode 100644 index 0000000000000000000000000000000000000000..21ea8e630e8b7fcd9bcfcc96d76b0e185dfd1f67 --- /dev/null +++ b/docs/tutorials/event_driven.zh.md @@ -0,0 +1,308 @@ +--- +title: Event-driven Applications +nav-id: event-driven +nav-pos: 5 +nav-title: Event-driven Applications +nav-parent_id: tutorials +--- + + +* This will be replaced by the TOC +{:toc} + +## Process Functions + +### Introduction + +A `ProcessFunction` combines event processing with timers and state, making it a powerful building +block for stream processing applications. This is the basis for creating event-driven applications +with Flink. It is very similar to a `RichFlatMapFunction`, but with the addition of timers. + +### Example + +If you've done the +[hands-on exercise]({% link tutorials/streaming_analytics.zh.md %}#hands-on) +in the [Streaming Analytics tutorial]({% link tutorials/streaming_analytics.zh.md %}), +you will recall that it uses a `TumblingEventTimeWindow` to compute the sum of the tips for +each driver during each hour, like this: + +{% highlight java %} +// compute the sum of the tips per hour for each driver +DataStream> hourlyTips = fares + .keyBy((TaxiFare fare) -> fare.driverId) + .window(TumblingEventTimeWindows.of(Time.hours(1))) + .process(new AddTips()); +{% endhighlight %} + +It is reasonably straightforward, and educational, to do the same thing with a +`KeyedProcessFunction`. Let us begin by replacing the code above with this: + +{% highlight java %} +// compute the sum of the tips per hour for each driver +DataStream> hourlyTips = fares + .keyBy((TaxiFare fare) -> fare.driverId) + .process(new PseudoWindow(Time.hours(1))); +{% endhighlight %} + +In this code snippet a `KeyedProcessFunction` called `PseudoWindow` is being applied to a keyed +stream, the result of which is a `DataStream>` (the same kind of stream +produced by the implementation that uses Flink's built-in time windows). + +The overall outline of `PseudoWindow` has this shape: + +{% highlight java %} +// Compute the sum of the tips for each driver in hour-long windows. +// The keys are driverIds. +public static class PseudoWindow extends + KeyedProcessFunction> { + + private final long durationMsec; + + public PseudoWindow(Time duration) { + this.durationMsec = duration.toMilliseconds(); + } + + @Override + // Called once during initialization. + public void open(Configuration conf) { + . . . + } + + @Override + // Called as each fare arrives to be processed. + public void processElement( + TaxiFare fare, + Context ctx, + Collector> out) throws Exception { + + . . . + } + + @Override + // Called when the current watermark indicates that a window is now complete. + public void onTimer(long timestamp, + OnTimerContext context, + Collector> out) throws Exception { + + . . . + } +} +{% endhighlight %} + +Things to be aware of: + +* There are several types of ProcessFunctions -- this is a `KeyedProcessFunction`, but there are also + `CoProcessFunctions`, `BroadcastProcessFunctions`, etc. + +* A `KeyedProcessFunction` is a kind of `RichFunction`. Being a `RichFunction`, it has access to the `open` + and `getRuntimeContext` methods needed for working with managed keyed state. + +* There are two callbacks to implement: `processElement` and `onTimer`. `processElement` is called + with each incoming event; `onTimer` is called when timers fire. These can be either event time or + processing time timers. Both `processElement` and `onTimer` are provided with a context object + that can be used to interact with a `TimerService` (among other things). Both callbacks are also + passed a `Collector` that can be used to emit results. + +#### The `open()` method + +{% highlight java %} +// Keyed, managed state, with an entry for each window, keyed by the window's end time. +// There is a separate MapState object for each driver. +private transient MapState sumOfTips; + +@Override +public void open(Configuration conf) { + + MapStateDescriptor sumDesc = + new MapStateDescriptor<>("sumOfTips", Long.class, Float.class); + sumOfTips = getRuntimeContext().getMapState(sumDesc); +} +{% endhighlight %} + +Because the fare events can arrive out of order, it will sometimes be necessary to process events +for one hour before having finished computing the results for the previous hour. In fact, if the +watermarking delay is much longer than the window length, then there may be many windows open +simultaneously, rather than just two. This implementation supports this by using a `MapState` that +maps the timestamp for the end of each window to the sum of the tips for that window. + +#### The `processElement()` method + +{% highlight java %} +public void processElement( + TaxiFare fare, + Context ctx, + Collector> out) throws Exception { + + long eventTime = fare.getEventTime(); + TimerService timerService = ctx.timerService(); + + if (eventTime <= timerService.currentWatermark()) { + // This event is late; its window has already been triggered. + } else { + // Round up eventTime to the end of the window containing this event. + long endOfWindow = (eventTime - (eventTime % durationMsec) + durationMsec - 1); + + // Schedule a callback for when the window has been completed. + timerService.registerEventTimeTimer(endOfWindow); + + // Add this fare's tip to the running total for that window. + Float sum = sumOfTips.get(endOfWindow); + if (sum == null) { + sum = 0.0F; + } + sum += fare.tip; + sumOfTips.put(endOfWindow, sum); + } +} +{% endhighlight %} + +Things to consider: + +* What happens with late events? Events that are behind the watermark (i.e., late) are being + dropped. If you want to do something better than this, consider using a side output, which is + explained in the [next section]({% link tutorials/event_driven.zh.md + %}#side-outputs). + +* This example uses a `MapState` where the keys are timestamps, and sets a `Timer` for that same + timestamp. This is a common pattern; it makes it easy and efficient to lookup relevant information + when the timer fires. + +#### The `onTimer()` method + +{% highlight java %} +public void onTimer( + long timestamp, + OnTimerContext context, + Collector> out) throws Exception { + + long driverId = context.getCurrentKey(); + // Look up the result for the hour that just ended. + Float sumOfTips = this.sumOfTips.get(timestamp); + + Tuple3 result = Tuple3.of(driverId, timestamp, sumOfTips); + out.collect(result); + this.sumOfTips.remove(timestamp); +} +{% endhighlight %} + +Observations: + +* The `OnTimerContext context` passed in to `onTimer` can be used to determine the current key. + +* Our pseudo-windows are being triggered when the current watermark reaches the end of each hour, at + which point `onTimer` is called. This onTimer method removes the related entry from `sumOfTips`, + which has the effect of making it impossible to accommodate late events. This is the equivalent of + setting the allowedLateness to zero when working with Flink's time windows. + +### Performance Considerations + +Flink provides `MapState` and `ListState` types that are optimized for RocksDB. Where possible, +these should be used instead of a `ValueState` object holding some sort of collection. The RocksDB +state backend can append to `ListState` without going through (de)serialization, and for `MapState`, each +key/value pair is a separate RocksDB object, so `MapState` can be efficiently accessed and updated. + +{% top %} + +## Side Outputs + +### Introduction + +There are several good reasons to want to have more than one output stream from a Flink operator, such as reporting: + +* exceptions +* malformed events +* late events +* operational alerts, such as timed-out connections to external services + +Side outputs are a convenient way to do this. Beyond error reporting, side outputs are also +a good way to implement an n-way split of a stream. + +### Example + +You are now in a position to do something with the late events that were ignored in the previous +section. + +A side output channel is associated with an `OutputTag`. These tags have generic types that +correspond to the type of the side output's `DataStream`, and they have names. + +{% highlight java %} +private static final OutputTag lateFares = new OutputTag("lateFares") {}; +{% endhighlight %} + +Shown above is a static `OutputTag` that can be referenced both when emitting +late events in the `processElement` method of the `PseudoWindow`: + +{% highlight java %} +if (eventTime <= timerService.currentWatermark()) { + // This event is late; its window has already been triggered. + ctx.output(lateFares, fare); +} else { + . . . +} +{% endhighlight %} + +and when accessing the stream from this side output in the `main` method of the job: + +{% highlight java %} +// compute the sum of the tips per hour for each driver +SingleOutputStreamOperator hourlyTips = fares + .keyBy((TaxiFare fare) -> fare.driverId) + .process(new PseudoWindow(Time.hours(1))); + +hourlyTips.getSideOutput(lateFares).print(); +{% endhighlight %} + +Alternatively, you can use two OutputTags with the +same name to refer to the same side output, but if you do, they must have the same type. + +{% top %} + +## Closing Remarks + +In this example you have seen how a `ProcessFunction` can be used to reimplement a straightforward time +window. Of course, if Flink's built-in windowing API meets your needs, by all means, go ahead and +use it. But if you find yourself considering doing something contorted with Flink's windows, don't +be afraid to roll your own. + +Also, `ProcessFunctions` are useful for many other use cases beyond computing analytics. The hands-on +exercise below provides an example of something completely different. + +Another common use case for ProcessFunctions is for expiring stale state. If you think back to the +[Rides and Fares Exercise](https://github.com/apache/flink-training/tree/{% if site.is_stable %}release-{{ site.version_title }}{% else %}master{% endif %}/rides-and-fares), +where a `RichCoFlatMapFunction` is used to compute a simple join, the sample solution assumes that +the TaxiRides and TaxiFares are perfectly matched, one-to-one for each `rideId`. If an event is lost, +the other event for the same `rideId` will be held in state forever. This could instead be implemented +as a `KeyedCoProcessFunction`, and a timer could be used to detect and clear any stale state. + +{% top %} + +## Hands-on + +The hands-on exercise that goes with this section is the [Long Ride Alerts +Exercise](https://github.com/apache/flink-training/tree/{% if site.is_stable %}release-{{ site.version_title }}{% else %}master{% endif %}/long-ride-alerts). + +{% top %} + +## Further Reading + +- [ProcessFunction]({% link dev/stream/operators/process_function.zh.md %}) +- [Side Outputs]({% link dev/stream/side_output.zh.md %}) + +{% top %} diff --git a/docs/tutorials/fault_tolerance.md b/docs/tutorials/fault_tolerance.md new file mode 100644 index 0000000000000000000000000000000000000000..bbe24a5ec0d550cbec10a745c8008051b4c62343 --- /dev/null +++ b/docs/tutorials/fault_tolerance.md @@ -0,0 +1,208 @@ +--- +title: Fault Tolerance via State Snapshots +nav-id: fault-tolerance +nav-pos: 6 +nav-title: Fault Tolerance +nav-parent_id: tutorials +--- + + +* This will be replaced by the TOC +{:toc} + +## State Backends + +The keyed state managed by Flink is a sort of sharded, key/value store, and the working copy of each +item of keyed state is kept somewhere local to the taskmanager responsible for that key. Operator +state is also local to the machine(s) that need(s) it. Flink periodically takes persistent snapshots +of all the state and copies these snapshots somewhere more durable, such as a distributed file +system. + +In the event of the failure, Flink can restore the complete state of your application and resume +processing as though nothing had gone wrong. + +This state that Flink manages is stored in a _state backend_. Two implementations of state backends +are available -- one based on RocksDB, an embedded key/value store that keeps its working state on +disk, and another heap-based state backend that keeps its working state in memory, on the Java heap. +This heap-based state backend comes in two flavors: the FsStateBackend that persists its state +snapshots to a distributed file system, and the MemoryStateBackend that uses the JobManager's heap. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
NameWorking StateState BackupSnapshotting
RocksDBStateBackendLocal disk (tmp dir)Distributed file systemFull / Incremental
+
    +
  • Supports state larger than available memory
  • +
  • Rule of thumb: 10x slower than heap-based backends
  • +
+
FsStateBackendJVM HeapDistributed file systemFull
+
    +
  • Fast, requires large heap
  • +
  • Subject to GC
  • +
+
MemoryStateBackendJVM HeapJobManager JVM HeapFull
+
    +
  • Good for testing and experimentation with small state (locally)
  • +
+
+ +When working with state kept in a heap-based state backend, accesses and updates involve reading and +writing objects on the heap. But for objects kept in the `RocksDBStateBackend`, accesses and updates +involve serialization and deserialization, and so are much more expensive. But the amount of state +you can have with RocksDB is limited only by the size of the local disk. Note also that only the +`RocksDBStateBackend` is able to do incremental snapshotting, which is a significant benefit for +applications with large amounts of slowly changing state. + +All of these state backends are able to do asynchronous snapshotting, meaning that they can take a +snapshot without impeding the ongoing stream processing. + +{% top %} + +## State Snapshots + +### Definitions + +* _Snapshot_ -- a generic term referring to a global, consistent image of the state of a Flink job. + A snapshot includes a pointer into each of the data sources (e.g., an offset into a file or Kafka + partition), as well as a copy of the state from each of the job's stateful operators that resulted + from having processed all of the events up to those positions in the sources. +* _Checkpoint_ -- a snapshot taken automatically by Flink for the purpose of being able to recover + from faults. Checkpoints can be incremental, and are optimized for being restored quickly. +* _Externalized Checkpoint_ -- normally checkpoints are not intended to be manipulated by users. + Flink retains only the _n_-most-recent checkpoints (_n_ being configurable) while a job is + running, and deletes them when a job is cancelled. But you can configure them to be retained + instead, in which case you can manually resume from them. +* _Savepoint_ -- a snapshot triggered manually by a user (or an API call) for some operational + purpose, such as a stateful redeploy/upgrade/rescaling operation. Savepoints are always complete, + and are optimized for operational flexibility. + +### How does State Snapshotting Work? + +Flink uses a variant of the [Chandy-Lamport +algorithm](https://en.wikipedia.org/wiki/Chandy-Lamport_algorithm) known as _asynchronous barrier +snapshotting_. + +When a task manager is instructed by the checkpoint coordinator (part of the job manager) to begin a +checkpoint, it has all of the sources record their offsets and insert numbered _checkpoint barriers_ +into their streams. These barriers flow through the job graph, indicating the part of the stream +before and after each checkpoint. + +Checkpoint barriers are inserted into the streams + +Checkpoint _n_ will contain the state of each operator that resulted from having consumed **every +event before checkpoint barrier _n_, and none of the events after it**. + +As each operator in the job graph receives one of these barriers, it records its state. Operators +with two input streams (such as a `CoProcessFunction`) perform _barrier alignment_ so that the +snapshot will reflect the state resulting from consuming events from both input streams up to (but +not past) both barriers. + +Barrier alignment + +Flink's state backends use a copy-on-write mechanism to allow stream processing to continue +unimpeded while older versions of the state are being asynchronously snapshotted. Only when the +snapshots have been durably persisted will these older versions of the state be garbage collected. + +### Exactly Once Guarantees + +When things go wrong in a stream processing application, it is possible to have either lost, or +duplicated results. With Flink, depending on the choices you make for your application and the +cluster you run it on, any of these outcomes is possible: + +- Flink makes no effort to recover from failures (_at most once_) +- Nothing is lost, but you may experience duplicated results (_at least once_) +- Nothing is lost or duplicated (_exactly once_) + +Given that Flink recovers from faults by rewinding and replaying the source data streams, when the +ideal situation is described as **exactly once** this does *not* mean that every event will be +processed exactly once. Instead, it means that _every event will affect the state being managed by +Flink exactly once_. + +Barrier alignment is only needed for providing exactly once guarantees. If you don't need this, you +can gain some performance by configuring Flink to use `CheckpointingMode.AT_LEAST_ONCE`, which has +the effect of disabling barrier alignment. + +### Exactly Once End-to-end + +To achieve exactly once end-to-end, so that every event from the sources affects the sinks exactly +once, the following must be true: + +1. your sources must be replayable, and +2. your sinks must be transactional (or idempotent) + +{% top %} + +## Hands-on + +The [Flink Operations Playground]({% link +getting-started/docker-playgrounds/flink-operations-playground.md %}) includes a section on +[Observing Failure & Recovery]({% link +getting-started/docker-playgrounds/flink-operations-playground.md %}#observing-failure--recovery). + +{% top %} + +## Further Reading + +- [Stateful Stream Processing]({% link concepts/stateful-stream-processing.md %}) +- [State Backends]({% link ops/state/state_backends.md %}) +- [Fault Tolerance Guarantees of Data Sources and Sinks]({% link dev/connectors/guarantees.md %}) +- [Enabling and Configuring Checkpointing]({% link dev/stream/state/checkpointing.md %}) +- [Checkpoints]({% link ops/state/checkpoints.md %}) +- [Savepoints]({% link ops/state/savepoints.md %}) +- [Tuning Checkpoints and Large State]({% link ops/state/large_state_tuning.md %}) +- [Monitoring Checkpointing]({% link monitoring/checkpoint_monitoring.md %}) +- [Task Failure Recovery]({% link dev/task_failure_recovery.md %}) + +{% top %} diff --git a/docs/tutorials/fault_tolerance.zh.md b/docs/tutorials/fault_tolerance.zh.md new file mode 100644 index 0000000000000000000000000000000000000000..538712060562216d9e609915ddbe093eb74ada57 --- /dev/null +++ b/docs/tutorials/fault_tolerance.zh.md @@ -0,0 +1,208 @@ +--- +title: Fault Tolerance via State Snapshots +nav-id: fault-tolerance +nav-pos: 6 +nav-title: Fault Tolerance +nav-parent_id: tutorials +--- + + +* This will be replaced by the TOC +{:toc} + +## State Backends + +The keyed state managed by Flink is a sort of sharded, key/value store, and the working copy of each +item of keyed state is kept somewhere local to the taskmanager responsible for that key. Operator +state is also local to the machine(s) that need(s) it. Flink periodically takes persistent snapshots +of all the state and copies these snapshots somewhere more durable, such as a distributed file +system. + +In the event of the failure, Flink can restore the complete state of your application and resume +processing as though nothing had gone wrong. + +This state that Flink manages is stored in a _state backend_. Two implementations of state backends +are available -- one based on RocksDB, an embedded key/value store that keeps its working state on +disk, and another heap-based state backend that keeps its working state in memory, on the Java heap. +This heap-based state backend comes in two flavors: the FsStateBackend that persists its state +snapshots to a distributed file system, and the MemoryStateBackend that uses the JobManager's heap. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
NameWorking StateState BackupSnapshotting
RocksDBStateBackendLocal disk (tmp dir)Distributed file systemFull / Incremental
+
    +
  • Supports state larger than available memory
  • +
  • Rule of thumb: 10x slower than heap-based backends
  • +
+
FsStateBackendJVM HeapDistributed file systemFull
+
    +
  • Fast, requires large heap
  • +
  • Subject to GC
  • +
+
MemoryStateBackendJVM HeapJobManager JVM HeapFull
+
    +
  • Good for testing and experimentation with small state (locally)
  • +
+
+ +When working with state kept in a heap-based state backend, accesses and updates involve reading and +writing objects on the heap. But for objects kept in the `RocksDBStateBackend`, accesses and updates +involve serialization and deserialization, and so are much more expensive. But the amount of state +you can have with RocksDB is limited only by the size of the local disk. Note also that only the +`RocksDBStateBackend` is able to do incremental snapshotting, which is a significant benefit for +applications with large amounts of slowly changing state. + +All of these state backends are able to do asynchronous snapshotting, meaning that they can take a +snapshot without impeding the ongoing stream processing. + +{% top %} + +## State Snapshots + +### Definitions + +* _Snapshot_ -- a generic term referring to a global, consistent image of the state of a Flink job. + A snapshot includes a pointer into each of the data sources (e.g., an offset into a file or Kafka + partition), as well as a copy of the state from each of the job's stateful operators that resulted + from having processed all of the events up to those positions in the sources. +* _Checkpoint_ -- a snapshot taken automatically by Flink for the purpose of being able to recover + from faults. Checkpoints can be incremental, and are optimized for being restored quickly. +* _Externalized Checkpoint_ -- normally checkpoints are not intended to be manipulated by users. + Flink retains only the _n_-most-recent checkpoints (_n_ being configurable) while a job is + running, and deletes them when a job is cancelled. But you can configure them to be retained + instead, in which case you can manually resume from them. +* _Savepoint_ -- a snapshot triggered manually by a user (or an API call) for some operational + purpose, such as a stateful redeploy/upgrade/rescaling operation. Savepoints are always complete, + and are optimized for operational flexibility. + +### How does State Snapshotting Work? + +Flink uses a variant of the [Chandy-Lamport +algorithm](https://en.wikipedia.org/wiki/Chandy-Lamport_algorithm) known as _asynchronous barrier +snapshotting_. + +When a task manager is instructed by the checkpoint coordinator (part of the job manager) to begin a +checkpoint, it has all of the sources record their offsets and insert numbered _checkpoint barriers_ +into their streams. These barriers flow through the job graph, indicating the part of the stream +before and after each checkpoint. + +Checkpoint barriers are inserted into the streams + +Checkpoint _n_ will contain the state of each operator that resulted from having consumed **every +event before checkpoint barrier _n_, and none of the events after it**. + +As each operator in the job graph receives one of these barriers, it records its state. Operators +with two input streams (such as a `CoProcessFunction`) perform _barrier alignment_ so that the +snapshot will reflect the state resulting from consuming events from both input streams up to (but +not past) both barriers. + +Barrier alignment + +Flink's state backends use a copy-on-write mechanism to allow stream processing to continue +unimpeded while older versions of the state are being asynchronously snapshotted. Only when the +snapshots have been durably persisted will these older versions of the state be garbage collected. + +### Exactly Once Guarantees + +When things go wrong in a stream processing application, it is possible to have either lost, or +duplicated results. With Flink, depending on the choices you make for your application and the +cluster you run it on, any of these outcomes is possible: + +- Flink makes no effort to recover from failures (_at most once_) +- Nothing is lost, but you may experience duplicated results (_at least once_) +- Nothing is lost or duplicated (_exactly once_) + +Given that Flink recovers from faults by rewinding and replaying the source data streams, when the +ideal situation is described as **exactly once** this does *not* mean that every event will be +processed exactly once. Instead, it means that _every event will affect the state being managed by +Flink exactly once_. + +Barrier alignment is only needed for providing exactly once guarantees. If you don't need this, you +can gain some performance by configuring Flink to use `CheckpointingMode.AT_LEAST_ONCE`, which has +the effect of disabling barrier alignment. + +### Exactly Once End-to-end + +To achieve exactly once end-to-end, so that every event from the sources affects the sinks exactly +once, the following must be true: + +1. your sources must be replayable, and +2. your sinks must be transactional (or idempotent) + +{% top %} + +## Hands-on + +The [Flink Operations Playground]({% link +getting-started/docker-playgrounds/flink-operations-playground.zh.md %}) includes a section on +[Observing Failure & Recovery]({% link +getting-started/docker-playgrounds/flink-operations-playground.zh.md %}#observing-failure--recovery). + +{% top %} + +## Further Reading + +- [Stateful Stream Processing]({% link concepts/stateful-stream-processing.zh.md %}) +- [State Backends]({% link ops/state/state_backends.zh.md %}) +- [Fault Tolerance Guarantees of Data Sources and Sinks]({% link dev/connectors/guarantees.zh.md %}) +- [Enabling and Configuring Checkpointing]({% link dev/stream/state/checkpointing.zh.md %}) +- [Checkpoints]({% link ops/state/checkpoints.zh.md %}) +- [Savepoints]({% link ops/state/savepoints.zh.md %}) +- [Tuning Checkpoints and Large State]({% link ops/state/large_state_tuning.zh.md %}) +- [Monitoring Checkpointing]({% link monitoring/checkpoint_monitoring.zh.md %}) +- [Task Failure Recovery]({% link dev/task_failure_recovery.zh.md %}) + +{% top %} diff --git a/docs/tutorials/index.md b/docs/tutorials/index.md new file mode 100644 index 0000000000000000000000000000000000000000..ac72d91dffb2700066636d4637c87ac50b4fbdcf --- /dev/null +++ b/docs/tutorials/index.md @@ -0,0 +1,186 @@ +--- +title: Hands-on Tutorials +nav-id: tutorials +nav-pos: 2 +nav-title: ' Hands-on Tutorials' +nav-parent_id: root +nav-show_overview: true +always-expand: true +--- + + +* This will be replaced by the TOC +{:toc} + +## Goals and Scope of these Tutorials + +These tutorials present an introduction to Apache Flink that includes just enough to get you started +writing scalable streaming ETL, analytics, and event-driven applications, while leaving out a lot of +(ultimately important) details. The focus is on providing straightforward introductions to Flink's +APIs for managing state and time, with the expectation that having mastered these fundamentals, +you'll be much better equipped to pick up the rest of what you need to know from the more detailed +reference documentation. The links at the end of each page will lead you to where you can learn +more. + +Specifically, you will learn: + +- how to implement streaming data processing pipelines +- how and why Flink manages state +- how to use event time to consistently compute accurate analytics +- how to build event-driven applications on continuous streams +- how Flink is able to provide fault-tolerant, stateful stream processing with exactly-once semantics + +These tutorials focus on four critical concepts: continuous processing of streaming data, event +time, stateful stream processing, and state snapshots. This page introduces these concepts. + +{% info Note %} Accompanying these tutorials are a set of hands-on exercises that will guide you +through learning how to work with the concepts being presented. A link to the relevant exercise +is provided at the end of each tutorial. + +{% top %} + +## Stream Processing + +Streams are data's natural habitat. Whether it is events from web servers, trades from a stock +exchange, or sensor readings from a machine on a factory floor, data is created as part of a stream. +But when you analyze data, you can either organize your processing around _bounded_ or _unbounded_ +streams, and which of these paradigms you choose has profound consequences. + +Bounded and unbounded streams + +**Batch processing** is the paradigm at work when you process a bounded data stream. In this mode of +operation you can choose to ingest the entire dataset before producing any results, which means that +it is possible, for example, to sort the data, compute global statistics, or produce a final report +that summarizes all of the input. + +**Stream processing**, on the other hand, involves unbounded data streams. Conceptually, at least, +the input may never end, and so you are forced to continuously process the data as it arrives. + +In Flink, applications are composed of **streaming dataflows** that may be transformed by +user-defined **operators**. These dataflows form directed graphs that start with one or more +**sources**, and end in one or more **sinks**. + +A DataStream program, and its dataflow. + +Often there is a one-to-one correspondence between the transformations in the program and the +operators in the dataflow. Sometimes, however, one transformation may consist of multiple operators. + +An application may consume real-time data from streaming sources such as message queues or +distributed logs, like Apache Kafka or Kinesis. But flink can also consume bounded, historic data +from a variety of data sources. Similarly, the streams of results being produced by a Flink +application can be sent to a wide variety of systems that can be connected as sinks. + +Flink application with sources and sinks + +### 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. +Different operators of the same program may have different levels of +parallelism. + +A parallel dataflow + +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, for example, the redistribution between the keyBy/window and + the Sink operators shown above introduces non-determinism regarding the + order in which the aggregated results for different keys arrive at the Sink. + +{% top %} + +## Timely Stream Processing + +For most streaming applications it is very valuable to be able re-process historic data with the +same code that is used to process live data -- and to produce deterministic, consistent results, +regardless. + +It can also be crucial to pay attention to the order in which events occurred, rather than the order +in which they are delivered for processing, and to be able to reason about when a set of events is +(or should be) complete. For example, consider the set of events involved in an e-commerce +transaction, or financial trade. + +These requirements for timely stream processing can be met by using event time timestamps that are +recorded in the data stream, rather than using the clocks of the machines processing the data. + +{% top %} + +## Stateful Stream Processing + +Flink's operations can be stateful. This means that how one event is handled can depend on the +accumulated effect of all the events that came before it. State may be used for something simple, +such as counting events per minute to display on a dashboard, or for something more complex, such as +computing features for a fraud detection model. + +A Flink application is run in parallel on a distributed cluster. The various parallel instances of a +given operator will execute independently, in separate threads, and in general will be running on +different machines. + +The set of parallel instances of a stateful operator is effectively a sharded key-value store. Each +parallel instance is responsible for handling events for a specific group of keys, and the state for +those keys is kept locally. + +The diagram below shows a job running with a parallelism of two across the first three operators in +the job graph, terminating in a sink that has a parallelism of one. The third operator is stateful, +and you can see that a fully-connected network shuffle is occurring between the second and third +operators. This is being done to partition the stream by some key, so that all of the events that +need to be processed together, will be. + +State is sharded + +State is always accessed locally, which helps Flink applications achieve high throughput and +low-latency. You can choose to keep state on the JVM heap, or if it is too large, in efficiently +organized on-disk data structures. + +State is local + +{% top %} + +## Fault Tolerance via State Snapshots + +Flink is able to provide fault-tolerant, exactly-once semantics through a combination of state +snapshots and stream replay. These snapshots capture the entire state of the distributed pipeline, +recording offsets into the input queues as well as the state throughout the job graph that has +resulted from having ingested the data up to that point. When a failure occurs, the sources are +rewound, the state is restored, and processing is resumed. As depicted above, these state snapshots +are captured asynchronously, without impeding the ongoing processing. + +{% top %} diff --git a/docs/tutorials/index.zh.md b/docs/tutorials/index.zh.md new file mode 100644 index 0000000000000000000000000000000000000000..ac72d91dffb2700066636d4637c87ac50b4fbdcf --- /dev/null +++ b/docs/tutorials/index.zh.md @@ -0,0 +1,186 @@ +--- +title: Hands-on Tutorials +nav-id: tutorials +nav-pos: 2 +nav-title: ' Hands-on Tutorials' +nav-parent_id: root +nav-show_overview: true +always-expand: true +--- + + +* This will be replaced by the TOC +{:toc} + +## Goals and Scope of these Tutorials + +These tutorials present an introduction to Apache Flink that includes just enough to get you started +writing scalable streaming ETL, analytics, and event-driven applications, while leaving out a lot of +(ultimately important) details. The focus is on providing straightforward introductions to Flink's +APIs for managing state and time, with the expectation that having mastered these fundamentals, +you'll be much better equipped to pick up the rest of what you need to know from the more detailed +reference documentation. The links at the end of each page will lead you to where you can learn +more. + +Specifically, you will learn: + +- how to implement streaming data processing pipelines +- how and why Flink manages state +- how to use event time to consistently compute accurate analytics +- how to build event-driven applications on continuous streams +- how Flink is able to provide fault-tolerant, stateful stream processing with exactly-once semantics + +These tutorials focus on four critical concepts: continuous processing of streaming data, event +time, stateful stream processing, and state snapshots. This page introduces these concepts. + +{% info Note %} Accompanying these tutorials are a set of hands-on exercises that will guide you +through learning how to work with the concepts being presented. A link to the relevant exercise +is provided at the end of each tutorial. + +{% top %} + +## Stream Processing + +Streams are data's natural habitat. Whether it is events from web servers, trades from a stock +exchange, or sensor readings from a machine on a factory floor, data is created as part of a stream. +But when you analyze data, you can either organize your processing around _bounded_ or _unbounded_ +streams, and which of these paradigms you choose has profound consequences. + +Bounded and unbounded streams + +**Batch processing** is the paradigm at work when you process a bounded data stream. In this mode of +operation you can choose to ingest the entire dataset before producing any results, which means that +it is possible, for example, to sort the data, compute global statistics, or produce a final report +that summarizes all of the input. + +**Stream processing**, on the other hand, involves unbounded data streams. Conceptually, at least, +the input may never end, and so you are forced to continuously process the data as it arrives. + +In Flink, applications are composed of **streaming dataflows** that may be transformed by +user-defined **operators**. These dataflows form directed graphs that start with one or more +**sources**, and end in one or more **sinks**. + +A DataStream program, and its dataflow. + +Often there is a one-to-one correspondence between the transformations in the program and the +operators in the dataflow. Sometimes, however, one transformation may consist of multiple operators. + +An application may consume real-time data from streaming sources such as message queues or +distributed logs, like Apache Kafka or Kinesis. But flink can also consume bounded, historic data +from a variety of data sources. Similarly, the streams of results being produced by a Flink +application can be sent to a wide variety of systems that can be connected as sinks. + +Flink application with sources and sinks + +### 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. +Different operators of the same program may have different levels of +parallelism. + +A parallel dataflow + +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, for example, the redistribution between the keyBy/window and + the Sink operators shown above introduces non-determinism regarding the + order in which the aggregated results for different keys arrive at the Sink. + +{% top %} + +## Timely Stream Processing + +For most streaming applications it is very valuable to be able re-process historic data with the +same code that is used to process live data -- and to produce deterministic, consistent results, +regardless. + +It can also be crucial to pay attention to the order in which events occurred, rather than the order +in which they are delivered for processing, and to be able to reason about when a set of events is +(or should be) complete. For example, consider the set of events involved in an e-commerce +transaction, or financial trade. + +These requirements for timely stream processing can be met by using event time timestamps that are +recorded in the data stream, rather than using the clocks of the machines processing the data. + +{% top %} + +## Stateful Stream Processing + +Flink's operations can be stateful. This means that how one event is handled can depend on the +accumulated effect of all the events that came before it. State may be used for something simple, +such as counting events per minute to display on a dashboard, or for something more complex, such as +computing features for a fraud detection model. + +A Flink application is run in parallel on a distributed cluster. The various parallel instances of a +given operator will execute independently, in separate threads, and in general will be running on +different machines. + +The set of parallel instances of a stateful operator is effectively a sharded key-value store. Each +parallel instance is responsible for handling events for a specific group of keys, and the state for +those keys is kept locally. + +The diagram below shows a job running with a parallelism of two across the first three operators in +the job graph, terminating in a sink that has a parallelism of one. The third operator is stateful, +and you can see that a fully-connected network shuffle is occurring between the second and third +operators. This is being done to partition the stream by some key, so that all of the events that +need to be processed together, will be. + +State is sharded + +State is always accessed locally, which helps Flink applications achieve high throughput and +low-latency. You can choose to keep state on the JVM heap, or if it is too large, in efficiently +organized on-disk data structures. + +State is local + +{% top %} + +## Fault Tolerance via State Snapshots + +Flink is able to provide fault-tolerant, exactly-once semantics through a combination of state +snapshots and stream replay. These snapshots capture the entire state of the distributed pipeline, +recording offsets into the input queues as well as the state throughout the job graph that has +resulted from having ingested the data up to that point. When a failure occurs, the sources are +rewound, the state is restored, and processing is resumed. As depicted above, these state snapshots +are captured asynchronously, without impeding the ongoing processing. + +{% top %} diff --git a/docs/tutorials/streaming_analytics.md b/docs/tutorials/streaming_analytics.md new file mode 100644 index 0000000000000000000000000000000000000000..ff4ee9558fb6b5597d561234f24faee91a1734a6 --- /dev/null +++ b/docs/tutorials/streaming_analytics.md @@ -0,0 +1,475 @@ +--- +title: Streaming Analytics +nav-id: analytics +nav-pos: 4 +nav-title: Streaming Analytics +nav-parent_id: tutorials +--- + + +* This will be replaced by the TOC +{:toc} + +## Event Time and Watermarks + +### Introduction + +Flink explicitly supports three different notions of time: + +* _event time:_ the time when an event occurred, as recorded by the device producing (or storing) the event + +* _ingestion time:_ a timestamp recorded by Flink at the moment it ingests the event + +* _processing time:_ the time when a specific operator in your pipeline is processing the event + +For reproducible results, e.g., when computing the maximum price a stock reached during the first +hour of trading on a given day, you should use event time. In this way the result won't depend on +when the calculation is performed. This kind of real-time application is sometimes performed using +processing time, but then the results are determined by the events that happen to be processed +during that hour, rather than the events that occurred then. Computing analytics based on processing +time causes inconsistencies, and makes it difficult to re-analyze historic data or test new +implementations. + +### Working with Event Time + +By default, Flink will use processing time. To change this, you can set the Time Characteristic: + +{% highlight java %} +final StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(); +env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); +{% endhighlight %} + +If you want to use event time, you will also need to supply a Timestamp Extractor and Watermark +Generator that Flink will use to track the progress of event time. This will be covered in the +section below on [Working with Watermarks]({% link +tutorials/streaming_analytics.md %}#working-with-watermarks), but first we should explain what +watermarks are. + +### Watermarks + +Let's work through a simple example that will show why watermarks are needed, and how they work. + +In this example you have a stream of timestamped events that arrive somewhat out of order, as shown +below. The numbers shown are timestamps that indicate when these events actually occurred. The first +event to arrive happened at time 4, and it is followed by an event that happened earlier, at time 2, +and so on: + +
+··· 23 19 22 24 21 14 17 13 12 15 9 11 7 2 4 → +
+ +Now imagine that you are trying create a stream sorter. This is meant to be an application that +processes each event from a stream as it arrives, and emits a new stream containing the same events, +but ordered by their timestamps. + +Some observations: + +(1) The first element your stream sorter sees is the 4, but you can't just immediately release it as +the first element of the sorted stream. It may have arrived out of order, and an earlier event might +yet arrive. In fact, you have the benefit of some god-like knowledge of this stream's future, and +you can see that your stream sorter should wait at least until the 2 arrives before producing any +results. + +*Some buffering, and some delay, is necessary.* + +(2) If you do this wrong, you could end up waiting forever. First the sorter saw an event from time +4, and then an event from time 2. Will an event with a timestamp less than 2 ever arrive? Maybe. +Maybe not. You could wait forever and never see a 1. + +*Eventually you have to be courageous and emit the 2 as the start of the sorted stream.* + +(3) What you need then is some sort of policy that defines when, for any given timestamped event, to +stop waiting for the arrival of earlier events. + +*This is precisely what watermarks do* — they define when to stop waiting for earlier events. + +Event time processing in Flink depends on *watermark generators* that insert special timestamped +elements into the stream, called *watermarks*. A watermark for time _t_ is an assertion that the +stream is (probably) now complete up through time _t_. + +When should this stream sorter stop waiting, and push out the 2 to start the sorted stream? When a +watermark arrives with a timestamp of 2, or greater. + +(4) You might imagine different policies for deciding how to generate watermarks. + +Each event arrives after some delay, and these delays vary, so some events are delayed more than +others. One simple approach is to assume that these delays are bounded by some maximum delay. Flink +refers to this strategy as *bounded-out-of-orderness* watermarking. It is easy to imagine more +complex approaches to watermarking, but for most applications a fixed delay works well enough. + +### Latency vs. Completeness + +Another way to think about watermarks is that they give you, the developer of a streaming +application, control over the tradeoff between latency and completeness. Unlike in batch processing, +where one has the luxury of being able to have complete knowledge of the input before producing any +results, with streaming you must eventually stop waiting to see more of the input, and produce some +sort of result. + +You can either configure your watermarking aggressively, with a short bounded delay, and thereby +take the risk of producing results with rather incomplete knowledge of the input -- i.e., a possibly +wrong result, produced quickly. Or you can wait longer, and produce results that take advantage of +having more complete knowledge of the input stream(s). + +It is also possible to implement hybrid solutions that produce initial results quickly, and then +supply updates to those results as additional (late) data is processed. This is a good approach for +some applications. + +### Lateness + +Lateness is defined relative to the watermarks. A `Watermark(t)` asserts that the stream is complete +up through time _t_; any event following this watermark whose timestamp is ≤ _t_ is late. + +### Working with Watermarks + +In order to perform event-time-based event processing, Flink needs to know the time associated with +each event, and it also needs the stream to include watermarks. + +The Taxi data sources used in the hands-on exercises take care of these details for you. But in your +own applications you will have to take care of this yourself, which is usually done by implementing +a class that extracts the timestamps from the events, and generates watermarks on demand. The +easiest way to do this is by extending the `BoundedOutOfOrdernessTimestampExtractor`: + +{% highlight java %} +DataStream stream = ... + +DataStream withTimestampsAndWatermarks = + stream.assignTimestampsAndWatermarks(new TimestampsAndWatermarks(Time.seconds(10))); + +public static class TimestampsAndWatermarks + extends BoundedOutOfOrdernessTimestampExtractor { + + public TimestampsAndWatermarks(Time t) { + super(t); + } + + @Override + public long extractTimestamp(Event event) { + return event.timestamp; + } +} +{% endhighlight %} + +Note that the constructor for `BoundedOutOfOrdernessTimestampExtractor` takes a parameter which +specifies the maximum expected out-of-orderness (10 seconds, in this example). + +{% top %} + +## Windows + +Flink features very expressive window semantics. + +In this section you will learn: + +* how windows are used to compute aggregates on unbounded streams, +* which types of windows Flink supports, and +* how to implement a DataStream program with a windowed aggregation + +### Introduction + +It is natural when doing stream processing to want to compute aggregated analytics on bounded subsets +of the streams in order to answer questions like these: + +* number of page views per minute +* number of sessions per user per week +* maximum temperature per sensor per minute + +Computing windowed analytics with Flink depends on two principal abstractions: _Window Assigners_ +that assign events to windows (creating new window objects as necessary), and _Window Functions_ +that are applied to the events assigned to a window. + +Flink's windowing API also has notions of _Triggers_, which determine when to call the window +function, and _Evictors_, which can remove elements collected in a window. + +In its basic form, you apply windowing to a keyed stream like this: + +{% highlight java %} +stream. + .keyBy() + .window() + .reduce|aggregate|process() +{% endhighlight %} + +You can also use windowing with non-keyed streams, but keep in mind that in this case, the +processing will _not_ be done in parallel: + +{% highlight java %} +stream. + .windowAll() + .reduce|aggregate|process() +{% endhighlight %} + +### Window Assigners + +Flink has several built-in types of window assigners, which are illustrated below: + +Window assigners + +Some examples of what these window assigners might be used for, and how to specify them: + +* Tumbling time windows + * _page views per minute_ + * `TumblingEventTimeWindows.of(Time.minutes(1))` +* Sliding time windows + * _page views per minute computed every 10 seconds_ + * `SlidingEventTimeWindows.of(Time.minutes(1), Time.seconds(10))` +* Session windows + * _page views per session, where sessions are defined by a gap of at least 30 minutes between sessions_ + * `EventTimeSessionWindows.withGap(Time.minutes(30))` + +Durations can be specified using one of `Time.milliseconds(n)`, `Time.seconds(n)`, `Time.minutes(n)`, `Time.hours(n)`, and `Time.days(n)`. + +The time-based window assigners (including session windows) come in both event time and processing +time flavors. There are significant tradeoffs between these two types of time windows. With +processing time windowing you have to accept these limitations: + +* can not correctly process historic data, +* can not correctly handle out-of-order data, +* results will be non-deterministic, + +but with the advantage of lower latency. + +When working with count-based windows, keep in mind that these windows will not fire until a batch +is complete. There's no option to time-out and process a partial window, though you could implement +that behavior yourself with a custom Trigger. + +A global window assigner assigns every event (with the same key) to the same global window. This is +only useful if you are going to do your own custom windowing, with a custom Trigger. In many cases +where this might seem useful you will be better off using a `ProcessFunction` as described +[in another section]({% link tutorials/event_driven.md %}#process-functions). + +### Window Functions + +You have three basic options for how to process the contents of your windows: + +1. as a batch, using a `ProcessWindowFunction` that will be passed an `Iterable` with the window's contents; +1. incrementally, with a `ReduceFunction` or an `AggregateFunction` that is called as each event is assigned to the window; +1. or with a combination of the two, wherein the pre-aggregated results of a `ReduceFunction` or an `AggregateFunction` are supplied to a `ProcessWindowFunction` when the window is triggered. + +Here are examples of approaches 1 and 3. Each implementation finds the peak value from each sensor +in 1 minute event time windows, and producing a stream of Tuples containing `(key, +end-of-window-timestamp, max_value)`. + +#### ProcessWindowFunction Example + +{% highlight java %} +DataStream input = ... + +input + .keyBy(x -> x.key) + .window(TumblingEventTimeWindows.of(Time.minutes(1))) + .process(new MyWastefulMax()); + +public static class MyWastefulMax extends ProcessWindowFunction< + SensorReading, // input type + Tuple3, // output type + String, // key type + TimeWindow> { // window type + + @Override + public void process( + String key, + Context context, + Iterable events, + Collector> out) { + + int max = 0; + for (SensorReading event : events) { + max = Math.max(event.value, max); + } + out.collect(Tuple3.of(key, context.window().getEnd(), max)); + } +} +{% endhighlight %} + +A couple of things to note in this implementation: + +* All of the events assigned to the window have to be buffered in keyed Flink state until the window + is triggered. This is potentially quite expensive. +* Our `ProcessWindowFunction` is being passed a `Context` object from which contains information about + the window. Its interface looks like this: + +{% highlight java %} +public abstract class Context implements java.io.Serializable { + public abstract W window(); + + public abstract long currentProcessingTime(); + public abstract long currentWatermark(); + + public abstract KeyedStateStore windowState(); + public abstract KeyedStateStore globalState(); +} +{% endhighlight %} + +`windowState` and `globalState` are places where you can store per-key, per-window, or global +per-key information for all windows of that key. This might be useful, for example, if you want to +record something about the +current window and use that when processing a subsequent window. + +#### Incremental Aggregation Example + +{% highlight java %} +DataStream input = ... + +input + .keyBy(x -> x.key) + .window(TumblingEventTimeWindows.of(Time.minutes(1))) + .reduce(new MyReducingMax(), new MyWindowFunction()); + +private static class MyReducingMax implements ReduceFunction { + public SensorReading reduce(SensorReading r1, SensorReading r2) { + return r1.value() > r2.value() ? r1 : r2; + } +} + +private static class MyWindowFunction extends ProcessWindowFunction< + SensorReading, Tuple3, String, TimeWindow> { + + @Override + public void process( + String key, + Context context, + Iterable maxReading, + Collector> out) { + + SensorReading max = maxReading.iterator().next(); + out.collect(Tuple3.of(key, context.window().getEnd(), max)); + } +} +{% endhighlight %} + +Notice that the `Iterable` +will contain exactly one reading -- the pre-aggregated maximum computed by `MyReducingMax`. + +### Late Events + +By default, when using event time windows, late events are dropped. There are two optional parts of +the window API that give you more control over this. + +You can arrange for the events that would be dropped to be collected to an alternate output stream +instead, using a mechanism called +[Side Outputs]({% link tutorials/event_driven.md %}#side-outputs). +Here is an example of what that might look like: + +{% highlight java %} +OutputTag lateTag = new OutputTag("late"){}; + +SingleOutputStreamOperator result = stream. + .keyBy(...) + .window(...) + .sideOutputLateData(lateTag) + .process(...); + +DataStream lateStream = result.getSideOutput(lateTag); +{% endhighlight %} + +You can also specify an interval of _allowed lateness_ during which the late events will continue to +be assigned to the appropriate window(s) (whose state will have been retained). By default each late +event will cause the window function to be called again (sometimes called a _late firing_). + +By default the allowed lateness is 0. In other words, elements behind the watermark are dropped (or +sent to the side output). + +For example: + +{% highlight java %} +stream. + .keyBy(...) + .window(...) + .allowedLateness(Time.seconds(10)) + .process(...); +{% endhighlight %} + +When the allowed lateness is greater than zero, only those events that are so late that they would +be dropped are sent to the side output (if it has been configured). + +### Surprises + +Some aspects of Flink's windowing API may not behave in the way you would expect. Based on +frequently asked questions on the [flink-user mailing +list](https://flink.apache.org/community.html#mailing-lists) and elsewhere, here are some facts +about windows that may surprise you. + +#### Sliding Windows Make Copies + +Sliding window assigners can create lots of window objects, and will copy each event into every +relevant window. For example, if you have sliding windows every 15 minutes that are 24-hours in +length, each event will be copied into 4 * 24 = 96 windows. + +#### Time Windows are Aligned to the Epoch + +Just because you are using hour-long processing-time windows and start your application running at +12:05 does not mean that the first window will close at 1:05. The first window will be 55 minutes +long and close at 1:00. + +Note, however, that the tumbling and sliding window assigners take an optional offset parameter +that can be used to change the alignment of the windows. See +[Tumbling Windows]({% link dev/stream/operators/windows.md %}#tumbling-windows) and +[Sliding Windows]({% link dev/stream/operators/windows.md %}#sliding-windows) for details. + +#### Windows Can Follow Windows + +For example, it works to do this: + +{% highlight java %} +stream + .keyBy(t -> t.key) + .timeWindow(
KeyKey Value
+ +
{{ accumulator.name }} {{ accumulator.type }}{{ accumulator.value }}{{ (accumulator.value | number:'1.0-3' ) || accumulator.value }}
Log NameSize (KB)
+ {{ log.name }} + {{ (log.size / 1024) | number : '1.0-2' }}