[
https://issues.apache.org/jira/browse/FLINK-5788?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15865607#comment-15865607
]
ASF GitHub Bot commented on FLINK-5788:
---------------------------------------
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 ...)
> Document assumptions about File Systems and persistence
> -------------------------------------------------------
>
> Key: FLINK-5788
> URL: https://issues.apache.org/jira/browse/FLINK-5788
> Project: Flink
> Issue Type: Improvement
> Components: Documentation
> Reporter: Stephan Ewen
> Assignee: Stephan Ewen
> Fix For: 1.3.0
>
>
> 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.
--
This message was sent by Atlassian JIRA
(v6.3.15#6346)