Skip to content

Commit

Permalink
[FLINK-1266] Generalize DistributedFileSystem implementation
Browse files Browse the repository at this point in the history
to HadoopFileSystem wrapper, which supports all subclasses of org.apache.hadoop.fs.FileSystem.
This allows us to let users use all file systems with support for HDFS.
The change has been tested with Tachyon, Google Cloud Storage Hadoop Adapter and HDFS.

The change also cleans up the Hadoop dependency exclusions.
  • Loading branch information
rmetzger committed Jan 8, 2015
1 parent d8dbaee commit f290929
Show file tree
Hide file tree
Showing 23 changed files with 926 additions and 391 deletions.
52 changes: 47 additions & 5 deletions docs/example_connectors.md
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
---
title: "Example: Connectors"
title: "Connecting to other systems"
---
<!--
Licensed to the Apache Software Foundation (ASF) under one
Expand All @@ -20,14 +20,56 @@ specific language governing permissions and limitations
under the License.
-->

Apache Flink allows users to access many different systems as data sources or sinks. The system is designed for very easy extensibility. Similar to Apache Hadoop, Flink has the concept of so called `InputFormat`s and `OutputFormat`s.
## Reading from filesystems.

One implementation of these `InputFormat`s is the `HadoopInputFormat`. This is a wrapper that allows users to use all existing Hadoop input formats with Flink.
Flink has build-in support for the following file systems:

This page shows some examples for connecting Flink to other systems.
| Filesystem | Since | Scheme | Notes |
| ------------- |-------------| -----| ------ |
| Hadoop Distributed File System (HDFS) | 0.2 | `hdfs:https://`| All HDFS versions are supported |
| Amazon S3 | 0.2 | `s3:https://` | |
| MapR file system | 0.7-incubating | `maprfs:https://` | The user has to manually place the required jar files in the `lib/` dir |
| Tachyon | 0.9 | `tachyon:https://` | Support through Hadoop file system implementation (see below) |


## Access Microsoft Azure Table Storage

### Using Hadoop file systems with Apache Flink

