[ https://issues.apache.org/jira/browse/FLINK-8432?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16341240#comment-16341240 ]
ASF GitHub Bot commented on FLINK-8432: --------------------------------------- Github user jelmerk commented on a diff in the pull request: https://github.com/apache/flink/pull/5296#discussion_r164158043 --- Diff: flink-filesystems/flink-openstack-fs-hadoop/src/main/java/org/apache/flink/fs/openstackhadoop/SwiftFileSystemFactory.java --- @@ -0,0 +1,132 @@ +/* + * 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.openstackhadoop; + +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.flink.runtime.util.HadoopUtils; + +import org.apache.hadoop.fs.swift.snative.SwiftNativeFileSystem; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.URI; + +/** + * Simple factory for the Swift file system. + */ +public class SwiftFileSystemFactory implements FileSystemFactory { + + private static final Logger LOG = LoggerFactory.getLogger(SwiftFileSystemFactory.class); + + /** The prefixes that Flink adds to the Hadoop config under 'fs.swift.'. */ + private static final String CONFIG_PREFIX = "swift."; + + /** Flink's configuration object. */ + private Configuration flinkConfig; + + /** Hadoop's configuration for the file systems, lazily initialized. */ + private org.apache.hadoop.conf.Configuration hadoopConfig; + + @Override + public String getScheme() { + return "swift"; + } + + @Override + public void configure(Configuration config) { + flinkConfig = config; + hadoopConfig = null; + } + + @Override + public FileSystem create(URI fsUri) throws IOException { + LOG.debug("Creating swift file system (backed by a Hadoop native swift file system)"); + + try { + // -- (1) get the loaded Hadoop config (or fall back to one loaded from the classpath) + + org.apache.hadoop.conf.Configuration hadoopConfig = this.hadoopConfig; + if (hadoopConfig == null) { + if (flinkConfig != null) { + LOG.debug("Loading Hadoop configuration for swift native file system"); + hadoopConfig = HadoopUtils.getHadoopConfiguration(flinkConfig); + + // hadoop.tmp.dir needs to be defined because it is used as buffer directory + if (hadoopConfig.get("hadoop.tmp.dir") == null) { + String tmpDir = System.getProperty("java.io.tmpdir") + "/" + "hadoop-" + System.getProperty("user.name"); --- End diff -- Changed in 718ac46f45bc0b1a275f301255c52cec7ecdd209 > Add openstack swift filesystem > ------------------------------ > > Key: FLINK-8432 > URL: https://issues.apache.org/jira/browse/FLINK-8432 > Project: Flink > Issue Type: Improvement > Components: filesystem-connector > Affects Versions: 1.4.0 > Reporter: Jelmer Kuperus > Priority: Major > Labels: features > > At ebay classifieds we started running our infrastructure on top of OpenStack. > The openstack project comes with its own amazon-s3-like filesystem, known as > Swift. It's built for scale and optimized for durability, availability, and > concurrency across the entire data set. Swift is ideal for storing > unstructured data that can grow without bound. > We would really like to be able to use it within flink without Hadoop > dependencies, as a sink or for storing savepoints etc > I've prepared a pull request that adds support for it. It wraps the hadoop > support for swift in a way that is very similar to the way the s3 connector > works. > You can find out more about the underlying hadoop implementation at > [https://hadoop.apache.org/docs/stable/hadoop-openstack/index.html] > Pull request : [https://github.com/apache/flink/pull/5296] -- This message was sent by Atlassian JIRA (v7.6.3#76005)