Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FLINK-20938] Flink cos fs hadoop feature #15143

Open
wants to merge 3 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
147 changes: 147 additions & 0 deletions flink-filesystems/flink-cos-fs-hadoop/pom.xml
@@ -0,0 +1,147 @@
<?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://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://maven.apache.org/POM/4.0.0"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<parent>
<artifactId>flink-filesystems</artifactId>
<groupId>org.apache.flink</groupId>
<version>1.13-SNAPSHOT</version>
</parent>
<modelVersion>4.0.0</modelVersion>

<artifactId>flink-cos-fs-hadoop</artifactId>
<name>Flink : FileSystems : COS FS</name>

<properties>
<fs.hadoop.cos.version>3.3.0</fs.hadoop.cos.version>
</properties>

<dependencies>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-core</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
</dependency>

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

<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-common</artifactId>
<version>2.9.2</version>
<!--<scope>provided</scope>-->
</dependency>

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

<!-- https://mvnrepository.com/artifact/org.apache.hadoop/hadoop-cos -->
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-cos</artifactId>
<version>${fs.hadoop.cos.version}</version>
</dependency>

<!-- for the behavior test suite -->
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-core</artifactId>
<version>${project.version}</version>
<scope>test</scope>
<type>test-jar</type>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-hadoop-fs</artifactId>
<version>${project.version}</version>
<scope>test</scope>
<type>test-jar</type>
</dependency>
</dependencies>

<build>
<plugins>

<!-- Relocate all COS related classes -->
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-shade-plugin</artifactId>
<executions>
<execution>
<id>shade-flink</id>
<phase>package</phase>
<goals>
<goal>shade</goal>
</goals>
<configuration>
<shadeTestJar>false</shadeTestJar>
<artifactSet>
<includes>
<include>*:*</include>
</includes>
</artifactSet>
<relocations>
<relocation>
<pattern>org.apache.hadoop</pattern>
<shadedPattern>org.apache.flink.fs.shaded.hadoop3.org.apache.hadoop</shadedPattern>
</relocation>
<!-- relocate the COS dependencies -->
<relocation>
<pattern>com.qcloud</pattern>
<shadedPattern>org.apache.flink.fs.coshadoop.shaded.com.qcloud</shadedPattern>
</relocation>
<!-- shade Flink's Hadoop FS adapter classes, forces plugin classloader for them -->
<relocation>
<pattern>org.apache.flink.runtime.fs.hdfs</pattern>
<shadedPattern>org.apache.flink.fs.coshadoop.common</shadedPattern>
</relocation>
<!-- shade Flink's Hadoop FS utility classes, forces plugin classloader for them -->
<relocation>
<pattern>org.apache.flink.runtime.util</pattern>
<shadedPattern>org.apache.flink.fs.coshadoop.common</shadedPattern>
</relocation>
</relocations>
<filters>
<filter>
<artifact>*</artifact>
<excludes>
<exclude>.gitkeep</exclude>
<exclude>mime.types</exclude>
<exclude>mozilla/**</exclude>
<exclude>META-INF/maven/**</exclude>
</excludes>
</filter>
</filters>
</configuration>
</execution>
</executions>
</plugin>
</plugins>
</build>
</project>
@@ -0,0 +1,107 @@
/*
* 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.
*/

package org.apache.flink.fs.coshadoop;

import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.fs.FileSystem;
import org.apache.flink.core.fs.FileSystemFactory;
import org.apache.flink.runtime.fs.hdfs.HadoopFileSystem;

import org.apache.hadoop.fs.cosn.CosNFileSystem;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.IOException;
import java.net.URI;