Apache Flink allows users to use any file system implementing the `org.apache.hadoop.fs.FileSystem`
interface. Hadoop ships adapters for FTP, [Hftp](http:https://hadoop.apache.org/docs/r1.2.1/hftp.html), and others.

Flink has integrated testcases to validate the integration with [Tachyon](http:https://tachyon-project.org/).
Other file systems we tested the integration is the
[Google Cloud Storage Connector for Hadoop](https://cloud.google.com/hadoop/google-cloud-storage-connector).

In order to use a Hadoop file system with Flink, make sure that the `flink-conf.yaml` has set the
`fs.hdfs.hadoopconf` property set to the Hadoop configuration directory.
In addition to that, the Hadoop configuration (in that directory) needs to have an entry for each supported file system.
For example for tachyon support, there must be the following entry in the `core-site.xml` file:

~~~xml
<property>
<name>fs.tachyon.impl</name>
<value>tachyon.hadoop.TFS</value>
</property>
~~~



## Connecting to other systems using Input / Output Format wrappers for Hadoop

Apache Flink allows users to access many different systems as data sources or sinks.
The system is designed for very easy extensibility. Similar to Apache Hadoop, Flink has the concept
of so called `InputFormat`s and `OutputFormat`s.

One implementation of these `InputFormat`s is the `HadoopInputFormat`. This is a wrapper that allows
users to use all existing Hadoop input formats with Flink.

This section shows some examples for connecting Flink to other systems.
[Read more about Hadoop compatibility in Flink](hadoop_compatibility.html).


### Access Microsoft Azure Table Storage

_Note: This example works starting from Flink 0.6-incubating_

Expand Down
3 changes: 2 additions & 1 deletion docs/hadoop_compatibility.md
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,8 @@ You can:
- use a Hadoop `Mapper` as [FlatMapFunction](dataset_transformations.html#flatmap).
- use a Hadoop `Reducer` as [GroupReduceFunction](dataset_transformations.html#groupreduce-on-grouped-dataset).

This document shows how to use existing Hadoop MapReduce code with Flink.
This document shows how to use existing Hadoop MapReduce code with Flink. Please refer to the
[Connecting to other systems](example_connectors.html) guide for reading from Hadoop supported file systems.

### Project Configuration

Expand Down
41 changes: 6 additions & 35 deletions flink-addons/flink-hadoop-compatibility/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@ specific language governing permissions and limitations
under the License.
-->
<project xmlns="http:https://maven.apache.org/POM/4.0.0" xmlns:xsi="http:https://www.w3.org/2001/XMLSchema-instance"

xsi:schemaLocation="http:https://maven.apache.org/POM/4.0.0 http:https://maven.apache.org/maven-v4_0_0.xsd">

<modelVersion>4.0.0</modelVersion>
Expand Down Expand Up @@ -47,6 +46,12 @@ under the License.
<artifactId>flink-clients</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-tests</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils</artifactId>
Expand All @@ -68,40 +73,6 @@ under the License.
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-mapreduce-client-core</artifactId>
<exclusions>
<exclusion>
<groupId>asm</groupId>
<artifactId>asm</artifactId>
</exclusion>
<exclusion>
<groupId>tomcat</groupId>
<artifactId>jasper-compiler</artifactId>
</exclusion>
<exclusion>
<groupId>tomcat</groupId>
<artifactId>jasper-runtime</artifactId>
</exclusion>
<exclusion>
<groupId>org.mortbay.jetty</groupId>
<artifactId>jetty</artifactId>
</exclusion>
<exclusion>
<groupId>org.mortbay.jetty</groupId>
<artifactId>jsp-api-2.1</artifactId>
</exclusion>
<exclusion>
<groupId>org.mortbay.jetty</groupId>
<artifactId>jsp-2.1</artifactId>
</exclusion>
<exclusion>
<groupId>org.mortbay.jetty</groupId>
<artifactId>jetty-util</artifactId>
</exclusion>
<exclusion>
<groupId>org.eclipse.jdt</groupId>
<artifactId>core</artifactId>
</exclusion>
</exclusions>
</dependency>
</dependencies>
</profile>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@
import java.lang.reflect.Constructor;
import java.util.Map;

import org.apache.flink.runtime.fs.hdfs.DistributedFileSystem;
import org.apache.flink.runtime.fs.hdfs.HadoopFileSystem;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.JobContext;
import org.apache.hadoop.mapred.JobID;
Expand All @@ -36,7 +36,7 @@ public class HadoopUtils {
* Merge HadoopConfiguration into JobConf. This is necessary for the HDFS configuration.
*/
public static void mergeHadoopConf(JobConf jobConf) {
org.apache.hadoop.conf.Configuration hadoopConf = DistributedFileSystem.getHadoopConfiguration();
org.apache.hadoop.conf.Configuration hadoopConf = HadoopFileSystem.getHadoopConfiguration();
for (Map.Entry<String, String> e : hadoopConf) {
jobConf.set(e.getKey(), e.getValue());
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@
import java.lang.reflect.Constructor;
import java.util.Map;

import org.apache.flink.runtime.fs.hdfs.DistributedFileSystem;
import org.apache.flink.runtime.fs.hdfs.HadoopFileSystem;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.JobID;
Expand All @@ -34,7 +34,7 @@ public class HadoopUtils {
* Merge HadoopConfiguration into Configuration. This is necessary for the HDFS configuration.
*/
public static void mergeHadoopConf(Configuration configuration) {
Configuration hadoopConf = DistributedFileSystem.getHadoopConfiguration();
Configuration hadoopConf = HadoopFileSystem.getHadoopConfiguration();

for (Map.Entry<String, String> e : hadoopConf) {
configuration.set(e.getKey(), e.getValue());
Expand Down
22 changes: 11 additions & 11 deletions flink-addons/flink-streaming/flink-streaming-connectors/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -17,8 +17,9 @@ KIND, either express or implied. See the License for the
specific language governing permissions and limitations
under the License.
-->
<project xmlns="http:https://maven.apache.org/POM/4.0.0" xmlns:xsi="http:https://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http:https://maven.apache.org/POM/4.0.0 http:https://maven.apache.org/maven-v4_0_0.xsd">
<project xmlns="http:https://maven.apache.org/POM/4.0.0"
xmlns:xsi="http:https://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http:https://maven.apache.org/POM/4.0.0 http:https://maven.apache.org/maven-v4_0_0.xsd">

<modelVersion>4.0.0</modelVersion>

Expand All @@ -41,7 +42,7 @@ under the License.
<artifactId>flink-shaded</artifactId>
<version>${project.version}</version>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-streaming-core</artifactId>
Expand Down Expand Up @@ -224,14 +225,13 @@ under the License.
</executions>
</plugin>
<plugin>
<artifactId>maven-assembly-plugin</artifactId>
<configuration>
<descriptorRefs>
<descriptorRef>jar-with-dependencies</descriptorRef>
</descriptorRefs>
</configuration>
</plugin>
<artifactId>maven-assembly-plugin</artifactId>
<configuration>
<descriptorRefs>
<descriptorRef>jar-with-dependencies</descriptorRef>
</descriptorRefs>
</configuration>
</plugin>
</plugins>
</build>

</project>
109 changes: 109 additions & 0 deletions flink-addons/flink-tachyon/pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,109 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
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:https://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.
-->
<project xmlns="http:https://maven.apache.org/POM/4.0.0" xmlns:xsi="http:https://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http:https://maven.apache.org/POM/4.0.0 http:https://maven.apache.org/maven-v4_0_0.xsd">

<modelVersion>4.0.0</modelVersion>

<parent>
<artifactId>flink-addons</artifactId>
<groupId>org.apache.flink</groupId>
<version>0.8-incubating-SNAPSHOT</version>
<relativePath>..</relativePath>
</parent>

<artifactId>flink-tachyon</artifactId>
<name>flink-tachyon</name>

<packaging>jar</packaging>

<!--
This is a Hadoop2 only flink module.
-->
<dependencies>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-core</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-java-examples</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.tachyonproject</groupId>
<artifactId>tachyon</artifactId>
<version>0.5.0</version>
</dependency>
<dependency>
<groupId>org.tachyonproject</groupId>
<artifactId>tachyon</artifactId>
<version>0.5.0</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.eclipse.jetty</groupId>
<artifactId>jetty-util</artifactId>
<version>7.6.8.v20121106</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-common</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-hdfs</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-hdfs</artifactId>
<scope>test</scope>
<type>test-jar</type>
<version>${hadoop.version}</version>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-common</artifactId>
<scope>test</scope>
<type>test-jar</type>
<version>${hadoop.version}</version>
</dependency>
</dependencies>
<dependencyManagement>
<dependencies>
<dependency>
<groupId>org.eclipse.jetty</groupId>
<artifactId>jetty-server</artifactId>
<version>7.6.8.v20121106</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.eclipse.jetty</groupId>
<artifactId>jetty-servlet</artifactId>
<version>7.6.8.v20121106</version>
<scope>test</scope>
</dependency>
</dependencies>
</dependencyManagement>
</project>
Loading

0 comments on commit f290929

Please sign in to comment.