Skip to content

Commit

Permalink
[FLINK-25553][filesystem] Remove MapR filesystem
Browse files Browse the repository at this point in the history
  • Loading branch information
MartijnVisser committed Jan 11, 2022
1 parent 7601bd3 commit f56771d
Show file tree
Hide file tree
Showing 19 changed files with 9 additions and 675 deletions.
6 changes: 2 additions & 4 deletions docs/content.zh/docs/deployment/filesystems/overview.md
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@ under the License.

# 文件系统

Apache Flink 使用文件系统来消费和持久化地存储数据,以处理应用结果以及容错与恢复。以下是一些最常用的文件系统:*本地存储**hadoop-compatible**Amazon S3**MapR FS**阿里云 OSS**Azure Blob Storage*
Apache Flink 使用文件系统来消费和持久化地存储数据,以处理应用结果以及容错与恢复。以下是一些最常用的文件系统:*本地存储**hadoop-compatible**Amazon S3**阿里云 OSS**Azure Blob Storage*

文件使用的文件系统通过其 URI Scheme 指定。例如 `file:https:///home/user/text.txt` 表示一个在本地文件系统中的文件,`hdfs:https://namenode:50010/data/user/text.txt` 表示一个在指定 HDFS 集群中的文件。

Expand All @@ -45,15 +45,13 @@ Flink 原生支持本地机器上的文件系统,包括任何挂载到本地
Apache Flink 支持下列文件系统:
- [**Amazon S3**]({{< ref "docs/deployment/filesystems/s3" >}}) 对象存储由 `flink-s3-fs-presto``flink-s3-fs-hadoop` 两种替代实现提供支持。这两种实现都是独立的,没有依赖项。

- **MapR FS** 文件系统适配器已在 Flink 的主发行版中通过 *maprfs:https://* URI Scheme 支持。MapR 库需要在 classpath 中指定(例如在 `lib` 目录中)。

- **[阿里云对象存储]({{< ref "docs/deployment/filesystems/oss" >}})**`flink-oss-fs-hadoop` 支持,并通过 *oss:https://* URI scheme 使用。该实现基于 [Hadoop Project](https://hadoop.apache.org/),但其是独立的,没有依赖项。

- **[Azure Blob Storage]({{< ref "docs/deployment/filesystems/azure" >}})**`flink-azure-fs-hadoop` 支持,并通过 *abfs(s):https://**wasb(s):https://* URI scheme 使用。该实现基于 [Hadoop Project](https://hadoop.apache.org/),但其是独立的,没有依赖项。

- **[Google Cloud Storage]({{< ref "docs/deployment/filesystems/gcs" >}})**`gcs-connector` 支持,并通过 *gs:https://* URI scheme 使用。该实现基于 [Hadoop Project](https://hadoop.apache.org/),但其是独立的,没有依赖项。

**MapR FS** 之外,上述文件系统可以并且需要作为[插件]({{< ref "docs/deployment/filesystems/plugins" >}})使用。
上述文件系统可以并且需要作为[插件]({{< ref "docs/deployment/filesystems/plugins" >}})使用。

使用外部文件系统时,在启动 Flink 之前需将对应的 JAR 文件从 `opt` 目录复制到 Flink 发行版 `plugin` 目录下的某一文件夹中,例如:

Expand Down
2 changes: 1 addition & 1 deletion docs/content.zh/docs/deployment/filesystems/plugins.md
Original file line number Diff line number Diff line change
Expand Up @@ -73,7 +73,7 @@ possible across Flink core, plugins, and user code.

## File Systems

