sap1ens commented on code in PR #370:
URL: 
https://github.com/apache/flink-kubernetes-operator/pull/370#discussion_r977209625


##########
flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/validation/DefaultValidator.java:
##########
@@ -180,10 +180,6 @@ private Optional<String> validateJobSpec(
             return Optional.empty();
         }
 
-        if (StringUtils.isNullOrWhitespaceOnly(job.getJarURI())) {
-            return Optional.of("Jar URI must be defined");
-        }

Review Comment:
   Hey @jeesmon, I've tweaked `AbstractFlinkService` to pass an empty noop jar, 
however now I'm seeing an exception on the JobManager side: 
   
   ```
   Caused by: java.lang.ClassNotFoundException: 
org.apache.flink.streaming.examples.statemachine.StateMachineExample
        at java.net.URLClassLoader.findClass(Unknown Source) ~[?:?]
        at java.lang.ClassLoader.loadClass(Unknown Source) ~[?:?]
        at 
org.apache.flink.util.FlinkUserCodeClassLoader.loadClassWithoutExceptionHandling(FlinkUserCodeClassLoader.java:68)
 ~[flink-dist-1.15.2.jar:1.15.2]
        at 
org.apache.flink.util.ChildFirstClassLoader.loadClassWithoutExceptionHandling(ChildFirstClassLoader.java:65)
 ~[flink-dist-1.15.2.jar:1.15.2]
        at 
org.apache.flink.util.FlinkUserCodeClassLoader.loadClass(FlinkUserCodeClassLoader.java:52)
 ~[flink-dist-1.15.2.jar:1.15.2]
        at java.lang.ClassLoader.loadClass(Unknown Source) ~[?:?]
        at 
org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders$SafetyNetWrapperClassLoader.loadClass(FlinkUserCodeClassLoaders.java:172)
 ~[flink-dist-1.15.2.jar:1.15.2]
        at java.lang.Class.forName0(Native Method) ~[?:?]
        at java.lang.Class.forName(Unknown Source) ~[?:?]
        at 
org.apache.flink.client.program.PackagedProgram.loadMainClass(PackagedProgram.java:479)
 ~[flink-dist-1.15.2.jar:1.15.2]
        at 
org.apache.flink.client.program.PackagedProgram.<init>(PackagedProgram.java:153)
 ~[flink-dist-1.15.2.jar:1.15.2]
        at 
org.apache.flink.client.program.PackagedProgram.<init>(PackagedProgram.java:65) 
~[flink-dist-1.15.2.jar:1.15.2]
        at 
org.apache.flink.client.program.PackagedProgram$Builder.build(PackagedProgram.java:691)
 ~[flink-dist-1.15.2.jar:1.15.2]
        at 
org.apache.flink.runtime.webmonitor.handlers.utils.JarHandlerUtils$JarHandlerContext.toPackagedProgram(JarHandlerUtils.java:182)
 ~[flink-dist-1.15.2.jar:1.15.2]
   ```
   
   My job config is the following:
   
   ```
   apiVersion: flink.apache.org/v1beta1
   kind: FlinkSessionJob
   metadata:
     name: basic-session-job-only-example-noop2
   spec:
     deploymentName: basic-session-deployment-only-example
     job:
       entryClass: 
org.apache.flink.streaming.examples.statemachine.StateMachineExample
       parallelism: 4
       upgradeMode: stateless
   ```
   
   I've manually copied the jar file with the StateMachineExample class to the 
/opt/flink/lib folder.
   
   After analyzing the code it looks like Flink creates and utilizes a special 
classloader that only uses the uploaded jar and nothing else. Of course, it 
can't find anything since the jar is empty.
   
   What do I miss? 



-- 
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-unsubscr...@flink.apache.org

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

Reply via email to