forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-8406] [bucketing sink] Fix proper access of Hadoop File Systems
This closes apache#5330
- Loading branch information
1 parent
524c501
commit c869eb9
Showing
2 changed files
with
85 additions
and
1 deletion.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
83 changes: 83 additions & 0 deletions
83
...test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkFsInitTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -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 | ||
* | ||
* 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. | ||
*/ | ||
|
||
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:https://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:https://localhost:51234/some/path/"); | ||
BucketingSink.createHadoopFileSystem(path, null); | ||
} | ||
} |