You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/04/20 10:05:21 UTC

[GitHub] [flink-kubernetes-operator] wangyang0918 commented on a diff in pull request #168: [FLINK-27161] Support fetch user jar from different source

wangyang0918 commented on code in PR #168:
URL: https://github.com/apache/flink-kubernetes-operator/pull/168#discussion_r853929995


##########
flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/artifact/HttpArtifactFetcher.java:
##########
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.kubernetes.operator.artifact;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.FilenameUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.net.URL;
+
+/** Download the jar from the http resource. */
+public class HttpArtifactFetcher implements ArtifactFetcher {
+
+    public static final Logger LOG = LoggerFactory.getLogger(HttpArtifactFetcher.class);
+    public static final HttpArtifactFetcher INSTANCE = new HttpArtifactFetcher();
+
+    @Override
+    public File fetch(String uri, File targetDir) throws Exception {
+        var start = System.currentTimeMillis();
+        URL url = new URL(uri);
+        String fileName = FilenameUtils.getName(url.getFile());
+        File targetFile = new File(targetDir, fileName);
+        FileUtils.copyToFile(new URL(uri).openStream(), targetFile);

Review Comment:
   Same as above.



##########
flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/artifact/FileSystemBasedArtifactFetcher.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.kubernetes.operator.artifact;
+
+import org.apache.flink.core.fs.FileSystem;
+
+import org.apache.commons.io.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+
+/** Leverage the flink filesystem plugin to fetch the artifact. */
+public class FileSystemBasedArtifactFetcher implements ArtifactFetcher {
+
+    public static final Logger LOG = LoggerFactory.getLogger(FileSystemBasedArtifactFetcher.class);
+    public static final FileSystemBasedArtifactFetcher INSTANCE =
+            new FileSystemBasedArtifactFetcher();
+
+    @Override
+    public File fetch(String uri, File targetDir) throws Exception {
+        org.apache.flink.core.fs.Path source = new org.apache.flink.core.fs.Path(uri);
+        var start = System.currentTimeMillis();
+        FileSystem fileSystem = source.getFileSystem();
+        String fileName = source.getName();
+        File targetFile = new File(targetDir, fileName);
+        FileUtils.copyToFile(fileSystem.open(source), targetFile);

Review Comment:
   It seems that the opened stream is not closed correctly.



##########
flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/artifact/ArtifactManagerTest.java:
##########
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.kubernetes.operator.artifact;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.kubernetes.operator.TestUtils;
+import org.apache.flink.kubernetes.operator.config.FlinkOperatorConfiguration;
+import org.apache.flink.kubernetes.operator.config.OperatorConfigOptions;
+import org.apache.flink.util.Preconditions;
+
+import com.sun.net.httpserver.HttpExchange;
+import com.sun.net.httpserver.HttpHandler;
+import com.sun.net.httpserver.HttpServer;
+import org.apache.commons.io.FileUtils;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.HttpURLConnection;
+import java.net.InetSocketAddress;
+import java.net.URL;
+import java.nio.file.Path;
+
+/** Test for {@link ArtifactManager}. */
+public class ArtifactManagerTest {
+
+    @TempDir Path tempDir;
+    private ArtifactManager artifactManager;
+
+    @BeforeEach
+    public void setup() {
+        Configuration configuration = new Configuration();
+        configuration.setString(
+                OperatorConfigOptions.OPERATOR_USER_JAR_BASE_DIR,
+                tempDir.toAbsolutePath().toString());
+        artifactManager =
+                new ArtifactManager(FlinkOperatorConfiguration.fromConfiguration(configuration));
+    }
+
+    @Test
+    public void testGenerateJarDir() {
+        String baseDir =
+                artifactManager.generateJarDir(
+                        TestUtils.buildSessionCluster(), TestUtils.buildSessionJob());
+        String expected =
+                tempDir.toString()
+                        + File.separator
+                        + TestUtils.TEST_NAMESPACE
+                        + File.separator
+                        + TestUtils.TEST_DEPLOYMENT_NAME
+                        + File.separator
+                        + TestUtils.TEST_SESSION_JOB_NAME;
+        Assertions.assertEquals(expected, baseDir);
+    }
+
+    @Test
+    public void testFilesystemFetch() throws Exception {
+        var sourceFile = mockTheJarFile();
+        File file =
+                artifactManager.fetch(
+                        String.format("file://%s", sourceFile.getAbsolutePath()),
+                        tempDir.toString());
+        Assertions.assertTrue(file.exists());
+        Assertions.assertEquals(tempDir.toString(), file.getParentFile().toString());
+    }
+
+    @Test
+    public void testHttpFetch() throws Exception {
+        HttpServer httpServer = null;
+        try {
+            httpServer = HttpServer.create(new InetSocketAddress(1234), 0);

Review Comment:
   The test will be unstable when port `1234` is not available or running concurrently.



##########
flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/OperatorConfigOptions.java:
##########
@@ -97,4 +97,10 @@ public class OperatorConfigOptions {
                     .withDescription(
                             "The timeout for deployments to become ready/stable "
                                     + "before being rolled back if rollback is enabled.");
+
+    public static final ConfigOption<String> OPERATOR_USER_JAR_BASE_DIR =

Review Comment:
   We might need to update the docs.



##########
flink-kubernetes-operator/pom.xml:
##########
@@ -135,6 +141,13 @@ under the License.
             <version>${fabric8.version}</version>
             <scope>test</scope>
         </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-test-utils-junit</artifactId>
+            <version>${flink.version}</version>
+            <scope>test</scope>
+        </dependency>

Review Comment:
   It seems that we do not need this dependency.



##########
flink-kubernetes-operator/pom.xml:
##########
@@ -75,6 +75,12 @@ under the License.
             <version>${flink.version}</version>
         </dependency>
 
+        <dependency>

Review Comment:
   We might need to update the notice file when introducing new dependency.



##########
flink-kubernetes-operator/pom.xml:
##########
@@ -270,6 +283,13 @@ under the License.
                             <version>${flink.version}</version>
                             <outputDirectory>${plugins.tmp.dir}/flink-metrics-statsd</outputDirectory>
                         </artifactItem>
+                        <artifactItem>

Review Comment:
   It is strange to only add the oss filesystem here. We need to bundle all the internal fs implementation or none.



##########
flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/FlinkService.java:
##########
@@ -204,12 +209,14 @@ private JarRunResponseBody runJar(
         }
     }
 
-    private JarUploadResponseBody uploadJar(FlinkSessionJob sessionJob, Configuration conf)
+    private JarUploadResponseBody uploadJar(
+            FlinkSessionJob sessionJob, FlinkDeployment sessionCluster, Configuration conf)
             throws Exception {
-        Path path = jarResolver.resolve(sessionJob.getSpec().getJob().getJarURI());
+        String targetDir = artifactManager.generateJarDir(sessionCluster, sessionJob);
+        File jarFile = artifactManager.fetch(sessionJob.getSpec().getJob().getJarURI(), targetDir);

Review Comment:
   If the user specified jar is a local file `file:///tmp/a.jar`, do we still need to copy it to `/opt/flink/artifacts`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org