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.