Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-5788

Document assumptions about File Systems and persistence

    Details

    • Type: Improvement
    • Status: Closed
    • Priority: Major
    • Resolution: Fixed
    • Affects Version/s: None
    • Fix Version/s: 1.3.0
    • Component/s: Documentation
    • Labels:
      None

      Description

      We should add some description about the assumptions we make for the behavior of FileSystem implementations to support proper checkpointing and recovery operations.

      This is especially critical for file systems like S3 with a somewhat tricky contract.

        Issue Links

          Activity

          Hide
          githubbot ASF GitHub Bot added a comment -

          GitHub user StephanEwen opened a pull request:

          https://github.com/apache/flink/pull/3301

          FLINK-5788 [docs] Improve documentation of FileSystem and spell out the data persistence contract

          This writes down the contract that the Flink `FileSystem` and `FSDataOutputStream` implementations have to adhere to in order to support proper consistency and failure recovery. The contract has so far been only implicitly defined and adhered to by the checkpointing and high-availability code.

            1. Contract

          Data written to an `FSDataOutputStream` created from a `FileSystem` is considered persistent, if two requirements are met:

          1. *Visibility Requirement:* It must be guaranteed that all other processes, machines,
          virtual machines, containers, etc. that are able to access the file see the data consistently
          when given the absolute file path. This requirement is similar to the close-to-open
          semantics defined by POSIX, but restricted to the file itself (by its absolute path).

          2. *Durability Requirement:* The file system's specific durability/persistence requirements
          must be met. These are specific to the particular file system. For example the
          `LocalFileSystem` does not provide any durability guarantees for crashes of both
          hardware and operating system, while replicated distributed file systems (like HDFS)
          guarantee typically durability in the presence of up to concurrent failure or n
          nodes, where n is the replication factor.

          Updates to the file's parent directory (such as that the file shows up when listing the directory contents) are not required to be complete for the data in the file stream to be considered persistent. This relaxation is important for file systems where updates to directory contents are only eventually consistent (like S3).

          You can merge this pull request into a Git repository by running:

          $ git pull https://github.com/StephanEwen/incubator-flink filesystem_docs

          Alternatively you can review and apply these changes as the patch at:

          https://github.com/apache/flink/pull/3301.patch

          To close this pull request, make a commit to your master/trunk branch
          with (at least) the following in the commit message:

          This closes #3301



          Show
          githubbot ASF GitHub Bot added a comment - GitHub user StephanEwen opened a pull request: https://github.com/apache/flink/pull/3301 FLINK-5788 [docs] Improve documentation of FileSystem and spell out the data persistence contract This writes down the contract that the Flink `FileSystem` and `FSDataOutputStream` implementations have to adhere to in order to support proper consistency and failure recovery. The contract has so far been only implicitly defined and adhered to by the checkpointing and high-availability code. Contract Data written to an `FSDataOutputStream` created from a `FileSystem` is considered persistent, if two requirements are met: 1. * Visibility Requirement: * It must be guaranteed that all other processes, machines, virtual machines, containers, etc. that are able to access the file see the data consistently when given the absolute file path. This requirement is similar to the close-to-open semantics defined by POSIX, but restricted to the file itself (by its absolute path). 2. * Durability Requirement: * The file system's specific durability/persistence requirements must be met. These are specific to the particular file system. For example the `LocalFileSystem` does not provide any durability guarantees for crashes of both hardware and operating system, while replicated distributed file systems (like HDFS) guarantee typically durability in the presence of up to concurrent failure or n nodes, where n is the replication factor. Updates to the file's parent directory (such as that the file shows up when listing the directory contents) are not required to be complete for the data in the file stream to be considered persistent. This relaxation is important for file systems where updates to directory contents are only eventually consistent (like S3). You can merge this pull request into a Git repository by running: $ git pull https://github.com/StephanEwen/incubator-flink filesystem_docs Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/3301.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #3301
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user alpinegizmo commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3301#discussion_r101010804

          — Diff: docs/internals/filesystems.md —
          @@ -0,0 +1,138 @@
          +---
          +title: "File Systems"
          +nav-parent_id: internals
          +nav-pos: 10
          +---
          +<!--
          +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.
          +-->
          +
          +* Replaced by the TOC
          +{:toc}
          +
          +Flink has its own file system abstraction via the `org.apache.flink.core.fs.FileSystem` class.
          +This abstraction provides a common set of operations and minimal guarantees across various types
          +of file system implementations.
          +
          +The `FileSystem`'s set of available operations is quite limited, in order to suport a wide
          — End diff –

          support

          Show
          githubbot ASF GitHub Bot added a comment - Github user alpinegizmo commented on a diff in the pull request: https://github.com/apache/flink/pull/3301#discussion_r101010804 — Diff: docs/internals/filesystems.md — @@ -0,0 +1,138 @@ +--- +title: "File Systems" +nav-parent_id: internals +nav-pos: 10 +--- +<!-- +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. +--> + +* Replaced by the TOC +{:toc} + +Flink has its own file system abstraction via the `org.apache.flink.core.fs.FileSystem` class. +This abstraction provides a common set of operations and minimal guarantees across various types +of file system implementations. + +The `FileSystem`'s set of available operations is quite limited, in order to suport a wide — End diff – support
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user alpinegizmo commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3301#discussion_r101011921

          — Diff: docs/internals/filesystems.md —
          @@ -0,0 +1,138 @@
          +---
          +title: "File Systems"
          +nav-parent_id: internals
          +nav-pos: 10
          +---
          +<!--
          +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.
          +-->
          +
          +* Replaced by the TOC
          +{:toc}
          +
          +Flink has its own file system abstraction via the `org.apache.flink.core.fs.FileSystem` class.
          +This abstraction provides a common set of operations and minimal guarantees across various types
          +of file system implementations.
          +
          +The `FileSystem`'s set of available operations is quite limited, in order to suport a wide
          +range of file systems. For example, appending to or mutating existing files is not supported.
          +
          +File systems are identified by a file system scheme, such as `file://`, `hdfs://`, etc.
          +
          +# Implementations
          +
          +Flink implements the file systems directly, with the following file system schemes:
          +
          + - `file`, which represents the machines local file system.
          +
          +Other file system types are accessed by an implementation that bridges to the suite of file systems supported by
          +[Apache Hadoop](https://hadoop.apache.org/). The following is an incomplete list of examples:
          +
          + - `hdfs`: Hadoop Distributed File System
          + - `s3`, `s3n`, and `s3a`: Amazon S3 file system
          + - `gcs`: Google Cloud Storage
          + - `maprfs`: The MapR distributed file system
          + - ...
          +
          +Flink loads Hadoop's file systems transparently if it finds the Hadoop File System classes in the class path and finds a valid
          +Hadoop configuration. By default, it looks for the Hadoop configuration in the class path. Alternatively, one can specify a
          +custom location via the configuration entry `fs.hdfs.hadoopconf`.
          +
          +
          +# Persistence Guarantees
          +
          +These `FileSystem` and its `FsDataOutputStream` instances are used to persistently store data, both for results of applications
          +and for fault tolerance and recovery. It is therefore crucial that the persistence semantics of these streams are well defined.
          +
          +## Definition of Persistence Guarantees
          +
          +Data written to an output stream is considered persistent, if two requirements are met:
          +
          + 1. *Visibility Requirement:* It must be guaranteed that all other processes, machines,
          + virtual machines, containers, etc. that are able to access the file see the data consistently
          + when given the absolute file path. This requirement is similar to the close-to-open
          + semantics defined by POSIX, but restricted to the file itself (by its absolute path).
          +
          + 2. *Durability Requirement:* The file system's specific durability/persistence requirements
          + must be met. These are specific to the particular file system. For example the
          +

          {@link LocalFileSystem}

          does not provide any durability guarantees for crashes of both
          + hardware and operating system, while replicated distributed file systems (like HDFS)
          + guarantee typically durability in the presence of up to concurrent failure or n
          + nodes, where n is the replication factor.
          +
          +Updates to the file's parent directory (such as that the file shows up when
          +listing the directory contents) are not required to be complete for the data in the file stream
          +to be considered persistent. This relaxation is important for file systems where updates to
          +directory contents are only eventually consistent.
          +
          +The `FSDataOutputStream` has to guarantee data persistence for the written bytes once the call to
          +`FSDataOutputStream.close()` returns.
          +
          +## Examples
          +
          + - For *fault-tolerant distributed file systems*, data is considered persistent once
          + it has been received and acknowledged by the file system, typically by having been replicated
          + to a quorum of machines (durability requirement). In addition the absolute file path
          + must be visible to all other machines that will potentially access the file (visibility requirement).
          +
          + Whether data has hit non-volatile storage on the storage nodes depends on the specific
          + guarantees of the particular file system.
          +
          + The metadata updates to the file's parent directory are not required to have reached
          + a consistent state. It is permissible that some machines see the file when listing the parent
          + directory's contents while other do not, as long as access to the file by its absolute path
          + is possible on all nodes.
          +
          + - A *local file system* must support the POSIX close-to-open semantics.
          + Because the local file system does not have any fault tolerance guarantees, no further
          + requirements exist.
          +
          + The above implies specifically that data may still be in the OS cache when considered
          + persistent from the local file system's perspective. Crashes that cause the OS cache loose
          + data are considered fatal to the local machine and not covered by the local file system's
          + guarantees as defined by Flink.
          +
          — End diff –

          Crashes that cause the OS cache to lose data are considered fatal to the local machine and are not covered by the local file system's guarantees as defined by Flink.

          Show
          githubbot ASF GitHub Bot added a comment - Github user alpinegizmo commented on a diff in the pull request: https://github.com/apache/flink/pull/3301#discussion_r101011921 — Diff: docs/internals/filesystems.md — @@ -0,0 +1,138 @@ +--- +title: "File Systems" +nav-parent_id: internals +nav-pos: 10 +--- +<!-- +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. +--> + +* Replaced by the TOC +{:toc} + +Flink has its own file system abstraction via the `org.apache.flink.core.fs.FileSystem` class. +This abstraction provides a common set of operations and minimal guarantees across various types +of file system implementations. + +The `FileSystem`'s set of available operations is quite limited, in order to suport a wide +range of file systems. For example, appending to or mutating existing files is not supported. + +File systems are identified by a file system scheme , such as ` file:// `, `hdfs://`, etc. + +# Implementations + +Flink implements the file systems directly, with the following file system schemes: + + - `file`, which represents the machines local file system. + +Other file system types are accessed by an implementation that bridges to the suite of file systems supported by + [Apache Hadoop] ( https://hadoop.apache.org/ ). The following is an incomplete list of examples: + + - `hdfs`: Hadoop Distributed File System + - `s3`, `s3n`, and `s3a`: Amazon S3 file system + - `gcs`: Google Cloud Storage + - `maprfs`: The MapR distributed file system + - ... + +Flink loads Hadoop's file systems transparently if it finds the Hadoop File System classes in the class path and finds a valid +Hadoop configuration. By default, it looks for the Hadoop configuration in the class path. Alternatively, one can specify a +custom location via the configuration entry `fs.hdfs.hadoopconf`. + + +# Persistence Guarantees + +These `FileSystem` and its `FsDataOutputStream` instances are used to persistently store data, both for results of applications +and for fault tolerance and recovery. It is therefore crucial that the persistence semantics of these streams are well defined. + +## Definition of Persistence Guarantees + +Data written to an output stream is considered persistent, if two requirements are met: + + 1. * Visibility Requirement: * It must be guaranteed that all other processes, machines, + virtual machines, containers, etc. that are able to access the file see the data consistently + when given the absolute file path. This requirement is similar to the close-to-open + semantics defined by POSIX, but restricted to the file itself (by its absolute path). + + 2. * Durability Requirement: * The file system's specific durability/persistence requirements + must be met. These are specific to the particular file system. For example the + {@link LocalFileSystem} does not provide any durability guarantees for crashes of both + hardware and operating system, while replicated distributed file systems (like HDFS) + guarantee typically durability in the presence of up to concurrent failure or n + nodes, where n is the replication factor. + +Updates to the file's parent directory (such as that the file shows up when +listing the directory contents) are not required to be complete for the data in the file stream +to be considered persistent. This relaxation is important for file systems where updates to +directory contents are only eventually consistent. + +The `FSDataOutputStream` has to guarantee data persistence for the written bytes once the call to +`FSDataOutputStream.close()` returns. + +## Examples + + - For * fault-tolerant distributed file systems *, data is considered persistent once + it has been received and acknowledged by the file system, typically by having been replicated + to a quorum of machines ( durability requirement ). In addition the absolute file path + must be visible to all other machines that will potentially access the file ( visibility requirement ). + + Whether data has hit non-volatile storage on the storage nodes depends on the specific + guarantees of the particular file system. + + The metadata updates to the file's parent directory are not required to have reached + a consistent state. It is permissible that some machines see the file when listing the parent + directory's contents while other do not, as long as access to the file by its absolute path + is possible on all nodes. + + - A * local file system * must support the POSIX close-to-open semantics. + Because the local file system does not have any fault tolerance guarantees, no further + requirements exist. + + The above implies specifically that data may still be in the OS cache when considered + persistent from the local file system's perspective. Crashes that cause the OS cache loose + data are considered fatal to the local machine and not covered by the local file system's + guarantees as defined by Flink. + — End diff – Crashes that cause the OS cache to lose data are considered fatal to the local machine and are not covered by the local file system's guarantees as defined by Flink.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user alpinegizmo commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3301#discussion_r101011358

          — Diff: docs/internals/filesystems.md —
          @@ -0,0 +1,138 @@
          +---
          +title: "File Systems"
          +nav-parent_id: internals
          +nav-pos: 10
          +---
          +<!--
          +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.
          +-->
          +
          +* Replaced by the TOC
          +{:toc}
          +
          +Flink has its own file system abstraction via the `org.apache.flink.core.fs.FileSystem` class.
          +This abstraction provides a common set of operations and minimal guarantees across various types
          +of file system implementations.
          +
          +The `FileSystem`'s set of available operations is quite limited, in order to suport a wide
          +range of file systems. For example, appending to or mutating existing files is not supported.
          +
          +File systems are identified by a file system scheme, such as `file://`, `hdfs://`, etc.
          +
          +# Implementations
          +
          +Flink implements the file systems directly, with the following file system schemes:
          +
          + - `file`, which represents the machines local file system.
          +
          +Other file system types are accessed by an implementation that bridges to the suite of file systems supported by
          +[Apache Hadoop](https://hadoop.apache.org/). The following is an incomplete list of examples:
          +
          + - `hdfs`: Hadoop Distributed File System
          + - `s3`, `s3n`, and `s3a`: Amazon S3 file system
          + - `gcs`: Google Cloud Storage
          + - `maprfs`: The MapR distributed file system
          + - ...
          +
          +Flink loads Hadoop's file systems transparently if it finds the Hadoop File System classes in the class path and finds a valid
          +Hadoop configuration. By default, it looks for the Hadoop configuration in the class path. Alternatively, one can specify a
          +custom location via the configuration entry `fs.hdfs.hadoopconf`.
          +
          +
          +# Persistence Guarantees
          +
          +These `FileSystem` and its `FsDataOutputStream` instances are used to persistently store data, both for results of applications
          +and for fault tolerance and recovery. It is therefore crucial that the persistence semantics of these streams are well defined.
          +
          +## Definition of Persistence Guarantees
          +
          +Data written to an output stream is considered persistent, if two requirements are met:
          +
          + 1. *Visibility Requirement:* It must be guaranteed that all other processes, machines,
          + virtual machines, containers, etc. that are able to access the file see the data consistently
          + when given the absolute file path. This requirement is similar to the close-to-open
          + semantics defined by POSIX, but restricted to the file itself (by its absolute path).
          +
          + 2. *Durability Requirement:* The file system's specific durability/persistence requirements
          + must be met. These are specific to the particular file system. For example the
          +

          {@link LocalFileSystem}

          does not provide any durability guarantees for crashes of both
          + hardware and operating system, while replicated distributed file systems (like HDFS)
          + guarantee typically durability in the presence of up to concurrent failure or n
          — End diff –

          typically guarantee durability in the presence of at most n concurrent node failures,

          Show
          githubbot ASF GitHub Bot added a comment - Github user alpinegizmo commented on a diff in the pull request: https://github.com/apache/flink/pull/3301#discussion_r101011358 — Diff: docs/internals/filesystems.md — @@ -0,0 +1,138 @@ +--- +title: "File Systems" +nav-parent_id: internals +nav-pos: 10 +--- +<!-- +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. +--> + +* Replaced by the TOC +{:toc} + +Flink has its own file system abstraction via the `org.apache.flink.core.fs.FileSystem` class. +This abstraction provides a common set of operations and minimal guarantees across various types +of file system implementations. + +The `FileSystem`'s set of available operations is quite limited, in order to suport a wide +range of file systems. For example, appending to or mutating existing files is not supported. + +File systems are identified by a file system scheme , such as ` file:// `, `hdfs://`, etc. + +# Implementations + +Flink implements the file systems directly, with the following file system schemes: + + - `file`, which represents the machines local file system. + +Other file system types are accessed by an implementation that bridges to the suite of file systems supported by + [Apache Hadoop] ( https://hadoop.apache.org/ ). The following is an incomplete list of examples: + + - `hdfs`: Hadoop Distributed File System + - `s3`, `s3n`, and `s3a`: Amazon S3 file system + - `gcs`: Google Cloud Storage + - `maprfs`: The MapR distributed file system + - ... + +Flink loads Hadoop's file systems transparently if it finds the Hadoop File System classes in the class path and finds a valid +Hadoop configuration. By default, it looks for the Hadoop configuration in the class path. Alternatively, one can specify a +custom location via the configuration entry `fs.hdfs.hadoopconf`. + + +# Persistence Guarantees + +These `FileSystem` and its `FsDataOutputStream` instances are used to persistently store data, both for results of applications +and for fault tolerance and recovery. It is therefore crucial that the persistence semantics of these streams are well defined. + +## Definition of Persistence Guarantees + +Data written to an output stream is considered persistent, if two requirements are met: + + 1. * Visibility Requirement: * It must be guaranteed that all other processes, machines, + virtual machines, containers, etc. that are able to access the file see the data consistently + when given the absolute file path. This requirement is similar to the close-to-open + semantics defined by POSIX, but restricted to the file itself (by its absolute path). + + 2. * Durability Requirement: * The file system's specific durability/persistence requirements + must be met. These are specific to the particular file system. For example the + {@link LocalFileSystem} does not provide any durability guarantees for crashes of both + hardware and operating system, while replicated distributed file systems (like HDFS) + guarantee typically durability in the presence of up to concurrent failure or n — End diff – typically guarantee durability in the presence of at most n concurrent node failures,
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user alpinegizmo commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3301#discussion_r101012059

          — Diff: docs/internals/filesystems.md —
          @@ -0,0 +1,138 @@
          +---
          +title: "File Systems"
          +nav-parent_id: internals
          +nav-pos: 10
          +---
          +<!--
          +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.
          +-->
          +
          +* Replaced by the TOC
          +{:toc}
          +
          +Flink has its own file system abstraction via the `org.apache.flink.core.fs.FileSystem` class.
          +This abstraction provides a common set of operations and minimal guarantees across various types
          +of file system implementations.
          +
          +The `FileSystem`'s set of available operations is quite limited, in order to suport a wide
          +range of file systems. For example, appending to or mutating existing files is not supported.
          +
          +File systems are identified by a file system scheme, such as `file://`, `hdfs://`, etc.
          +
          +# Implementations
          +
          +Flink implements the file systems directly, with the following file system schemes:
          +
          + - `file`, which represents the machines local file system.
          +
          +Other file system types are accessed by an implementation that bridges to the suite of file systems supported by
          +[Apache Hadoop](https://hadoop.apache.org/). The following is an incomplete list of examples:
          +
          + - `hdfs`: Hadoop Distributed File System
          + - `s3`, `s3n`, and `s3a`: Amazon S3 file system
          + - `gcs`: Google Cloud Storage
          + - `maprfs`: The MapR distributed file system
          + - ...
          +
          +Flink loads Hadoop's file systems transparently if it finds the Hadoop File System classes in the class path and finds a valid
          +Hadoop configuration. By default, it looks for the Hadoop configuration in the class path. Alternatively, one can specify a
          +custom location via the configuration entry `fs.hdfs.hadoopconf`.
          +
          +
          +# Persistence Guarantees
          +
          +These `FileSystem` and its `FsDataOutputStream` instances are used to persistently store data, both for results of applications
          +and for fault tolerance and recovery. It is therefore crucial that the persistence semantics of these streams are well defined.
          +
          +## Definition of Persistence Guarantees
          +
          +Data written to an output stream is considered persistent, if two requirements are met:
          +
          + 1. *Visibility Requirement:* It must be guaranteed that all other processes, machines,
          + virtual machines, containers, etc. that are able to access the file see the data consistently
          + when given the absolute file path. This requirement is similar to the close-to-open
          + semantics defined by POSIX, but restricted to the file itself (by its absolute path).
          +
          + 2. *Durability Requirement:* The file system's specific durability/persistence requirements
          + must be met. These are specific to the particular file system. For example the
          +

          {@link LocalFileSystem}

          does not provide any durability guarantees for crashes of both
          + hardware and operating system, while replicated distributed file systems (like HDFS)
          + guarantee typically durability in the presence of up to concurrent failure or n
          + nodes, where n is the replication factor.
          +
          +Updates to the file's parent directory (such as that the file shows up when
          +listing the directory contents) are not required to be complete for the data in the file stream
          +to be considered persistent. This relaxation is important for file systems where updates to
          +directory contents are only eventually consistent.
          +
          +The `FSDataOutputStream` has to guarantee data persistence for the written bytes once the call to
          +`FSDataOutputStream.close()` returns.
          +
          +## Examples
          +
          + - For *fault-tolerant distributed file systems*, data is considered persistent once
          + it has been received and acknowledged by the file system, typically by having been replicated
          + to a quorum of machines (durability requirement). In addition the absolute file path
          + must be visible to all other machines that will potentially access the file (visibility requirement).
          +
          + Whether data has hit non-volatile storage on the storage nodes depends on the specific
          + guarantees of the particular file system.
          +
          + The metadata updates to the file's parent directory are not required to have reached
          + a consistent state. It is permissible that some machines see the file when listing the parent
          + directory's contents while other do not, as long as access to the file by its absolute path
          + is possible on all nodes.
          +
          + - A *local file system* must support the POSIX close-to-open semantics.
          + Because the local file system does not have any fault tolerance guarantees, no further
          + requirements exist.
          +
          + The above implies specifically that data may still be in the OS cache when considered
          + persistent from the local file system's perspective. Crashes that cause the OS cache loose
          + data are considered fatal to the local machine and not covered by the local file system's
          + guarantees as defined by Flink.
          +
          + That means that computed results, checkpoints, and savepoints that are written only to
          + the local filesystem are not guaranteed to be recoverable from the local machine's failure,
          + making local file systems unsuitable for production setups.
          +
          +# Updating File Contents
          +
          +Many file systems either do not support overwriting contents of existing files at all, or do not support consistent visibility of the
          +updated contents in that case. For that reason, Flink's FileSystem does not support appending to existing files, or seeking output streams
          — End diff –

          or seeking within output streams

          Show
          githubbot ASF GitHub Bot added a comment - Github user alpinegizmo commented on a diff in the pull request: https://github.com/apache/flink/pull/3301#discussion_r101012059 — Diff: docs/internals/filesystems.md — @@ -0,0 +1,138 @@ +--- +title: "File Systems" +nav-parent_id: internals +nav-pos: 10 +--- +<!-- +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. +--> + +* Replaced by the TOC +{:toc} + +Flink has its own file system abstraction via the `org.apache.flink.core.fs.FileSystem` class. +This abstraction provides a common set of operations and minimal guarantees across various types +of file system implementations. + +The `FileSystem`'s set of available operations is quite limited, in order to suport a wide +range of file systems. For example, appending to or mutating existing files is not supported. + +File systems are identified by a file system scheme , such as ` file:// `, `hdfs://`, etc. + +# Implementations + +Flink implements the file systems directly, with the following file system schemes: + + - `file`, which represents the machines local file system. + +Other file system types are accessed by an implementation that bridges to the suite of file systems supported by + [Apache Hadoop] ( https://hadoop.apache.org/ ). The following is an incomplete list of examples: + + - `hdfs`: Hadoop Distributed File System + - `s3`, `s3n`, and `s3a`: Amazon S3 file system + - `gcs`: Google Cloud Storage + - `maprfs`: The MapR distributed file system + - ... + +Flink loads Hadoop's file systems transparently if it finds the Hadoop File System classes in the class path and finds a valid +Hadoop configuration. By default, it looks for the Hadoop configuration in the class path. Alternatively, one can specify a +custom location via the configuration entry `fs.hdfs.hadoopconf`. + + +# Persistence Guarantees + +These `FileSystem` and its `FsDataOutputStream` instances are used to persistently store data, both for results of applications +and for fault tolerance and recovery. It is therefore crucial that the persistence semantics of these streams are well defined. + +## Definition of Persistence Guarantees + +Data written to an output stream is considered persistent, if two requirements are met: + + 1. * Visibility Requirement: * It must be guaranteed that all other processes, machines, + virtual machines, containers, etc. that are able to access the file see the data consistently + when given the absolute file path. This requirement is similar to the close-to-open + semantics defined by POSIX, but restricted to the file itself (by its absolute path). + + 2. * Durability Requirement: * The file system's specific durability/persistence requirements + must be met. These are specific to the particular file system. For example the + {@link LocalFileSystem} does not provide any durability guarantees for crashes of both + hardware and operating system, while replicated distributed file systems (like HDFS) + guarantee typically durability in the presence of up to concurrent failure or n + nodes, where n is the replication factor. + +Updates to the file's parent directory (such as that the file shows up when +listing the directory contents) are not required to be complete for the data in the file stream +to be considered persistent. This relaxation is important for file systems where updates to +directory contents are only eventually consistent. + +The `FSDataOutputStream` has to guarantee data persistence for the written bytes once the call to +`FSDataOutputStream.close()` returns. + +## Examples + + - For * fault-tolerant distributed file systems *, data is considered persistent once + it has been received and acknowledged by the file system, typically by having been replicated + to a quorum of machines ( durability requirement ). In addition the absolute file path + must be visible to all other machines that will potentially access the file ( visibility requirement ). + + Whether data has hit non-volatile storage on the storage nodes depends on the specific + guarantees of the particular file system. + + The metadata updates to the file's parent directory are not required to have reached + a consistent state. It is permissible that some machines see the file when listing the parent + directory's contents while other do not, as long as access to the file by its absolute path + is possible on all nodes. + + - A * local file system * must support the POSIX close-to-open semantics. + Because the local file system does not have any fault tolerance guarantees, no further + requirements exist. + + The above implies specifically that data may still be in the OS cache when considered + persistent from the local file system's perspective. Crashes that cause the OS cache loose + data are considered fatal to the local machine and not covered by the local file system's + guarantees as defined by Flink. + + That means that computed results, checkpoints, and savepoints that are written only to + the local filesystem are not guaranteed to be recoverable from the local machine's failure, + making local file systems unsuitable for production setups. + +# Updating File Contents + +Many file systems either do not support overwriting contents of existing files at all, or do not support consistent visibility of the +updated contents in that case. For that reason, Flink's FileSystem does not support appending to existing files, or seeking output streams — End diff – or seeking within output streams
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user alpinegizmo commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3301#discussion_r101011563

          — Diff: docs/internals/filesystems.md —
          @@ -0,0 +1,138 @@
          +---
          +title: "File Systems"
          +nav-parent_id: internals
          +nav-pos: 10
          +---
          +<!--
          +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.
          +-->
          +
          +* Replaced by the TOC
          +{:toc}
          +
          +Flink has its own file system abstraction via the `org.apache.flink.core.fs.FileSystem` class.
          +This abstraction provides a common set of operations and minimal guarantees across various types
          +of file system implementations.
          +
          +The `FileSystem`'s set of available operations is quite limited, in order to suport a wide
          +range of file systems. For example, appending to or mutating existing files is not supported.
          +
          +File systems are identified by a file system scheme, such as `file://`, `hdfs://`, etc.
          +
          +# Implementations
          +
          +Flink implements the file systems directly, with the following file system schemes:
          +
          + - `file`, which represents the machines local file system.
          +
          +Other file system types are accessed by an implementation that bridges to the suite of file systems supported by
          +[Apache Hadoop](https://hadoop.apache.org/). The following is an incomplete list of examples:
          +
          + - `hdfs`: Hadoop Distributed File System
          + - `s3`, `s3n`, and `s3a`: Amazon S3 file system
          + - `gcs`: Google Cloud Storage
          + - `maprfs`: The MapR distributed file system
          + - ...
          +
          +Flink loads Hadoop's file systems transparently if it finds the Hadoop File System classes in the class path and finds a valid
          +Hadoop configuration. By default, it looks for the Hadoop configuration in the class path. Alternatively, one can specify a
          +custom location via the configuration entry `fs.hdfs.hadoopconf`.
          +
          +
          +# Persistence Guarantees
          +
          +These `FileSystem` and its `FsDataOutputStream` instances are used to persistently store data, both for results of applications
          +and for fault tolerance and recovery. It is therefore crucial that the persistence semantics of these streams are well defined.
          +
          +## Definition of Persistence Guarantees
          +
          +Data written to an output stream is considered persistent, if two requirements are met:
          +
          + 1. *Visibility Requirement:* It must be guaranteed that all other processes, machines,
          + virtual machines, containers, etc. that are able to access the file see the data consistently
          + when given the absolute file path. This requirement is similar to the close-to-open
          + semantics defined by POSIX, but restricted to the file itself (by its absolute path).
          +
          + 2. *Durability Requirement:* The file system's specific durability/persistence requirements
          + must be met. These are specific to the particular file system. For example the
          +

          {@link LocalFileSystem}

          does not provide any durability guarantees for crashes of both
          + hardware and operating system, while replicated distributed file systems (like HDFS)
          + guarantee typically durability in the presence of up to concurrent failure or n
          + nodes, where n is the replication factor.
          +
          +Updates to the file's parent directory (such as that the file shows up when
          — End diff –

          (such that the file ...)

          Show
          githubbot ASF GitHub Bot added a comment - Github user alpinegizmo commented on a diff in the pull request: https://github.com/apache/flink/pull/3301#discussion_r101011563 — Diff: docs/internals/filesystems.md — @@ -0,0 +1,138 @@ +--- +title: "File Systems" +nav-parent_id: internals +nav-pos: 10 +--- +<!-- +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. +--> + +* Replaced by the TOC +{:toc} + +Flink has its own file system abstraction via the `org.apache.flink.core.fs.FileSystem` class. +This abstraction provides a common set of operations and minimal guarantees across various types +of file system implementations. + +The `FileSystem`'s set of available operations is quite limited, in order to suport a wide +range of file systems. For example, appending to or mutating existing files is not supported. + +File systems are identified by a file system scheme , such as ` file:// `, `hdfs://`, etc. + +# Implementations + +Flink implements the file systems directly, with the following file system schemes: + + - `file`, which represents the machines local file system. + +Other file system types are accessed by an implementation that bridges to the suite of file systems supported by + [Apache Hadoop] ( https://hadoop.apache.org/ ). The following is an incomplete list of examples: + + - `hdfs`: Hadoop Distributed File System + - `s3`, `s3n`, and `s3a`: Amazon S3 file system + - `gcs`: Google Cloud Storage + - `maprfs`: The MapR distributed file system + - ... + +Flink loads Hadoop's file systems transparently if it finds the Hadoop File System classes in the class path and finds a valid +Hadoop configuration. By default, it looks for the Hadoop configuration in the class path. Alternatively, one can specify a +custom location via the configuration entry `fs.hdfs.hadoopconf`. + + +# Persistence Guarantees + +These `FileSystem` and its `FsDataOutputStream` instances are used to persistently store data, both for results of applications +and for fault tolerance and recovery. It is therefore crucial that the persistence semantics of these streams are well defined. + +## Definition of Persistence Guarantees + +Data written to an output stream is considered persistent, if two requirements are met: + + 1. * Visibility Requirement: * It must be guaranteed that all other processes, machines, + virtual machines, containers, etc. that are able to access the file see the data consistently + when given the absolute file path. This requirement is similar to the close-to-open + semantics defined by POSIX, but restricted to the file itself (by its absolute path). + + 2. * Durability Requirement: * The file system's specific durability/persistence requirements + must be met. These are specific to the particular file system. For example the + {@link LocalFileSystem} does not provide any durability guarantees for crashes of both + hardware and operating system, while replicated distributed file systems (like HDFS) + guarantee typically durability in the presence of up to concurrent failure or n + nodes, where n is the replication factor. + +Updates to the file's parent directory (such as that the file shows up when — End diff – (such that the file ...)
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user alpinegizmo commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3301#discussion_r101010905

          — Diff: docs/internals/filesystems.md —
          @@ -0,0 +1,138 @@
          +---
          +title: "File Systems"
          +nav-parent_id: internals
          +nav-pos: 10
          +---
          +<!--
          +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.
          +-->
          +
          +* Replaced by the TOC
          +{:toc}
          +
          +Flink has its own file system abstraction via the `org.apache.flink.core.fs.FileSystem` class.
          +This abstraction provides a common set of operations and minimal guarantees across various types
          +of file system implementations.
          +
          +The `FileSystem`'s set of available operations is quite limited, in order to suport a wide
          +range of file systems. For example, appending to or mutating existing files is not supported.
          +
          +File systems are identified by a file system scheme, such as `file://`, `hdfs://`, etc.
          +
          +# Implementations
          +
          +Flink implements the file systems directly, with the following file system schemes:
          +
          + - `file`, which represents the machines local file system.
          — End diff –

          machine's

          Show
          githubbot ASF GitHub Bot added a comment - Github user alpinegizmo commented on a diff in the pull request: https://github.com/apache/flink/pull/3301#discussion_r101010905 — Diff: docs/internals/filesystems.md — @@ -0,0 +1,138 @@ +--- +title: "File Systems" +nav-parent_id: internals +nav-pos: 10 +--- +<!-- +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. +--> + +* Replaced by the TOC +{:toc} + +Flink has its own file system abstraction via the `org.apache.flink.core.fs.FileSystem` class. +This abstraction provides a common set of operations and minimal guarantees across various types +of file system implementations. + +The `FileSystem`'s set of available operations is quite limited, in order to suport a wide +range of file systems. For example, appending to or mutating existing files is not supported. + +File systems are identified by a file system scheme , such as ` file:// `, `hdfs://`, etc. + +# Implementations + +Flink implements the file systems directly, with the following file system schemes: + + - `file`, which represents the machines local file system. — End diff – machine's
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user alpinegizmo commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3301#discussion_r101011728

          — Diff: docs/internals/filesystems.md —
          @@ -0,0 +1,138 @@
          +---
          +title: "File Systems"
          +nav-parent_id: internals
          +nav-pos: 10
          +---
          +<!--
          +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.
          +-->
          +
          +* Replaced by the TOC
          +{:toc}
          +
          +Flink has its own file system abstraction via the `org.apache.flink.core.fs.FileSystem` class.
          +This abstraction provides a common set of operations and minimal guarantees across various types
          +of file system implementations.
          +
          +The `FileSystem`'s set of available operations is quite limited, in order to suport a wide
          +range of file systems. For example, appending to or mutating existing files is not supported.
          +
          +File systems are identified by a file system scheme, such as `file://`, `hdfs://`, etc.
          +
          +# Implementations
          +
          +Flink implements the file systems directly, with the following file system schemes:
          +
          + - `file`, which represents the machines local file system.
          +
          +Other file system types are accessed by an implementation that bridges to the suite of file systems supported by
          +[Apache Hadoop](https://hadoop.apache.org/). The following is an incomplete list of examples:
          +
          + - `hdfs`: Hadoop Distributed File System
          + - `s3`, `s3n`, and `s3a`: Amazon S3 file system
          + - `gcs`: Google Cloud Storage
          + - `maprfs`: The MapR distributed file system
          + - ...
          +
          +Flink loads Hadoop's file systems transparently if it finds the Hadoop File System classes in the class path and finds a valid
          +Hadoop configuration. By default, it looks for the Hadoop configuration in the class path. Alternatively, one can specify a
          +custom location via the configuration entry `fs.hdfs.hadoopconf`.
          +
          +
          +# Persistence Guarantees
          +
          +These `FileSystem` and its `FsDataOutputStream` instances are used to persistently store data, both for results of applications
          +and for fault tolerance and recovery. It is therefore crucial that the persistence semantics of these streams are well defined.
          +
          +## Definition of Persistence Guarantees
          +
          +Data written to an output stream is considered persistent, if two requirements are met:
          +
          + 1. *Visibility Requirement:* It must be guaranteed that all other processes, machines,
          + virtual machines, containers, etc. that are able to access the file see the data consistently
          + when given the absolute file path. This requirement is similar to the close-to-open
          + semantics defined by POSIX, but restricted to the file itself (by its absolute path).
          +
          + 2. *Durability Requirement:* The file system's specific durability/persistence requirements
          + must be met. These are specific to the particular file system. For example the
          +

          {@link LocalFileSystem}

          does not provide any durability guarantees for crashes of both
          + hardware and operating system, while replicated distributed file systems (like HDFS)
          + guarantee typically durability in the presence of up to concurrent failure or n
          + nodes, where n is the replication factor.
          +
          +Updates to the file's parent directory (such as that the file shows up when
          +listing the directory contents) are not required to be complete for the data in the file stream
          +to be considered persistent. This relaxation is important for file systems where updates to
          +directory contents are only eventually consistent.
          +
          +The `FSDataOutputStream` has to guarantee data persistence for the written bytes once the call to
          +`FSDataOutputStream.close()` returns.
          +
          +## Examples
          +
          + - For *fault-tolerant distributed file systems*, data is considered persistent once
          + it has been received and acknowledged by the file system, typically by having been replicated
          + to a quorum of machines (durability requirement). In addition the absolute file path
          + must be visible to all other machines that will potentially access the file (visibility requirement).
          +
          + Whether data has hit non-volatile storage on the storage nodes depends on the specific
          + guarantees of the particular file system.
          +
          + The metadata updates to the file's parent directory are not required to have reached
          + a consistent state. It is permissible that some machines see the file when listing the parent
          + directory's contents while other do not, as long as access to the file by its absolute path
          — End diff –

          while others do not

          Show
          githubbot ASF GitHub Bot added a comment - Github user alpinegizmo commented on a diff in the pull request: https://github.com/apache/flink/pull/3301#discussion_r101011728 — Diff: docs/internals/filesystems.md — @@ -0,0 +1,138 @@ +--- +title: "File Systems" +nav-parent_id: internals +nav-pos: 10 +--- +<!-- +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. +--> + +* Replaced by the TOC +{:toc} + +Flink has its own file system abstraction via the `org.apache.flink.core.fs.FileSystem` class. +This abstraction provides a common set of operations and minimal guarantees across various types +of file system implementations. + +The `FileSystem`'s set of available operations is quite limited, in order to suport a wide +range of file systems. For example, appending to or mutating existing files is not supported. + +File systems are identified by a file system scheme , such as ` file:// `, `hdfs://`, etc. + +# Implementations + +Flink implements the file systems directly, with the following file system schemes: + + - `file`, which represents the machines local file system. + +Other file system types are accessed by an implementation that bridges to the suite of file systems supported by + [Apache Hadoop] ( https://hadoop.apache.org/ ). The following is an incomplete list of examples: + + - `hdfs`: Hadoop Distributed File System + - `s3`, `s3n`, and `s3a`: Amazon S3 file system + - `gcs`: Google Cloud Storage + - `maprfs`: The MapR distributed file system + - ... + +Flink loads Hadoop's file systems transparently if it finds the Hadoop File System classes in the class path and finds a valid +Hadoop configuration. By default, it looks for the Hadoop configuration in the class path. Alternatively, one can specify a +custom location via the configuration entry `fs.hdfs.hadoopconf`. + + +# Persistence Guarantees + +These `FileSystem` and its `FsDataOutputStream` instances are used to persistently store data, both for results of applications +and for fault tolerance and recovery. It is therefore crucial that the persistence semantics of these streams are well defined. + +## Definition of Persistence Guarantees + +Data written to an output stream is considered persistent, if two requirements are met: + + 1. * Visibility Requirement: * It must be guaranteed that all other processes, machines, + virtual machines, containers, etc. that are able to access the file see the data consistently + when given the absolute file path. This requirement is similar to the close-to-open + semantics defined by POSIX, but restricted to the file itself (by its absolute path). + + 2. * Durability Requirement: * The file system's specific durability/persistence requirements + must be met. These are specific to the particular file system. For example the + {@link LocalFileSystem} does not provide any durability guarantees for crashes of both + hardware and operating system, while replicated distributed file systems (like HDFS) + guarantee typically durability in the presence of up to concurrent failure or n + nodes, where n is the replication factor. + +Updates to the file's parent directory (such as that the file shows up when +listing the directory contents) are not required to be complete for the data in the file stream +to be considered persistent. This relaxation is important for file systems where updates to +directory contents are only eventually consistent. + +The `FSDataOutputStream` has to guarantee data persistence for the written bytes once the call to +`FSDataOutputStream.close()` returns. + +## Examples + + - For * fault-tolerant distributed file systems *, data is considered persistent once + it has been received and acknowledged by the file system, typically by having been replicated + to a quorum of machines ( durability requirement ). In addition the absolute file path + must be visible to all other machines that will potentially access the file ( visibility requirement ). + + Whether data has hit non-volatile storage on the storage nodes depends on the specific + guarantees of the particular file system. + + The metadata updates to the file's parent directory are not required to have reached + a consistent state. It is permissible that some machines see the file when listing the parent + directory's contents while other do not, as long as access to the file by its absolute path — End diff – while others do not
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user StephanEwen commented on the issue:

          https://github.com/apache/flink/pull/3301

          Thanks for the review. Merging this...

          Show
          githubbot ASF GitHub Bot added a comment - Github user StephanEwen commented on the issue: https://github.com/apache/flink/pull/3301 Thanks for the review. Merging this...
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user asfgit closed the pull request at:

          https://github.com/apache/flink/pull/3301

          Show
          githubbot ASF GitHub Bot added a comment - Github user asfgit closed the pull request at: https://github.com/apache/flink/pull/3301
          Hide
          StephanEwen Stephan Ewen added a comment -

          Fixed via f7af3b01681592787db16a555b55d6b11d35f869

          Show
          StephanEwen Stephan Ewen added a comment - Fixed via f7af3b01681592787db16a555b55d6b11d35f869

            People

            • Assignee:
              StephanEwen Stephan Ewen
              Reporter:
              StephanEwen Stephan Ewen
            • Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development