文件系统

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 typesof file system implementations.

The FileSystem’s set of available operations is quite limited, in order to support a widerange 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 machine’s local file system.

Other file system types are accessed by an implementation that bridges to the suite of file systems supported byApache Hadoop. 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 validHadoop configuration. By default, it looks for the Hadoop configuration in the class path. Alternatively, one can specify acustom 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 applicationsand 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:

  • 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 consistentlywhen 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).

  • Durability Requirement: The file system’s specific durability/persistence requirementsmust be met. These are specific to the particular file system. For example the{@link LocalFileSystem} does not provide any durability guarantees for crashes of bothhardware and operating system, while replicated distributed file systems (like HDFS)guarantee typically durability in the presence of up n concurrent node failures,where n is the replication factor.

Updates to the file’s parent directory (such that the file shows up whenlisting the directory contents) are not required to be complete for the data in the file streamto be considered persistent. This relaxation is important for file systems where updates todirectory contents are only eventually consistent.

The FSDataOutputStream has to guarantee data persistence for the written bytes once the call toFSDataOutputStream.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 replicatedto a quorum of machines (durability requirement). In addition the absolute file pathmust 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 specificguarantees of the particular file system.

The metadata updates to the file’s parent directory are not required to have reacheda consistent state. It is permissible that some machines see the file when listing the parentdirectory’s contents while others do not, as long as access to the file by its absolute pathis 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 furtherrequirements exist.

The above implies specifically that data may still be in the OS cache when consideredpersistent from the local file system’s perspective. Crashes that cause the OS cache to loosedata are considered fatal to the local machine and are not covered by the local file system’sguarantees as defined by Flink.

That means that computed results, checkpoints, and savepoints that are written only tothe 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 theupdated contents in that case. For that reason, Flink’s FileSystem does not support appending to existing files, or seeking withinoutput streams such that previously written data could be changed within the same file.

Overwriting Files

Overwriting files is in general possible. A file is overwritten by deleting it and creating a new file.However, certain filesystems cannot make that change synchronously visible to all parties that have access to the file.For example Amazon S3 guarantees only eventual consistency inthe visibility of the file replacement: Some machines may see the old file, some machines may see the new file.

To avoid these consistency issues, the implementations of failure/recovery mechanisms in Flink strictly avoid writing tothe same file path more than once.

Thread Safety

Implementations of FileSystem must be thread-safe: The same instance of FileSystem is frequently shared across multiple threadsin Flink and must be able to concurrently create input/output streams and list file metadata.

The FSDataOutputStream and FSDataOutputStream implementations are strictly not thread-safe.Instances of the streams should also not be passed between threads in between read or write operations, because there are no guaranteesabout the visibility of operations across threads (many operations do not create memory fences).