You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by ch...@apache.org on 2018/07/26 07:52:29 UTC
[flink] 01/03: [FLINK-5860] [tests] Replace java.io.tmpdir with
JUnit TemporaryFolder in tests
This is an automated email from the ASF dual-hosted git repository.
chesnay pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git
commit 2c0dd5422b788511637c8fa7a67c91997f6a883f
Author: Mahesh Senniappan <ma...@gmail.com>
AuthorDate: Sun Jul 22 21:35:40 2018 -0500
[FLINK-5860] [tests] Replace java.io.tmpdir with JUnit TemporaryFolder in tests
JUnit's TemporaryFolder is a better way to handle temporary folders
in tests as JUnit takes care of creation and cleanup of temporary
folders automatically.
This closes #6399.
---
.../io/DelimitedInputFormatSamplingTest.java | 17 ++-
.../api/common/io/EnumerateNestedFilesTest.java | 125 +++++----------------
.../flink/api/common/io/FileInputFormatTest.java | 8 +-
.../apache/flink/testutils/TestConfigUtils.java | 7 +-
.../org/apache/flink/testutils/TestFileUtils.java | 10 +-
.../flink/runtime/operators/DataSinkTaskTest.java | 64 ++++-------
.../runtime/operators/DataSourceTaskTest.java | 65 ++++-------
.../runtime/taskexecutor/TaskExecutorITCase.java | 7 +-
.../flink/runtime/util/JarFileCreatorTest.java | 30 +++--
.../state/RocksDBStateBackendConfigTest.java | 28 ++---
.../flink/core/testutils/CommonTestUtils.java | 9 --
11 files changed, 131 insertions(+), 239 deletions(-)
diff --git a/flink-core/src/test/java/org/apache/flink/api/common/io/DelimitedInputFormatSamplingTest.java b/flink-core/src/test/java/org/apache/flink/api/common/io/DelimitedInputFormatSamplingTest.java
index fe93332..b40913c 100644
--- a/flink-core/src/test/java/org/apache/flink/api/common/io/DelimitedInputFormatSamplingTest.java
+++ b/flink-core/src/test/java/org/apache/flink/api/common/io/DelimitedInputFormatSamplingTest.java
@@ -28,7 +28,11 @@ import org.apache.flink.testutils.TestFileUtils;
import org.apache.flink.types.IntValue;
import org.junit.Assert;
import org.junit.BeforeClass;
+import org.junit.ClassRule;
import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
public class DelimitedInputFormatSamplingTest {
@@ -68,6 +72,11 @@ public class DelimitedInputFormatSamplingTest {
private static final int DEFAULT_NUM_SAMPLES = 4;
private static Configuration CONFIG;
+
+ @ClassRule
+ public static TemporaryFolder tempFolder = new TemporaryFolder();
+
+ private static File testTempFolder;
// ========================================================================
// Setup
@@ -76,11 +85,13 @@ public class DelimitedInputFormatSamplingTest {
@BeforeClass
public static void initialize() {
try {
+ testTempFolder = tempFolder.newFolder();
// make sure we do 4 samples
CONFIG = TestConfigUtils.loadGlobalConf(
new String[] { OptimizerOptions.DELIMITED_FORMAT_MIN_LINE_SAMPLES.key(),
OptimizerOptions.DELIMITED_FORMAT_MAX_LINE_SAMPLES.key() },
- new String[] { "4", "4" });
+ new String[] { "4", "4" },
+ testTempFolder);
} catch (Throwable t) {
@@ -125,7 +136,7 @@ public class DelimitedInputFormatSamplingTest {
@Test
public void testNumSamplesMultipleFiles() {
try {
- final String tempFile = TestFileUtils.createTempFileDir(TEST_DATA1, TEST_DATA1, TEST_DATA1, TEST_DATA1);
+ final String tempFile = TestFileUtils.createTempFileDir(testTempFolder, TEST_DATA1, TEST_DATA1, TEST_DATA1, TEST_DATA1);
final Configuration conf = new Configuration();
final TestDelimitedInputFormat format = new TestDelimitedInputFormat(CONFIG);
@@ -175,7 +186,7 @@ public class DelimitedInputFormatSamplingTest {
@Test
public void testSamplingDirectory() {
try {
- final String tempFile = TestFileUtils.createTempFileDir(TEST_DATA1, TEST_DATA2);
+ final String tempFile = TestFileUtils.createTempFileDir(testTempFolder, TEST_DATA1, TEST_DATA2);
final Configuration conf = new Configuration();
final TestDelimitedInputFormat format = new TestDelimitedInputFormat(CONFIG);
diff --git a/flink-core/src/test/java/org/apache/flink/api/common/io/EnumerateNestedFilesTest.java b/flink-core/src/test/java/org/apache/flink/api/common/io/EnumerateNestedFilesTest.java
index 3ac17db..53aa1eb 100644
--- a/flink-core/src/test/java/org/apache/flink/api/common/io/EnumerateNestedFilesTest.java
+++ b/flink-core/src/test/java/org/apache/flink/api/common/io/EnumerateNestedFilesTest.java
@@ -30,12 +30,16 @@ import org.apache.flink.types.IntValue;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
+import org.junit.Rule;
import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
-public class EnumerateNestedFilesTest {
+public class EnumerateNestedFilesTest {
+
+ @Rule
+ public TemporaryFolder tempFolder = new TemporaryFolder();
protected Configuration config;
- final String tempPath = System.getProperty("java.io.tmpdir");
private DummyFileInputFormat format;
@@ -81,15 +85,8 @@ public class EnumerateNestedFilesTest {
String firstLevelDir = TestFileUtils.randomFileName();
String secondLevelDir = TestFileUtils.randomFileName();
- File nestedDir = new File(tempPath + System.getProperty("file.separator")
- + firstLevelDir);
- nestedDir.mkdirs();
- nestedDir.deleteOnExit();
-
- File insideNestedDir = new File(tempPath + System.getProperty("file.separator")
- + firstLevelDir + System.getProperty("file.separator") + secondLevelDir);
- insideNestedDir.mkdirs();
- insideNestedDir.deleteOnExit();
+ File insideNestedDir = tempFolder.newFolder(firstLevelDir, secondLevelDir);
+ File nestedDir = insideNestedDir.getParentFile();
// create a file in the first-level and two files in the nested dir
TestFileUtils.createTempFileInDirectory(nestedDir.getAbsolutePath(), "paella");
@@ -117,15 +114,8 @@ public class EnumerateNestedFilesTest {
String firstLevelDir = TestFileUtils.randomFileName();
String secondLevelDir = TestFileUtils.randomFileName();
- File nestedDir = new File(tempPath + System.getProperty("file.separator")
- + firstLevelDir);
- nestedDir.mkdirs();
- nestedDir.deleteOnExit();
-
- File insideNestedDir = new File(tempPath + System.getProperty("file.separator")
- + firstLevelDir + System.getProperty("file.separator") + secondLevelDir);
- insideNestedDir.mkdirs();
- insideNestedDir.deleteOnExit();
+ File insideNestedDir = tempFolder.newFolder(firstLevelDir, secondLevelDir);
+ File nestedDir = insideNestedDir.getParentFile();
// create a file in the first-level and two files in the nested dir
TestFileUtils.createTempFileInDirectory(nestedDir.getAbsolutePath(), "paella");
@@ -154,21 +144,9 @@ public class EnumerateNestedFilesTest {
String secondLevelDir = TestFileUtils.randomFileName();
String thirdLevelDir = TestFileUtils.randomFileName();
- File nestedDir = new File(tempPath + System.getProperty("file.separator")
- + firstLevelDir);
- nestedDir.mkdirs();
- nestedDir.deleteOnExit();
-
- File insideNestedDir = new File(tempPath + System.getProperty("file.separator")
- + firstLevelDir + System.getProperty("file.separator") + secondLevelDir);
- insideNestedDir.mkdirs();
- insideNestedDir.deleteOnExit();
-
- File nestedNestedDir = new File(tempPath + System.getProperty("file.separator")
- + firstLevelDir + System.getProperty("file.separator") + secondLevelDir
- + System.getProperty("file.separator") + thirdLevelDir);
- nestedNestedDir.mkdirs();
- nestedNestedDir.deleteOnExit();
+ File nestedNestedDir = tempFolder.newFolder(firstLevelDir, secondLevelDir, thirdLevelDir);
+ File insideNestedDir = nestedNestedDir.getParentFile();
+ File nestedDir = insideNestedDir.getParentFile();
// create a file in the first-level, two files in the second level and one in the third level
TestFileUtils.createTempFileInDirectory(nestedDir.getAbsolutePath(), "paella");
@@ -199,23 +177,10 @@ public class EnumerateNestedFilesTest {
String firstNestedNestedDir = TestFileUtils.randomFileName();
String secondNestedNestedDir = TestFileUtils.randomFileName();
- File testDir = new File(tempPath + System.getProperty("file.separator") + rootDir);
- testDir.mkdirs();
- testDir.deleteOnExit();
-
- File nested = new File(testDir.getAbsolutePath() + System.getProperty("file.separator") + nestedDir);
- nested.mkdirs();
- nested.deleteOnExit();
-
- File nestedNestedDir1 = new File(nested.getAbsolutePath() + System.getProperty("file.separator")
- + firstNestedNestedDir);
- nestedNestedDir1.mkdirs();
- nestedNestedDir1.deleteOnExit();
-
- File nestedNestedDir2 = new File(nested.getAbsolutePath() + System.getProperty("file.separator")
- + secondNestedNestedDir);
- nestedNestedDir2.mkdirs();
- nestedNestedDir2.deleteOnExit();
+ File testDir = tempFolder.newFolder(rootDir);
+ tempFolder.newFolder(rootDir, nestedDir);
+ File nestedNestedDir1 = tempFolder.newFolder(rootDir, nestedDir, firstNestedNestedDir);
+ File nestedNestedDir2 = tempFolder.newFolder(rootDir, nestedDir, secondNestedNestedDir);
// create files in second level
TestFileUtils.createTempFileInDirectory(nestedNestedDir1.getAbsolutePath(), "paella");
@@ -240,9 +205,6 @@ public class EnumerateNestedFilesTest {
*/
@Test
public void testTwoNestedDirectoriesWithFilteredFilesTrue() {
-
- String sep = System.getProperty("file.separator");
-
try {
String firstLevelDir = TestFileUtils.randomFileName();
String secondLevelDir = TestFileUtils.randomFileName();
@@ -250,26 +212,13 @@ public class EnumerateNestedFilesTest {
String secondLevelFilterDir = "_"+TestFileUtils.randomFileName();
String thirdLevelFilterDir = "_"+TestFileUtils.randomFileName();
- File nestedDir = new File(tempPath + sep + firstLevelDir);
- nestedDir.mkdirs();
- nestedDir.deleteOnExit();
-
- File insideNestedDir = new File(tempPath + sep + firstLevelDir + sep + secondLevelDir);
- insideNestedDir.mkdirs();
- insideNestedDir.deleteOnExit();
- File insideNestedDirFiltered = new File(tempPath + sep + firstLevelDir + sep + secondLevelFilterDir);
- insideNestedDirFiltered.mkdirs();
- insideNestedDirFiltered.deleteOnExit();
- File filteredFile = new File(tempPath + sep + firstLevelDir + sep + "_IWillBeFiltered");
+ File nestedNestedDirFiltered = tempFolder.newFolder(firstLevelDir, secondLevelDir, thirdLevelDir, thirdLevelFilterDir);
+ File nestedNestedDir = nestedNestedDirFiltered.getParentFile();
+ File insideNestedDir = nestedNestedDir.getParentFile();
+ File nestedDir = insideNestedDir.getParentFile();
+ File insideNestedDirFiltered = tempFolder.newFolder(firstLevelDir, secondLevelFilterDir);
+ File filteredFile = new File(nestedDir, "_IWillBeFiltered");
filteredFile.createNewFile();
- filteredFile.deleteOnExit();
-
- File nestedNestedDir = new File(tempPath + sep + firstLevelDir + sep + secondLevelDir + sep + thirdLevelDir);
- nestedNestedDir.mkdirs();
- nestedNestedDir.deleteOnExit();
- File nestedNestedDirFiltered = new File(tempPath + sep + firstLevelDir + sep + secondLevelDir + sep + thirdLevelFilterDir);
- nestedNestedDirFiltered.mkdirs();
- nestedNestedDirFiltered.deleteOnExit();
// create a file in the first-level, two files in the second level and one in the third level
TestFileUtils.createTempFileInDirectory(nestedDir.getAbsolutePath(), "paella");
@@ -300,15 +249,8 @@ public class EnumerateNestedFilesTest {
String firstLevelDir = TestFileUtils.randomFileName();
String secondLevelDir = TestFileUtils.randomFileName();
- File nestedDir = new File(tempPath + System.getProperty("file.separator")
- + firstLevelDir);
- nestedDir.mkdirs();
- nestedDir.deleteOnExit();
-
- File insideNestedDir = new File(tempPath + System.getProperty("file.separator")
- + firstLevelDir + System.getProperty("file.separator") + secondLevelDir);
- insideNestedDir.mkdirs();
- insideNestedDir.deleteOnExit();
+ File insideNestedDir = tempFolder.newFolder(firstLevelDir, secondLevelDir);
+ File nestedDir = insideNestedDir.getParentFile();
// create a file in the nested dir
TestFileUtils.createTempFileInDirectory(insideNestedDir.getAbsolutePath(), SIZE);
@@ -338,20 +280,9 @@ public class EnumerateNestedFilesTest {
String secondLevelDir = TestFileUtils.randomFileName();
String secondLevelDir2 = TestFileUtils.randomFileName();
- File nestedDir = new File(tempPath + System.getProperty("file.separator")
- + firstLevelDir);
- nestedDir.mkdirs();
- nestedDir.deleteOnExit();
-
- File insideNestedDir = new File(tempPath + System.getProperty("file.separator")
- + firstLevelDir + System.getProperty("file.separator") + secondLevelDir);
- insideNestedDir.mkdirs();
- insideNestedDir.deleteOnExit();
-
- File insideNestedDir2 = new File(tempPath + System.getProperty("file.separator")
- + firstLevelDir + System.getProperty("file.separator") + secondLevelDir2);
- insideNestedDir2.mkdirs();
- insideNestedDir2.deleteOnExit();
+ File insideNestedDir = tempFolder.newFolder(firstLevelDir, secondLevelDir);
+ File insideNestedDir2 = tempFolder.newFolder(firstLevelDir, secondLevelDir2);
+ File nestedDir = insideNestedDir.getParentFile();
// create a file in the first-level and two files in the nested dir
TestFileUtils.createTempFileInDirectory(nestedDir.getAbsolutePath(), SIZE1);
diff --git a/flink-core/src/test/java/org/apache/flink/api/common/io/FileInputFormatTest.java b/flink-core/src/test/java/org/apache/flink/api/common/io/FileInputFormatTest.java
index b464a43..5f79875 100644
--- a/flink-core/src/test/java/org/apache/flink/api/common/io/FileInputFormatTest.java
+++ b/flink-core/src/test/java/org/apache/flink/api/common/io/FileInputFormatTest.java
@@ -299,7 +299,7 @@ public class FileInputFormatTest {
final long SIZE3 = 10;
final long TOTAL = SIZE1 + SIZE2 + SIZE3;
- String tempDir = TestFileUtils.createTempFileDir(SIZE1, SIZE2, SIZE3);
+ String tempDir = TestFileUtils.createTempFileDir(temporaryFolder.newFolder(), SIZE1, SIZE2, SIZE3);
final DummyFileInputFormat format = new DummyFileInputFormat();
format.setFilePath(tempDir);
@@ -455,13 +455,13 @@ public class FileInputFormatTest {
final long size3 = 10;
final long totalSize123 = size1 + size2 + size3;
- String tempDir = TestFileUtils.createTempFileDir(size1, size2, size3);
+ String tempDir = TestFileUtils.createTempFileDir(temporaryFolder.newFolder(), size1, size2, size3);
final long size4 = 2051;
final long size5 = 31902;
final long size6 = 15;
final long totalSize456 = size4 + size5 + size6;
- String tempDir2 = TestFileUtils.createTempFileDir(size4, size5, size6);
+ String tempDir2 = TestFileUtils.createTempFileDir(temporaryFolder.newFolder(), size4, size5, size6);
final MultiDummyFileInputFormat format = new MultiDummyFileInputFormat();
format.setFilePaths(tempDir, tempDir2);
@@ -532,7 +532,7 @@ public class FileInputFormatTest {
@Test
public void testFileInputSplit() {
try {
- String tempFile = TestFileUtils.createTempFileDirExtension(".deflate", "some", "stupid", "meaningless", "files");
+ String tempFile = TestFileUtils.createTempFileDirExtension(temporaryFolder.newFolder(), ".deflate", "some", "stupid", "meaningless", "files");
final DummyFileInputFormat format = new DummyFileInputFormat();
format.setFilePath(tempFile);
format.configure(new Configuration());
diff --git a/flink-core/src/test/java/org/apache/flink/testutils/TestConfigUtils.java b/flink-core/src/test/java/org/apache/flink/testutils/TestConfigUtils.java
index d34f20a..b1627d2 100644
--- a/flink-core/src/test/java/org/apache/flink/testutils/TestConfigUtils.java
+++ b/flink-core/src/test/java/org/apache/flink/testutils/TestConfigUtils.java
@@ -31,12 +31,11 @@ import org.apache.flink.configuration.GlobalConfiguration;
*/
public final class TestConfigUtils {
- public static Configuration loadGlobalConf(String[] keys, String[] values) throws IOException {
- return loadGlobalConf(getConfAsString(keys, values));
+ public static Configuration loadGlobalConf(String[] keys, String[] values, File tempDir) throws IOException {
+ return loadGlobalConf(getConfAsString(keys, values), tempDir);
}
- public static Configuration loadGlobalConf(String contents) throws IOException {
- final File tempDir = new File(System.getProperty("java.io.tmpdir"));
+ public static Configuration loadGlobalConf(String contents, File tempDir) throws IOException {
File confDir;
do {
confDir = new File(tempDir, TestFileUtils.randomFileName());
diff --git a/flink-core/src/test/java/org/apache/flink/testutils/TestFileUtils.java b/flink-core/src/test/java/org/apache/flink/testutils/TestFileUtils.java
index ebe68f0..e8ba295 100644
--- a/flink-core/src/test/java/org/apache/flink/testutils/TestFileUtils.java
+++ b/flink-core/src/test/java/org/apache/flink/testutils/TestFileUtils.java
@@ -90,8 +90,7 @@ public class TestFileUtils {
// ------------------------------------------------------------------------
- public static String createTempFileDir(long ... bytes) throws IOException {
- File tempDir = new File(System.getProperty("java.io.tmpdir"));
+ public static String createTempFileDir(File tempDir, long ... bytes) throws IOException {
File f = null;
do {
f = new File(tempDir, randomFileName());
@@ -112,12 +111,11 @@ public class TestFileUtils {
return f.toURI().toString();
}
- public static String createTempFileDir(String ... contents) throws IOException {
- return createTempFileDirExtension(FILE_SUFFIX, contents);
+ public static String createTempFileDir(File tempDir, String ... contents) throws IOException {
+ return createTempFileDirExtension(tempDir, FILE_SUFFIX, contents);
}
- public static String createTempFileDirExtension(String fileExtension, String ... contents ) throws IOException {
- File tempDir = new File(System.getProperty("java.io.tmpdir"));
+ public static String createTempFileDirExtension(File tempDir, String fileExtension, String ... contents ) throws IOException {
File f = null;
do {
f = new File(tempDir, randomFileName(FILE_SUFFIX));
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/DataSinkTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/DataSinkTaskTest.java
index c92b2f0..d2194e1 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/DataSinkTaskTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/DataSinkTaskTest.java
@@ -30,9 +30,10 @@ import org.apache.flink.runtime.operators.util.LocalStrategy;
import org.apache.flink.runtime.testutils.recordutils.RecordComparatorFactory;
import org.apache.flink.types.IntValue;
import org.apache.flink.types.Record;
-import org.junit.After;
import org.junit.Assert;
+import org.junit.Rule;
import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -44,11 +45,15 @@ import java.io.IOException;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Set;
+import java.util.UUID;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
public class DataSinkTaskTest extends TaskTestBase {
+
+ @Rule
+ public TemporaryFolder tempFolder = new TemporaryFolder();
private static final Logger LOG = LoggerFactory.getLogger(DataSinkTaskTest.class);
@@ -56,16 +61,6 @@ public class DataSinkTaskTest extends TaskTestBase {
private static final int NETWORK_BUFFER_SIZE = 1024;
- private final String tempTestPath = constructTestPath(DataSinkTaskTest.class, "dst_test");
-
- @After
- public void cleanUp() {
- File tempTestFile = new File(this.tempTestPath);
- if(tempTestFile.exists()) {
- tempTestFile.delete();
- }
- }
-
@Test
public void testDataSinkTask() {
FileReader fr = null;
@@ -79,12 +74,11 @@ public class DataSinkTaskTest extends TaskTestBase {
DataSinkTask<Record> testTask = new DataSinkTask<>(this.mockEnv);
- super.registerFileOutputTask(MockOutputFormat.class, new File(tempTestPath).toURI().toString());
+ File tempTestFile = new File(tempFolder.getRoot(), UUID.randomUUID().toString());
+ super.registerFileOutputTask(MockOutputFormat.class, tempTestFile.toURI().toString());
testTask.invoke();
- File tempTestFile = new File(this.tempTestPath);
-
Assert.assertTrue("Temp output file does not exist", tempTestFile.exists());
fr = new FileReader(tempTestFile);
@@ -141,7 +135,8 @@ public class DataSinkTaskTest extends TaskTestBase {
DataSinkTask<Record> testTask = new DataSinkTask<>(this.mockEnv);
- super.registerFileOutputTask(MockOutputFormat.class, new File(tempTestPath).toURI().toString());
+ File tempTestFile = new File(tempFolder.getRoot(), UUID.randomUUID().toString());
+ super.registerFileOutputTask(MockOutputFormat.class, tempTestFile.toURI().toString());
try {
// For the union reader to work, we need to start notifications *after* the union reader
@@ -157,8 +152,6 @@ public class DataSinkTaskTest extends TaskTestBase {
Assert.fail("Invoke method caused exception.");
}
- File tempTestFile = new File(this.tempTestPath);
-
Assert.assertTrue("Temp output file does not exist",tempTestFile.exists());
FileReader fr = null;
@@ -225,7 +218,8 @@ public class DataSinkTaskTest extends TaskTestBase {
super.getTaskConfig().setFilehandlesInput(0, 8);
super.getTaskConfig().setSpillingThresholdInput(0, 0.8f);
- super.registerFileOutputTask(MockOutputFormat.class, new File(tempTestPath).toURI().toString());
+ File tempTestFile = new File(tempFolder.getRoot(), UUID.randomUUID().toString());;
+ super.registerFileOutputTask(MockOutputFormat.class, tempTestFile.toURI().toString());
try {
testTask.invoke();
@@ -234,8 +228,6 @@ public class DataSinkTaskTest extends TaskTestBase {
Assert.fail("Invoke method caused exception.");
}
- File tempTestFile = new File(this.tempTestPath);
-
Assert.assertTrue("Temp output file does not exist",tempTestFile.exists());
FileReader fr = null;
@@ -297,7 +289,8 @@ public class DataSinkTaskTest extends TaskTestBase {
Configuration stubParams = new Configuration();
super.getTaskConfig().setStubParameters(stubParams);
- super.registerFileOutputTask(MockFailingOutputFormat.class, new File(tempTestPath).toURI().toString());
+ File tempTestFile = new File(tempFolder.getRoot(), UUID.randomUUID().toString());
+ super.registerFileOutputTask(MockFailingOutputFormat.class, tempTestFile.toURI().toString());
boolean stubFailed = false;
@@ -309,7 +302,6 @@ public class DataSinkTaskTest extends TaskTestBase {
Assert.assertTrue("Function exception was not forwarded.", stubFailed);
// assert that temp file was removed
- File tempTestFile = new File(this.tempTestPath);
Assert.assertFalse("Temp output file has not been removed", tempTestFile.exists());
}
@@ -337,7 +329,8 @@ public class DataSinkTaskTest extends TaskTestBase {
super.getTaskConfig().setFilehandlesInput(0, 8);
super.getTaskConfig().setSpillingThresholdInput(0, 0.8f);
- super.registerFileOutputTask(MockFailingOutputFormat.class, new File(tempTestPath).toURI().toString());
+ File tempTestFile = new File(tempFolder.getRoot(), UUID.randomUUID().toString());
+ super.registerFileOutputTask(MockFailingOutputFormat.class, tempTestFile.toURI().toString());
boolean stubFailed = false;
@@ -349,7 +342,6 @@ public class DataSinkTaskTest extends TaskTestBase {
Assert.assertTrue("Function exception was not forwarded.", stubFailed);
// assert that temp file was removed
- File tempTestFile = new File(this.tempTestPath);
Assert.assertFalse("Temp output file has not been removed", tempTestFile.exists());
}
@@ -363,7 +355,9 @@ public class DataSinkTaskTest extends TaskTestBase {
Configuration stubParams = new Configuration();
super.getTaskConfig().setStubParameters(stubParams);
- super.registerFileOutputTask(MockOutputFormat.class, new File(tempTestPath).toURI().toString());
+ File tempTestFile = new File(tempFolder.getRoot(), UUID.randomUUID().toString());
+
+ super.registerFileOutputTask(MockOutputFormat.class, tempTestFile.toURI().toString());
Thread taskRunner = new Thread() {
@Override
@@ -377,8 +371,6 @@ public class DataSinkTaskTest extends TaskTestBase {
}
};
taskRunner.start();
-
- File tempTestFile = new File(this.tempTestPath);
// wait until the task created the file
long deadline = System.currentTimeMillis() + 60000;
@@ -419,7 +411,8 @@ public class DataSinkTaskTest extends TaskTestBase {
super.getTaskConfig().setFilehandlesInput(0, 8);
super.getTaskConfig().setSpillingThresholdInput(0, 0.8f);
- super.registerFileOutputTask(MockOutputFormat.class, new File(tempTestPath).toURI().toString());
+ File tempTestFile = new File(tempFolder.getRoot(), UUID.randomUUID().toString());
+ super.registerFileOutputTask(MockOutputFormat.class, tempTestFile.toURI().toString());
Thread taskRunner = new Thread() {
@Override
@@ -492,20 +485,5 @@ public class DataSinkTaskTest extends TaskTestBase {
super.writeRecord(rec);
}
}
-
- public static String constructTestPath(Class<?> forClass, String folder) {
- // we create test path that depends on class to prevent name clashes when two tests
- // create temp files with the same name
- String path = System.getProperty("java.io.tmpdir");
- if (!(path.endsWith("/") || path.endsWith("\\")) ) {
- path += System.getProperty("file.separator");
- }
- path += (forClass.getName() + "-" + folder);
- return path;
- }
-
- public static String constructTestURI(Class<?> forClass, String folder) {
- return new File(constructTestPath(forClass, folder)).toURI().toString();
- }
}
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/DataSourceTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/DataSourceTaskTest.java
index 9a3c956..df75a4f 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/DataSourceTaskTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/DataSourceTaskTest.java
@@ -29,9 +29,10 @@ import org.apache.flink.types.IntValue;
import org.apache.flink.types.Record;
import org.apache.flink.util.MutableObjectIterator;
-import org.junit.After;
import org.junit.Assert;
+import org.junit.Rule;
import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
import java.io.BufferedWriter;
import java.io.File;
@@ -42,45 +43,35 @@ import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
+import java.util.UUID;
public class DataSourceTaskTest extends TaskTestBase {
+ @Rule
+ public TemporaryFolder tempFolder = new TemporaryFolder();
+
private static final int MEMORY_MANAGER_SIZE = 1024 * 1024;
private static final int NETWORK_BUFFER_SIZE = 1024;
private List<Record> outList;
- private String tempTestPath = DataSinkTaskTest.constructTestPath(DataSourceTaskTest.class, "dst_test");
-
- @After
- public void cleanUp() {
- File tempTestFile = new File(this.tempTestPath);
- if(tempTestFile.exists()) {
- tempTestFile.delete();
- }
- }
-
@Test
- public void testDataSourceTask() {
+ public void testDataSourceTask() throws IOException {
int keyCnt = 100;
int valCnt = 20;
this.outList = new ArrayList<Record>();
-
- try {
- InputFilePreparator.prepareInputFile(new UniformRecordGenerator(keyCnt, valCnt, false),
- this.tempTestPath, true);
- } catch (IOException e1) {
- Assert.fail("Unable to set-up test input file");
- }
+ File tempTestFile = new File(tempFolder.getRoot(), UUID.randomUUID().toString());
+ InputFilePreparator.prepareInputFile(new UniformRecordGenerator(keyCnt, valCnt, false),
+ tempTestFile, true);
super.initEnvironment(MEMORY_MANAGER_SIZE, NETWORK_BUFFER_SIZE);
super.addOutput(this.outList);
DataSourceTask<Record> testTask = new DataSourceTask<>(this.mockEnv);
- super.registerFileInputTask(testTask, MockInputFormat.class, new File(tempTestPath).toURI().toString(), "\n");
+ super.registerFileInputTask(testTask, MockInputFormat.class, tempTestFile.toURI().toString(), "\n");
try {
testTask.invoke();
@@ -128,25 +119,21 @@ public class DataSourceTaskTest extends TaskTestBase {
}
@Test
- public void testFailingDataSourceTask() {
+ public void testFailingDataSourceTask() throws IOException {
int keyCnt = 20;
int valCnt = 10;
this.outList = new NirvanaOutputList();
-
- try {
- InputFilePreparator.prepareInputFile(new UniformRecordGenerator(keyCnt, valCnt, false),
- this.tempTestPath, false);
- } catch (IOException e1) {
- Assert.fail("Unable to set-up test input file");
- }
+ File tempTestFile = new File(tempFolder.getRoot(), UUID.randomUUID().toString());
+ InputFilePreparator.prepareInputFile(new UniformRecordGenerator(keyCnt, valCnt, false),
+ tempTestFile, false);
super.initEnvironment(MEMORY_MANAGER_SIZE, NETWORK_BUFFER_SIZE);
super.addOutput(this.outList);
DataSourceTask<Record> testTask = new DataSourceTask<>(this.mockEnv);
- super.registerFileInputTask(testTask, MockFailingInputFormat.class, new File(tempTestPath).toURI().toString(), "\n");
+ super.registerFileInputTask(testTask, MockFailingInputFormat.class, tempTestFile.toURI().toString(), "\n");
boolean stubFailed = false;
@@ -158,29 +145,24 @@ public class DataSourceTaskTest extends TaskTestBase {
Assert.assertTrue("Function exception was not forwarded.", stubFailed);
// assert that temp file was created
- File tempTestFile = new File(this.tempTestPath);
Assert.assertTrue("Temp output file does not exist",tempTestFile.exists());
}
@Test
- public void testCancelDataSourceTask() {
+ public void testCancelDataSourceTask() throws IOException {
int keyCnt = 20;
int valCnt = 4;
super.initEnvironment(MEMORY_MANAGER_SIZE, NETWORK_BUFFER_SIZE);
super.addOutput(new NirvanaOutputList());
-
- try {
- InputFilePreparator.prepareInputFile(new UniformRecordGenerator(keyCnt, valCnt, false),
- this.tempTestPath, false);
- } catch (IOException e1) {
- Assert.fail("Unable to set-up test input file");
- }
+ File tempTestFile = new File(tempFolder.getRoot(), UUID.randomUUID().toString());
+ InputFilePreparator.prepareInputFile(new UniformRecordGenerator(keyCnt, valCnt, false),
+ tempTestFile, false);
final DataSourceTask<Record> testTask = new DataSourceTask<>(this.mockEnv);
- super.registerFileInputTask(testTask, MockDelayingInputFormat.class, new File(tempTestPath).toURI().toString(), "\n");
+ super.registerFileInputTask(testTask, MockDelayingInputFormat.class, tempTestFile.toURI().toString(), "\n");
Thread taskRunner = new Thread() {
@Override
@@ -206,16 +188,15 @@ public class DataSourceTaskTest extends TaskTestBase {
}
// assert that temp file was created
- File tempTestFile = new File(this.tempTestPath);
Assert.assertTrue("Temp output file does not exist",tempTestFile.exists());
}
private static class InputFilePreparator {
- public static void prepareInputFile(MutableObjectIterator<Record> inIt, String inputFilePath, boolean insertInvalidData)
+ public static void prepareInputFile(MutableObjectIterator<Record> inIt, File inputFile, boolean insertInvalidData)
throws IOException {
- try (BufferedWriter bw = new BufferedWriter(new FileWriter(inputFilePath))) {
+ try (BufferedWriter bw = new BufferedWriter(new FileWriter(inputFile))) {
if (insertInvalidData) {
bw.write("####_I_AM_INVALID_########\n");
}
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java
index 9a864bd..f0e0efe 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java
@@ -59,7 +59,9 @@ import org.apache.flink.runtime.util.TestingFatalErrorHandler;
import org.apache.flink.util.TestLogger;
import org.hamcrest.Matchers;
+import org.junit.Rule;
import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
import org.mockito.Mockito;
import java.io.File;
@@ -84,6 +86,9 @@ import static org.mockito.Mockito.when;
public class TaskExecutorITCase extends TestLogger {
+ @Rule
+ public TemporaryFolder tempFolder = new TemporaryFolder();
+
private final Time timeout = Time.seconds(10L);
@Test
@@ -127,7 +132,7 @@ public class TaskExecutorITCase extends TestLogger {
TestingUtils.infiniteTime());
final File[] taskExecutorLocalStateRootDirs =
- new File[]{new File(System.getProperty("java.io.tmpdir"), "localRecovery")};
+ new File[]{ new File(tempFolder.getRoot(),"localRecovery") };
final TaskExecutorLocalStateStoresManager taskStateManager = new TaskExecutorLocalStateStoresManager(
false,
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/JarFileCreatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/JarFileCreatorTest.java
index f5d07de..60e3292 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/JarFileCreatorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/JarFileCreatorTest.java
@@ -31,7 +31,10 @@ import org.apache.flink.runtime.util.jartestprogram.AnonymousInNonStaticMethod2;
import org.apache.flink.runtime.util.jartestprogram.NestedAnonymousInnerClass;
import org.junit.Assert;
import org.junit.Ignore;
+import org.junit.Rule;
import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
@@ -42,10 +45,13 @@ import java.util.zip.ZipEntry;
public class JarFileCreatorTest {
+ @Rule
+ public TemporaryFolder tempFolder = new TemporaryFolder();
+
//anonymous inner class in static method accessing a local variable in its closure.
@Test
public void TestAnonymousInnerClassTrick1() throws Exception {
- File out = new File(System.getProperty("java.io.tmpdir"), "jarcreatortest.jar");
+ File out = tempFolder.newFile("jarcreatortest.jar");
JarFileCreator jfc = new JarFileCreator(out);
jfc.addClass(AnonymousInStaticMethod.class)
.createJarFile();
@@ -63,7 +69,7 @@ public class JarFileCreatorTest {
//anonymous inner class in non static method accessing a local variable in its closure.
@Test
public void TestAnonymousInnerClassTrick2() throws Exception {
- File out = new File(System.getProperty("java.io.tmpdir"), "jarcreatortest.jar");
+ File out = new File(tempFolder.getRoot(), "jarcreatortest.jar");
JarFileCreator jfc = new JarFileCreator(out);
jfc.addClass(AnonymousInNonStaticMethod.class)
.createJarFile();
@@ -81,7 +87,7 @@ public class JarFileCreatorTest {
//anonymous inner class in non static method accessing a field of its enclosing class.
@Test
public void TestAnonymousInnerClassTrick3() throws Exception {
- File out = new File(System.getProperty("java.io.tmpdir"), "jarcreatortest.jar");
+ File out = new File(tempFolder.getRoot(), "jarcreatortest.jar");
JarFileCreator jfc = new JarFileCreator(out);
jfc.addClass(AnonymousInNonStaticMethod2.class)
.createJarFile();
@@ -99,7 +105,7 @@ public class JarFileCreatorTest {
//anonymous inner class in an anonymous inner class accessing a field of the outermost enclosing class.
@Test
public void TestAnonymousInnerClassTrick4() throws Exception {
- File out = new File(System.getProperty("java.io.tmpdir"), "jarcreatortest.jar");
+ File out = new File(tempFolder.getRoot(), "jarcreatortest.jar");
JarFileCreator jfc = new JarFileCreator(out);
jfc.addClass(NestedAnonymousInnerClass.class)
.createJarFile();
@@ -118,7 +124,7 @@ public class JarFileCreatorTest {
@Ignore // this is currently not supported (see FLINK-9520)
@Test
public void testFilterWithMethodReference() throws Exception {
- File out = new File(System.getProperty("java.io.tmpdir"), "jarcreatortest.jar");
+ File out = new File(tempFolder.getRoot(), "jarcreatortest.jar");
JarFileCreator jfc = new JarFileCreator(out);
jfc.addClass(FilterWithMethodReference.class)
.createJarFile();
@@ -133,7 +139,7 @@ public class JarFileCreatorTest {
@Test
public void testFilterWithLambda() throws Exception{
- File out = new File(System.getProperty("java.io.tmpdir"), "jarcreatortest.jar");
+ File out = new File(tempFolder.getRoot(), "jarcreatortest.jar");
JarFileCreator jfc = new JarFileCreator(out);
jfc.addClass(FilterWithLambda.class)
.createJarFile();
@@ -148,7 +154,7 @@ public class JarFileCreatorTest {
@Test
public void testFilterWithIndirection() throws Exception {
- File out = new File(System.getProperty("java.io.tmpdir"), "jarcreatortest.jar");
+ File out = new File(tempFolder.getRoot(), "jarcreatortest.jar");
JarFileCreator jfc = new JarFileCreator(out);
jfc.addClass(FilterWithIndirection.class)
.createJarFile();
@@ -167,7 +173,7 @@ public class JarFileCreatorTest {
@Test
public void TestExternalClass() throws IOException {
- File out = new File(System.getProperty("java.io.tmpdir"), "jarcreatortest.jar");
+ File out = new File(tempFolder.getRoot(), "jarcreatortest.jar");
JarFileCreator jfc = new JarFileCreator(out);
jfc.addClass(WordCountWithExternalClass.class)
.createJarFile();
@@ -184,7 +190,7 @@ public class JarFileCreatorTest {
@Test
public void TestInnerClass() throws IOException {
- File out = new File(System.getProperty("java.io.tmpdir"), "jarcreatortest.jar");
+ File out = new File(tempFolder.getRoot(), "jarcreatortest.jar");
JarFileCreator jfc = new JarFileCreator(out);
jfc.addClass(WordCountWithInnerClass.class)
.createJarFile();
@@ -201,7 +207,7 @@ public class JarFileCreatorTest {
@Test
public void TestAnonymousClass() throws IOException {
- File out = new File(System.getProperty("java.io.tmpdir"), "jarcreatortest.jar");
+ File out = new File(tempFolder.getRoot(), "jarcreatortest.jar");
JarFileCreator jfc = new JarFileCreator(out);
jfc.addClass(WordCountWithAnonymousClass.class)
.createJarFile();
@@ -218,7 +224,7 @@ public class JarFileCreatorTest {
@Test
public void TestExtendIdentifier() throws IOException {
- File out = new File(System.getProperty("java.io.tmpdir"), "jarcreatortest.jar");
+ File out = new File(tempFolder.getRoot(), "jarcreatortest.jar");
JarFileCreator jfc = new JarFileCreator(out);
jfc.addClass(WordCountWithExternalClass2.class)
.createJarFile();
@@ -236,7 +242,7 @@ public class JarFileCreatorTest {
@Test
public void TestUDFPackage() throws IOException {
- File out = new File(System.getProperty("java.io.tmpdir"), "jarcreatortest.jar");
+ File out = new File(tempFolder.getRoot(), "jarcreatortest.jar");
JarFileCreator jfc = new JarFileCreator(out);
jfc.addClass(WordCountWithInnerClass.class)
.addPackage("org.apache.flink.util")
diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest.java b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest.java
index 4bc2f9f..d6c3144 100644
--- a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest.java
+++ b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest.java
@@ -26,7 +26,6 @@ import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.CoreOptions;
import org.apache.flink.core.fs.FileSystem;
import org.apache.flink.core.fs.Path;
-import org.apache.flink.core.testutils.CommonTestUtils;
import org.apache.flink.runtime.execution.Environment;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.jobgraph.JobVertexID;
@@ -109,7 +108,8 @@ public class RocksDBStateBackendConfigTest {
rocksDbBackend.setDbStoragePaths(testDir1, testDir2);
assertArrayEquals(new String[] { testDir1, testDir2 }, rocksDbBackend.getDbStoragePaths());
- final RocksDBKeyedStateBackend<Integer> keyedBackend = createKeyedStateBackend(rocksDbBackend);
+ final Environment env = getMockEnvironment(tempFolder.newFolder());
+ final RocksDBKeyedStateBackend<Integer> keyedBackend = createKeyedStateBackend(rocksDbBackend, env);
try {
File instanceBasePath = keyedBackend.getInstanceBasePath();
@@ -142,8 +142,7 @@ public class RocksDBStateBackendConfigTest {
Configuration config = new Configuration();
config.setString(CoreOptions.DEFAULT_FILESYSTEM_SCHEME, "s3://mydomain.com:8020/flink");
FileSystem.initialize(config);
-
- testLocalDbPaths(null, new File(CommonTestUtils.getTempDir()));
+ testLocalDbPaths(null, tempFolder.getRoot());
}
finally {
FileSystem.initialize(new Configuration());
@@ -172,7 +171,8 @@ public class RocksDBStateBackendConfigTest {
final RocksDBStateBackend rocksDbBackend = new RocksDBStateBackend(tempFolder.newFolder().toURI().toString());
rocksDbBackend.setDbStoragePath(configuredPath);
- RocksDBKeyedStateBackend<Integer> keyedBackend = createKeyedStateBackend(rocksDbBackend);
+ final Environment env = getMockEnvironment(tempFolder.newFolder());
+ RocksDBKeyedStateBackend<Integer> keyedBackend = createKeyedStateBackend(rocksDbBackend, env);
try {
File instanceBasePath = keyedBackend.getInstanceBasePath();
@@ -229,11 +229,9 @@ public class RocksDBStateBackendConfigTest {
File dir1 = tempFolder.newFolder();
File dir2 = tempFolder.newFolder();
- File[] tempDirs = new File[] { dir1, dir2 };
-
assertNull(rocksDbBackend.getDbStoragePaths());
- Environment env = getMockEnvironment(tempDirs);
+ Environment env = getMockEnvironment(dir1, dir2);
RocksDBKeyedStateBackend<Integer> keyedBackend = (RocksDBKeyedStateBackend<Integer>) rocksDbBackend.
createKeyedStateBackend(
env,
@@ -273,7 +271,7 @@ public class RocksDBStateBackendConfigTest {
boolean hasFailure = false;
try {
- Environment env = getMockEnvironment();
+ Environment env = getMockEnvironment(tempFolder.newFolder());
rocksDbBackend.createKeyedStateBackend(
env,
env.getJobID(),
@@ -314,7 +312,7 @@ public class RocksDBStateBackendConfigTest {
rocksDbBackend.setDbStoragePaths(targetDir1.getAbsolutePath(), targetDir2.getAbsolutePath());
try {
- Environment env = getMockEnvironment();
+ Environment env = getMockEnvironment(tempFolder.newFolder());
AbstractKeyedStateBackend<Integer> keyedStateBackend = rocksDbBackend.createKeyedStateBackend(
env,
env.getJobID(),
@@ -466,9 +464,7 @@ public class RocksDBStateBackendConfigTest {
// ------------------------------------------------------------------------
static RocksDBKeyedStateBackend<Integer> createKeyedStateBackend(
- RocksDBStateBackend rocksDbBackend) throws Exception {
-
- final Environment env = getMockEnvironment();
+ RocksDBStateBackend rocksDbBackend, Environment env) throws Exception {
return (RocksDBKeyedStateBackend<Integer>) rocksDbBackend.
createKeyedStateBackend(
@@ -481,11 +477,7 @@ public class RocksDBStateBackendConfigTest {
env.getTaskKvStateRegistry());
}
- static Environment getMockEnvironment() {
- return getMockEnvironment(new File[] { new File(System.getProperty("java.io.tmpdir")) });
- }
-
- static Environment getMockEnvironment(File[] tempDirs) {
+ static Environment getMockEnvironment(File... tempDirs) {
final String[] tempDirStrings = new String[tempDirs.length];
for (int i = 0; i < tempDirs.length; i++) {
tempDirStrings[i] = tempDirs[i].getAbsolutePath();
diff --git a/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/core/testutils/CommonTestUtils.java b/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/core/testutils/CommonTestUtils.java
index ecaed1a..28859f5 100644
--- a/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/core/testutils/CommonTestUtils.java
+++ b/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/core/testutils/CommonTestUtils.java
@@ -40,15 +40,6 @@ import java.util.Map;
public class CommonTestUtils {
/**
- * Reads the path to the directory for temporary files from the configuration and returns it.
- *
- * @return the path to the directory for temporary files
- */
- public static String getTempDir() {
- return System.getProperty("java.io.tmpdir");
- }
-
- /**
* Creates a copy of an object via Java Serialization.
*
* @param original The original object.