[FLINK-8406] [bucketing sink] Fix proper access of Hadoop File Systems

This closes #5330


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/c869eb9d
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/c869eb9d
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/c869eb9d

Branch: refs/heads/master
Commit: c869eb9d1cee7e52f5dcf4a5803cdf7681d979fd
Parents: 524c501
Author: Stephan Ewen <[email protected]>
Authored: Mon Jan 22 13:32:09 2018 +0100
Committer: Stephan Ewen <[email protected]>
Committed: Wed Jan 24 18:03:16 2018 +0100

----------------------------------------------------------------------
 .../connectors/fs/bucketing/BucketingSink.java  |  3 +-
 .../fs/bucketing/BucketingSinkFsInitTest.java   | 83 ++++++++++++++++++++
 2 files changed, 85 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/c869eb9d/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSink.java
----------------------------------------------------------------------
diff --git 
a/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSink.java
 
b/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSink.java
index db0a5d8..6293fe0 100644
--- 
a/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSink.java
+++ 
b/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSink.java
@@ -1122,7 +1122,8 @@ public class BucketingSink<T>
                // try to get the Hadoop File System via the Flink File Systems
                // that way we get the proper configuration
 
-               final org.apache.flink.core.fs.FileSystem flinkFs = 
org.apache.flink.core.fs.FileSystem.get(path.toUri());
+               final org.apache.flink.core.fs.FileSystem flinkFs =
+                               
org.apache.flink.core.fs.FileSystem.getUnguardedFileSystem(path.toUri());
                final FileSystem hadoopFs = (flinkFs instanceof 
HadoopFileSystem) ?
                                ((HadoopFileSystem) 
flinkFs).getHadoopFileSystem() : null;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/c869eb9d/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkFsInitTest.java
----------------------------------------------------------------------
diff --git 
a/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkFsInitTest.java
 
b/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkFsInitTest.java
new file mode 100644
index 0000000..f684ef0
--- /dev/null
+++ 
b/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkFsInitTest.java
@@ -0,0 +1,83 @@
+/*
+ * 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.streaming.connectors.fs.bucketing;
+
+import org.apache.flink.core.fs.FileSystemSafetyNet;
+import org.apache.flink.core.fs.UnsupportedFileSystemSchemeException;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests for the file system initialization of the Bucketing sink.
+ *
+ * <p>NOTE: These tests can probably go away once the bucketing sink has been
+ * migrated to properly use Flink's file system abstraction.
+ */
+public class BucketingSinkFsInitTest {
+
+       @Rule
+       public final TemporaryFolder tempFolder = new TemporaryFolder();
+
+       // 
------------------------------------------------------------------------
+
+       // to properly mimic what happens in the runtime task, we need to make 
sure that
+       // the file system safety net is in place
+
+       @Before
+       public void activateSafetyNet() {
+               FileSystemSafetyNet.initializeSafetyNetForThread();
+       }
+
+       @After
+       public void deactivateSafetyNet() {
+               
FileSystemSafetyNet.closeSafetyNetAndGuardedResourcesForThread();
+       }
+
+       // 
------------------------------------------------------------------------
+
+       @Test
+       public void testInitForLocalFileSystem() throws Exception {
+               final Path path = new Path(tempFolder.newFolder().toURI());
+               FileSystem fs = BucketingSink.createHadoopFileSystem(path, 
null);
+
+               assertEquals("file", fs.getUri().getScheme());
+       }
+
+       @Test
+       public void testInitForHadoopFileSystem() throws Exception {
+               final Path path = new Path("hdfs://localhost:51234/some/path/");
+               FileSystem fs = BucketingSink.createHadoopFileSystem(path, 
null);
+
+               assertEquals("hdfs", fs.getUri().getScheme());
+       }
+
+       @Test(expected = UnsupportedFileSystemSchemeException.class)
+       public void testInitForUnsupportedFileSystem() throws Exception {
+               final Path path = new Path("nofs://localhost:51234/some/path/");
+               BucketingSink.createHadoopFileSystem(path, null);
+       }
+}

Reply via email to