You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "mingleizhang (JIRA)" <ji...@apache.org> on 2017/07/26 03:11:00 UTC

[jira] [Commented] (FLINK-5789) Make Bucketing Sink independent of Hadoop's FileSystem

    [ https://issues.apache.org/jira/browse/FLINK-5789?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16101108#comment-16101108 ] 

mingleizhang commented on FLINK-5789:
-------------------------------------

{{Path}} class in flink does not support {{FileSystem getFileSystem(org.apache.flink.configuration.Configuration conf)}}. I would like to ask, do we need to support it ? It not, we probably not initialize {{BucketingSink#initFileSystem}} with a user-defined configuration. All I can do now with implementing like following


{code:java}
private void initFileSystem() throws IOException {
	if (fs != null) {
	 return;
	}
	fs = new Path(basePath).getFileSystem();
}
{code}


> Make Bucketing Sink independent of Hadoop's FileSystem
> ------------------------------------------------------
>
>                 Key: FLINK-5789
>                 URL: https://issues.apache.org/jira/browse/FLINK-5789
>             Project: Flink
>          Issue Type: Bug
>          Components: Streaming Connectors
>    Affects Versions: 1.2.0, 1.1.4
>            Reporter: Stephan Ewen
>             Fix For: 1.4.0
>
>
> The {{BucketingSink}} is hard wired to Hadoop's FileSystem, bypassing Flink's file system abstraction.
> This causes several issues:
>   - The bucketing sink will behave different than other file sinks with respect to configuration
>   - Directly supported file systems (not through hadoop) like the MapR File System does not work in the same way with the BuketingSink as other file systems
>   - The previous point is all the more problematic in the effort to make Hadoop an optional dependency and with in other stacks (Mesos, Kubernetes, AWS, GCE, Azure) with ideally no Hadoop dependency.
> We should port the {{BucketingSink}} to use Flink's FileSystem classes.
> To support the *truncate* functionality that is needed for the exactly-once semantics of the Bucketing Sink, we should extend Flink's FileSystem abstraction to have the methods
>   - {{boolean supportsTruncate()}}
>   - {{void truncate(Path, long)}}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)