This is an automated email from the ASF dual-hosted git repository.

afedulov pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
     new 26436ac27ae [FLINK-35512][tests] Do not depend on the `flink-clients` 
jar actually existing in `ArtifactFetchManagerTest`
26436ac27ae is described below

commit 26436ac27ae9e4705910b0502abb5bdd33ec686b
Author: Sam Barker <sbar...@redhat.com>
AuthorDate: Thu Jun 13 17:08:20 2024 +1200

    [FLINK-35512][tests] Do not depend on the `flink-clients` jar actually 
existing in `ArtifactFetchManagerTest`
---
 .../program/artifact/ArtifactFetchManagerTest.java | 24 ++++++++--------------
 .../org/apache/flink/testutils/TestingUtils.java   |  4 +++-
 2 files changed, 12 insertions(+), 16 deletions(-)

diff --git 
a/flink-clients/src/test/java/org/apache/flink/client/program/artifact/ArtifactFetchManagerTest.java
 
b/flink-clients/src/test/java/org/apache/flink/client/program/artifact/ArtifactFetchManagerTest.java
index cb36f76450a..a3ed2ce255e 100644
--- 
a/flink-clients/src/test/java/org/apache/flink/client/program/artifact/ArtifactFetchManagerTest.java
+++ 
b/flink-clients/src/test/java/org/apache/flink/client/program/artifact/ArtifactFetchManagerTest.java
@@ -95,10 +95,12 @@ class ArtifactFetchManagerTest {
     }
 
     @Test
-    void testFileSystemFetchWithAdditionalUri() throws Exception {
+    void testFileSystemFetchWithAdditionalUri(@TempDir Path pseudoJarDir) 
throws Exception {
         File sourceFile = TestingUtils.getClassFile(getClass());
         String uriStr = "file://" + sourceFile.toURI().getPath();
-        File additionalSrcFile = getFlinkClientsJar();
+        File additionalSrcFile =
+                Files.createTempFile(pseudoJarDir, 
"testFileSystemFetchWithAdditionalUri", ".jar")
+                        .toFile();
         String additionalUriStr = "file://" + 
additionalSrcFile.toURI().getPath();
 
         ArtifactFetchManager fetchMgr = new 
ArtifactFetchManager(configuration);
@@ -112,12 +114,12 @@ class ArtifactFetchManagerTest {
     }
 
     @Test
-    void testHttpFetch() throws Exception {
+    void testHttpFetch(@TempDir Path pseudoJarDir) throws Exception {
         configuration.set(ArtifactFetchOptions.RAW_HTTP_ENABLED, true);
         HttpServer httpServer = null;
         try {
             httpServer = startHttpServer();
-            File sourceFile = getFlinkClientsJar();
+            File sourceFile = Files.createTempFile(pseudoJarDir, 
"testHttpFetch", ".jar").toFile();
             httpServer.createContext(
                     "/download/" + sourceFile.getName(), new 
DummyHttpDownloadHandler(sourceFile));
             String uriStr =
@@ -138,10 +140,11 @@ class ArtifactFetchManagerTest {
     }
 
     @Test
-    void testMixedArtifactFetch() throws Exception {
+    void testMixedArtifactFetch(@TempDir Path pseudoJarDir) throws Exception {
         File sourceFile = TestingUtils.getClassFile(getClass());
         String uriStr = "file://" + sourceFile.toURI().getPath();
-        File sourceFile2 = getFlinkClientsJar();
+        File sourceFile2 =
+                Files.createTempFile(pseudoJarDir, "testMixedArtifactFetch", 
".jar").toFile();
         String uriStr2 = "file://" + sourceFile2.toURI().getPath();
 
         ArtifactFetchManager fetchMgr = new 
ArtifactFetchManager(configuration);
@@ -230,15 +233,6 @@ class ArtifactFetchManagerTest {
         return httpServer;
     }
 
-    private File getFlinkClientsJar() throws IOException {
-        return TestingUtils.getFileFromTargetDir(
-                ArtifactFetchManager.class,
-                p ->
-                        org.apache.flink.util.FileUtils.isJarFile(p)
-                                && 
p.toFile().getName().startsWith("flink-clients")
-                                && 
!p.toFile().getName().contains("test-utils"));
-    }
-
     private static class DummyHttpDownloadHandler implements HttpHandler {
 
         final File file;
diff --git 
a/flink-core/src/test/java/org/apache/flink/testutils/TestingUtils.java 
b/flink-core/src/test/java/org/apache/flink/testutils/TestingUtils.java
index 584efd48c21..2bceea09fda 100644
--- a/flink-core/src/test/java/org/apache/flink/testutils/TestingUtils.java
+++ b/flink-core/src/test/java/org/apache/flink/testutils/TestingUtils.java
@@ -80,7 +80,9 @@ public class TestingUtils {
 
         final Collection<Path> jarPaths =
                 
org.apache.flink.util.FileUtils.listFilesInDirectory(mvnTargetDir, fileFilter);
-        assertThat(jarPaths).isNotEmpty();
+        assertThat(jarPaths)
+                .describedAs("Could not find any matching files in %s", 
mvnTargetDir)
+                .isNotEmpty();
 
         return jarPaths.iterator().next().toFile();
     }

Reply via email to