/** Simple factory for the COSN file system. */
public class COSNFileSystemFactory implements FileSystemFactory {
private static final Logger LOG = LoggerFactory.getLogger(COSNFileSystemFactory.class);

private Configuration flinkConfig;

private org.apache.hadoop.conf.Configuration hadoopConfig;

private static final String[] FLINK_CONFIG_PREFIXES = {"fs.cosn."};
private static final String FLINK_ABSTRACT_FILESYSTEM_COSN_IMPL =
"fs.cosn.AbstractFileSystem.cosn.impl";

@Override
public String getScheme() {
return "cosn";
}

@Override
public void configure(Configuration config) {
flinkConfig = config;
hadoopConfig = null;
}

@Override
public FileSystem create(URI fsUri) throws IOException {
this.hadoopConfig = getHadoopConfiguration();

final String scheme = fsUri.getScheme();
final String authority = fsUri.getAuthority();

if (scheme == null && authority == null) {
fsUri = org.apache.hadoop.fs.FileSystem.getDefaultUri(hadoopConfig);
} else if (scheme != null && authority == null) {
URI defaultUri = org.apache.hadoop.fs.FileSystem.getDefaultUri(hadoopConfig);
if (scheme.equals(defaultUri.getScheme()) && defaultUri.getAuthority() != null) {
fsUri = defaultUri;
}
}

final CosNFileSystem fs = new CosNFileSystem();
fs.initialize(fsUri, hadoopConfig);
return new HadoopFileSystem(fs);
}

@VisibleForTesting
org.apache.hadoop.conf.Configuration getHadoopConfiguration() {
org.apache.hadoop.conf.Configuration conf = new org.apache.hadoop.conf.Configuration();
if (flinkConfig == null) {
return conf;
}

// read all configuration with prefix 'FLINK_CONFIG_PREFIXES'
for (String key : flinkConfig.keySet()) {
for (String prefix : FLINK_CONFIG_PREFIXES) {
if (key.startsWith(prefix)) {
if (FLINK_ABSTRACT_FILESYSTEM_COSN_IMPL.equals(key)) {
conf.set(
"fs.AbstractFileSystem.cosn.impl",
flinkConfig.getString(key, null));
}

String value = flinkConfig.getString(key, null);
conf.set(key, value);

LOG.debug(
"Adding Flink config entry for {} as {} to Hadoop config",
key,
conf.get(key));
}
}
}
return conf;
}
}
@@ -0,0 +1,14 @@
flink-cos-fs-hadoop
Copyright 2014-2020 The Apache Software Foundation

This project includes software developed at
The Apache Software Foundation (http://www.apache.org/).

This project bundles the following dependencies under the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt)

- org.apache.hadoop:hadoop-cos:3.3.0

This project bundles the following dependencies under the MIT license.
See bundled license files for details.

- com.qcloud:cos_api-bundle:5.6.19
@@ -0,0 +1,16 @@
# 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.

org.apache.flink.fs.coshadoop.COSNFileSystemFactory
@@ -0,0 +1,71 @@
/*
* 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.
*/

package org.apache.flink.fs.coshadoop;

import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.fs.FileSystem;
import org.apache.flink.core.fs.FileSystemBehaviorTestSuite;
import org.apache.flink.core.fs.FileSystemKind;
import org.apache.flink.core.fs.Path;
import org.apache.flink.testutils.cos.COSTestCredentials;

import org.junit.AfterClass;
import org.junit.BeforeClass;

import java.io.IOException;
import java.util.UUID;

/** An implementation of the {@link FileSystemBehaviorTestSuite} for the COSN file system. */
public class HadoopCOSNFileSystemBehaviorITCase extends FileSystemBehaviorTestSuite {
private static final String TEST_DATA_DIR = "tests-" + UUID.randomUUID();

@BeforeClass
public static void setup() throws IOException {
COSTestCredentials.assumeCredentialsAvailable();

final Configuration conf = new Configuration();
conf.setString("fs.cosn.userinfo.secretId", COSTestCredentials.getCosTestSecretId());
conf.setString("fs.cosn.userinfo.secretKey", COSTestCredentials.getCosTestSecretKey());
conf.setString("fs.cosn.impl", COSTestCredentials.getCosTestCosnImpl());
conf.setString(
"fs.AbstractFileSystem.cosn.impl", COSTestCredentials.getCosTestAfsCosnImpl());
conf.setString("fs.cosn.bucket.region", COSTestCredentials.getCosTestRegion());
FileSystem.initialize(conf);
}

@Override
public FileSystem getFileSystem() throws Exception {
return getBasePath().getFileSystem();
}

@Override
public Path getBasePath() throws Exception {
return new Path(COSTestCredentials.getTestBucketUri() + TEST_DATA_DIR);
}

@Override
public FileSystemKind getFileSystemKind() {
return FileSystemKind.OBJECT_STORE;
}

@AfterClass
public static void clearFsConfig() throws IOException {
FileSystem.initialize(new Configuration());
}
}