All [file systems]({{< ref "docs/deployment/filesystems/overview" >}}) **except MapR** are pluggable. That means they can and should
All [file systems]({{< ref "docs/deployment/filesystems/overview" >}}) 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.
Expand Down
1 change: 0 additions & 1 deletion docs/content.zh/docs/internals/filesystems.md
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,6 @@ Other file system types are accessed by an implementation that bridges to the su
- `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
Expand Down
7 changes: 2 additions & 5 deletions docs/content/docs/deployment/filesystems/overview.md
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@ under the License.
# File Systems

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*, *Aliyun OSS* and *Azure Blob Storage*.
These are some of most of the popular file systems, including *local*, *hadoop-compatible*, *Amazon S3*, *Aliyun OSS* and *Azure Blob Storage*.

The file system used for a particular file is determined by its URI scheme.
For example, `file:https:///home/user/text.txt` refers to a file in the local file system, while `hdfs:https://namenode:50010/data/user/text.txt` is a file in a specific HDFS cluster.
Expand All @@ -50,9 +50,6 @@ The Apache Flink project supports the following file systems:
- [**Amazon S3**]({{< ref "docs/deployment/filesystems/s3" >}}) 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** file system adapter is already supported in the main Flink distribution under the *maprfs:https://* URI scheme.
You must provide the MapR libraries in the classpath (for example in `lib` directory).

- **[Aliyun Object Storage Service]({{< ref "docs/deployment/filesystems/oss" >}})** is supported by `flink-oss-fs-hadoop` and registered under the *oss:https://* URI scheme.
The implementation is based on the [Hadoop Project](https://hadoop.apache.org/) but is self-contained with no dependency footprint.

Expand All @@ -65,7 +62,7 @@ The Apache Flink project supports the following file systems:
- **[Google Cloud Storage]({{< ref "docs/deployment/filesystems/gcs" >}})** is supported by `gcs-connector` and registered under the *gs:https://* URI scheme.
The implementation is based on the [Hadoop Project](https://hadoop.apache.org/) but is self-contained with no dependency footprint.

Except **MapR FS**, you can and should use any of them as [plugins]({{< ref "docs/deployment/filesystems/plugins" >}}).
You can and should use any of them as [plugins]({{< ref "docs/deployment/filesystems/plugins" >}}).

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.
Expand Down
2 changes: 1 addition & 1 deletion docs/content/docs/deployment/filesystems/plugins.md
Original file line number Diff line number Diff line change
Expand Up @@ -73,7 +73,7 @@ possible across Flink core, plugins, and user code.

## File Systems

All [file systems]({{< ref "docs/deployment/filesystems/overview" >}}) **except MapR** are pluggable. That means they can and should
All [file systems]({{< ref "docs/deployment/filesystems/overview" >}}) 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.
Expand Down
1 change: 0 additions & 1 deletion docs/content/docs/internals/filesystems.md
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,6 @@ Other file system types are accessed by an implementation that bridges to the su
- `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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -255,7 +255,6 @@ public enum WriteMode {
.put("s3", "flink-s3-fs-presto")
.put("s3a", "flink-s3-fs-hadoop")
.put("s3p", "flink-s3-fs-presto")
// mapr deliberately omitted for now (no dedicated plugin)
.build();

/** Exceptions for DIRECTLY_SUPPORTED_FILESYSTEM. */
Expand Down
10 changes: 2 additions & 8 deletions flink-dist/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -126,21 +126,15 @@ under the License.
</exclusions>
</dependency>

<!-- Default file system support. The Hadoop and MapR dependencies -->
<!-- are optional, so not being added to the dist jar -->
<!-- Default file system support. The Hadoop dependency -->
<!-- is optional, so not being added to the dist jar -->

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-hadoop-fs</artifactId>
<version>${project.version}</version>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-mapr-fs</artifactId>
<version>${project.version}</version>
</dependency>

<!-- Concrete logging framework - we add this only here (and not in the
root POM) to not tie the projects to one specific framework and make
it easier for users to swap logging frameworks -->
Expand Down
70 changes: 0 additions & 70 deletions flink-filesystems/flink-mapr-fs/pom.xml

This file was deleted.

Loading

0 comments on commit f56771d

Please sign in to comment.