Re: Spark-on-Yarn ClassNotFound Exception

2022-12-18 Thread Hariharan
Hi scrypso,

Sorry for the late reply. Yes, I did mean spark.driver.extraClassPath. I
was able to work around this issue by removing the need for an extra class,
but I'll investigate along these lines nonetheless.

Thanks again for all your help!

On Thu, Dec 15, 2022 at 9:56 PM scrypso  wrote:

> Hmm, did you mean spark.*driver*.extraClassPath? That is very odd then -
> if you check the logs directory for the driver (on the cluster) I think
> there should be a launch container log, where you can see the exact command
> used to start the JVM (at the very end), and a line starting "export
> CLASSPATH" - you can double check that your jar looks to be included
> correctly there. If it is I think you have a really "interesting" issue on
> your hands!
>
> - scrypso
>
> On Wed, Dec 14, 2022, 05:17 Hariharan  wrote:
>
>> Hi scrypso,
>>
>> Thanks for the help so far, and I think you're definitely on to something
>> here. I tried loading the class as you suggested with the code below:
>>
>> try {
>> 
>> Thread.currentThread().getContextClassLoader().loadClass(MyS3ClientFactory.class.getCanonicalName());
>> logger.info("Loaded custom class");
>> } catch (ClassNotFoundException e) {
>> logger.error("Unable to load class", e);
>> }
>> return spark.read().option("mode", 
>> "DROPMALFORMED").format("avro").load();
>>
>> I am able to load the custom class as above
>> *2022-12-14 04:12:34,158 INFO  [Driver] utils.S3Reader - Loaded custom
>> class*
>>
>> But the spark.read code below it tries to initialize the s3 client and is
>> not able to load the same class.
>>
>> I tried adding
>> *--conf spark.executor.extraClassPath=myjar*
>>
>> as well, but no luck :-(
>>
>> Thanks again!
>>
>> On Tue, Dec 13, 2022 at 10:09 PM scrypso  wrote:
>>
>>> I'm on my phone, so can't compare with the Spark source, but that looks
>>> to me like it should be well after the ctx loader has been set. You could
>>> try printing the classpath of the loader
>>> Thread.currentThread().getThreadContextClassLoader(), or try to load your
>>> class from that yourself to see if you get the same error.
>>>
>>> Can you see which thread is throwing the exception? If it is a different
>>> thread than the "main" application thread it might not have the thread ctx
>>> loader set correctly. I can't see any of your classes in the stacktrace - I
>>> assume that is because of your scrubbing, but it could also be because this
>>> is run in separate thread without ctx loader set.
>>>
>>> It also looks like Hadoop is caching the FileSystems somehow - perhaps
>>> you can create the S3A filesystem yourself and hope it picks that up? (Wild
>>> guess, no idea if that works or how hard it would be.)
>>>
>>>
>>> On Tue, Dec 13, 2022, 17:29 Hariharan  wrote:
>>>
 Thanks for the response, scrypso! I will try adding the extraClassPath
 option. Meanwhile, please find the full stack trace below (I have
 masked/removed references to proprietary code)

 java.lang.RuntimeException: java.lang.RuntimeException:
 java.lang.ClassNotFoundException: Class foo.bar.MyS3ClientFactory not found
 at
 org.apache.hadoop.conf.Configuration.getClass(Configuration.java:2720)
 at
 org.apache.hadoop.fs.s3a.S3AFileSystem.bindAWSClient(S3AFileSystem.java:888)
 at
 org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:542)
 at
 org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3469)
 at
 org.apache.hadoop.fs.FileSystem.access$300(FileSystem.java:174)
 at
 org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3574)
 at
 org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3521)
 at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:540)
 at org.apache.hadoop.fs.Path.getFileSystem(Path.java:365)
 at
 org.apache.spark.sql.execution.datasources.DataSource$.$anonfun$checkAndGlobPathIfNecessary$1(DataSource.scala:752)
 at scala.collection.immutable.List.map(List.scala:293)
 at
 org.apache.spark.sql.execution.datasources.DataSource$.checkAndGlobPathIfNecessary(DataSource.scala:750)
 at
 org.apache.spark.sql.execution.datasources.DataSource.checkAndGlobPathIfNecessary(DataSource.scala:579)
 at
 org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:408)
 at
 org.apache.spark.sql.DataFrameReader.loadV1Source(DataFrameReader.scala:228)
 at
 org.apache.spark.sql.DataFrameReader.$anonfun$load$2(DataFrameReader.scala:210)
 at scala.Option.getOrElse(Option.scala:189)
 at
 org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:210)

 Thanks again!

 On Tue, Dec 13, 2022 at 9:52 PM scrypso  wrote:

> Two ideas you could try:
>
> You can try spark.driver.extraClassPath as well. Spark loads the

Re: Spark-on-Yarn ClassNotFound Exception

2022-12-15 Thread scrypso
Hmm, did you mean spark.*driver*.extraClassPath? That is very odd then - if
you check the logs directory for the driver (on the cluster) I think there
should be a launch container log, where you can see the exact command used
to start the JVM (at the very end), and a line starting "export CLASSPATH"
- you can double check that your jar looks to be included correctly there.
If it is I think you have a really "interesting" issue on your hands!

- scrypso

On Wed, Dec 14, 2022, 05:17 Hariharan  wrote:

> Hi scrypso,
>
> Thanks for the help so far, and I think you're definitely on to something
> here. I tried loading the class as you suggested with the code below:
>
> try {
> 
> Thread.currentThread().getContextClassLoader().loadClass(MyS3ClientFactory.class.getCanonicalName());
> logger.info("Loaded custom class");
> } catch (ClassNotFoundException e) {
> logger.error("Unable to load class", e);
> }
> return spark.read().option("mode", 
> "DROPMALFORMED").format("avro").load();
>
> I am able to load the custom class as above
> *2022-12-14 04:12:34,158 INFO  [Driver] utils.S3Reader - Loaded custom
> class*
>
> But the spark.read code below it tries to initialize the s3 client and is
> not able to load the same class.
>
> I tried adding
> *--conf spark.executor.extraClassPath=myjar*
>
> as well, but no luck :-(
>
> Thanks again!
>
> On Tue, Dec 13, 2022 at 10:09 PM scrypso  wrote:
>
>> I'm on my phone, so can't compare with the Spark source, but that looks
>> to me like it should be well after the ctx loader has been set. You could
>> try printing the classpath of the loader
>> Thread.currentThread().getThreadContextClassLoader(), or try to load your
>> class from that yourself to see if you get the same error.
>>
>> Can you see which thread is throwing the exception? If it is a different
>> thread than the "main" application thread it might not have the thread ctx
>> loader set correctly. I can't see any of your classes in the stacktrace - I
>> assume that is because of your scrubbing, but it could also be because this
>> is run in separate thread without ctx loader set.
>>
>> It also looks like Hadoop is caching the FileSystems somehow - perhaps
>> you can create the S3A filesystem yourself and hope it picks that up? (Wild
>> guess, no idea if that works or how hard it would be.)
>>
>>
>> On Tue, Dec 13, 2022, 17:29 Hariharan  wrote:
>>
>>> Thanks for the response, scrypso! I will try adding the extraClassPath
>>> option. Meanwhile, please find the full stack trace below (I have
>>> masked/removed references to proprietary code)
>>>
>>> java.lang.RuntimeException: java.lang.RuntimeException:
>>> java.lang.ClassNotFoundException: Class foo.bar.MyS3ClientFactory not found
>>> at
>>> org.apache.hadoop.conf.Configuration.getClass(Configuration.java:2720)
>>> at
>>> org.apache.hadoop.fs.s3a.S3AFileSystem.bindAWSClient(S3AFileSystem.java:888)
>>> at
>>> org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:542)
>>> at
>>> org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3469)
>>> at
>>> org.apache.hadoop.fs.FileSystem.access$300(FileSystem.java:174)
>>> at
>>> org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3574)
>>> at
>>> org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3521)
>>> at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:540)
>>> at org.apache.hadoop.fs.Path.getFileSystem(Path.java:365)
>>> at
>>> org.apache.spark.sql.execution.datasources.DataSource$.$anonfun$checkAndGlobPathIfNecessary$1(DataSource.scala:752)
>>> at scala.collection.immutable.List.map(List.scala:293)
>>> at
>>> org.apache.spark.sql.execution.datasources.DataSource$.checkAndGlobPathIfNecessary(DataSource.scala:750)
>>> at
>>> org.apache.spark.sql.execution.datasources.DataSource.checkAndGlobPathIfNecessary(DataSource.scala:579)
>>> at
>>> org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:408)
>>> at
>>> org.apache.spark.sql.DataFrameReader.loadV1Source(DataFrameReader.scala:228)
>>> at
>>> org.apache.spark.sql.DataFrameReader.$anonfun$load$2(DataFrameReader.scala:210)
>>> at scala.Option.getOrElse(Option.scala:189)
>>> at
>>> org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:210)
>>>
>>> Thanks again!
>>>
>>> On Tue, Dec 13, 2022 at 9:52 PM scrypso  wrote:
>>>
 Two ideas you could try:

 You can try spark.driver.extraClassPath as well. Spark loads the user's
 jar in a child classloader, so Spark/Yarn/Hadoop can only see your classes
 reflectively. Hadoop's Configuration should use the thread ctx classloader,
 and Spark should set that to the loader that loads your jar. The
 extraClassPath option just adds jars directly to the Java command that
 creates the driver/executor.

 I can't immediately tell how your error might arise, 

Re: Spark-on-Yarn ClassNotFound Exception

2022-12-13 Thread Hariharan
Hi scrypso,

Thanks for the help so far, and I think you're definitely on to something
here. I tried loading the class as you suggested with the code below:

try {

Thread.currentThread().getContextClassLoader().loadClass(MyS3ClientFactory.class.getCanonicalName());
logger.info("Loaded custom class");
} catch (ClassNotFoundException e) {
logger.error("Unable to load class", e);
}
return spark.read().option("mode",
"DROPMALFORMED").format("avro").load();

I am able to load the custom class as above
*2022-12-14 04:12:34,158 INFO  [Driver] utils.S3Reader - Loaded custom
class*

But the spark.read code below it tries to initialize the s3 client and is
not able to load the same class.

I tried adding
*--conf spark.executor.extraClassPath=myjar*

as well, but no luck :-(

Thanks again!

On Tue, Dec 13, 2022 at 10:09 PM scrypso  wrote:

> I'm on my phone, so can't compare with the Spark source, but that looks to
> me like it should be well after the ctx loader has been set. You could try
> printing the classpath of the loader
> Thread.currentThread().getThreadContextClassLoader(), or try to load your
> class from that yourself to see if you get the same error.
>
> Can you see which thread is throwing the exception? If it is a different
> thread than the "main" application thread it might not have the thread ctx
> loader set correctly. I can't see any of your classes in the stacktrace - I
> assume that is because of your scrubbing, but it could also be because this
> is run in separate thread without ctx loader set.
>
> It also looks like Hadoop is caching the FileSystems somehow - perhaps you
> can create the S3A filesystem yourself and hope it picks that up? (Wild
> guess, no idea if that works or how hard it would be.)
>
>
> On Tue, Dec 13, 2022, 17:29 Hariharan  wrote:
>
>> Thanks for the response, scrypso! I will try adding the extraClassPath
>> option. Meanwhile, please find the full stack trace below (I have
>> masked/removed references to proprietary code)
>>
>> java.lang.RuntimeException: java.lang.RuntimeException:
>> java.lang.ClassNotFoundException: Class foo.bar.MyS3ClientFactory not found
>> at
>> org.apache.hadoop.conf.Configuration.getClass(Configuration.java:2720)
>> at
>> org.apache.hadoop.fs.s3a.S3AFileSystem.bindAWSClient(S3AFileSystem.java:888)
>> at
>> org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:542)
>> at
>> org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3469)
>> at org.apache.hadoop.fs.FileSystem.access$300(FileSystem.java:174)
>> at
>> org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3574)
>> at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3521)
>> at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:540)
>> at org.apache.hadoop.fs.Path.getFileSystem(Path.java:365)
>> at
>> org.apache.spark.sql.execution.datasources.DataSource$.$anonfun$checkAndGlobPathIfNecessary$1(DataSource.scala:752)
>> at scala.collection.immutable.List.map(List.scala:293)
>> at
>> org.apache.spark.sql.execution.datasources.DataSource$.checkAndGlobPathIfNecessary(DataSource.scala:750)
>> at
>> org.apache.spark.sql.execution.datasources.DataSource.checkAndGlobPathIfNecessary(DataSource.scala:579)
>> at
>> org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:408)
>> at
>> org.apache.spark.sql.DataFrameReader.loadV1Source(DataFrameReader.scala:228)
>> at
>> org.apache.spark.sql.DataFrameReader.$anonfun$load$2(DataFrameReader.scala:210)
>> at scala.Option.getOrElse(Option.scala:189)
>> at
>> org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:210)
>>
>> Thanks again!
>>
>> On Tue, Dec 13, 2022 at 9:52 PM scrypso  wrote:
>>
>>> Two ideas you could try:
>>>
>>> You can try spark.driver.extraClassPath as well. Spark loads the user's
>>> jar in a child classloader, so Spark/Yarn/Hadoop can only see your classes
>>> reflectively. Hadoop's Configuration should use the thread ctx classloader,
>>> and Spark should set that to the loader that loads your jar. The
>>> extraClassPath option just adds jars directly to the Java command that
>>> creates the driver/executor.
>>>
>>> I can't immediately tell how your error might arise, unless there is
>>> some timing issue with the Spark and Hadoop setup. Can you share the full
>>> stacktrace of the ClassNotFound exception? That might tell us when Hadoop
>>> is looking up this class.
>>>
>>> Good luck!
>>> - scrypso
>>>
>>>
>>> On Tue, Dec 13, 2022, 17:05 Hariharan  wrote:
>>>
 Missed to mention it above, but just to add, the error is coming from
 the driver. I tried using *--driver-class-path /path/to/my/jar* as
 well, but no luck.

 Thanks!

 On Mon, Dec 12, 2022 at 4:21 PM Hariharan 
 wrote:

> Hello folks,
>
> I have a spark app with a custom implementation of

Re: Spark-on-Yarn ClassNotFound Exception

2022-12-13 Thread scrypso
I'm on my phone, so can't compare with the Spark source, but that looks to
me like it should be well after the ctx loader has been set. You could try
printing the classpath of the loader
Thread.currentThread().getThreadContextClassLoader(), or try to load your
class from that yourself to see if you get the same error.

Can you see which thread is throwing the exception? If it is a different
thread than the "main" application thread it might not have the thread ctx
loader set correctly. I can't see any of your classes in the stacktrace - I
assume that is because of your scrubbing, but it could also be because this
is run in separate thread without ctx loader set.

It also looks like Hadoop is caching the FileSystems somehow - perhaps you
can create the S3A filesystem yourself and hope it picks that up? (Wild
guess, no idea if that works or how hard it would be.)


On Tue, Dec 13, 2022, 17:29 Hariharan  wrote:

> Thanks for the response, scrypso! I will try adding the extraClassPath
> option. Meanwhile, please find the full stack trace below (I have
> masked/removed references to proprietary code)
>
> java.lang.RuntimeException: java.lang.RuntimeException:
> java.lang.ClassNotFoundException: Class foo.bar.MyS3ClientFactory not found
> at
> org.apache.hadoop.conf.Configuration.getClass(Configuration.java:2720)
> at
> org.apache.hadoop.fs.s3a.S3AFileSystem.bindAWSClient(S3AFileSystem.java:888)
> at
> org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:542)
> at
> org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3469)
> at org.apache.hadoop.fs.FileSystem.access$300(FileSystem.java:174)
> at
> org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3574)
> at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3521)
> at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:540)
> at org.apache.hadoop.fs.Path.getFileSystem(Path.java:365)
> at
> org.apache.spark.sql.execution.datasources.DataSource$.$anonfun$checkAndGlobPathIfNecessary$1(DataSource.scala:752)
> at scala.collection.immutable.List.map(List.scala:293)
> at
> org.apache.spark.sql.execution.datasources.DataSource$.checkAndGlobPathIfNecessary(DataSource.scala:750)
> at
> org.apache.spark.sql.execution.datasources.DataSource.checkAndGlobPathIfNecessary(DataSource.scala:579)
> at
> org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:408)
> at
> org.apache.spark.sql.DataFrameReader.loadV1Source(DataFrameReader.scala:228)
> at
> org.apache.spark.sql.DataFrameReader.$anonfun$load$2(DataFrameReader.scala:210)
> at scala.Option.getOrElse(Option.scala:189)
> at
> org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:210)
>
> Thanks again!
>
> On Tue, Dec 13, 2022 at 9:52 PM scrypso  wrote:
>
>> Two ideas you could try:
>>
>> You can try spark.driver.extraClassPath as well. Spark loads the user's
>> jar in a child classloader, so Spark/Yarn/Hadoop can only see your classes
>> reflectively. Hadoop's Configuration should use the thread ctx classloader,
>> and Spark should set that to the loader that loads your jar. The
>> extraClassPath option just adds jars directly to the Java command that
>> creates the driver/executor.
>>
>> I can't immediately tell how your error might arise, unless there is some
>> timing issue with the Spark and Hadoop setup. Can you share the full
>> stacktrace of the ClassNotFound exception? That might tell us when Hadoop
>> is looking up this class.
>>
>> Good luck!
>> - scrypso
>>
>>
>> On Tue, Dec 13, 2022, 17:05 Hariharan  wrote:
>>
>>> Missed to mention it above, but just to add, the error is coming from
>>> the driver. I tried using *--driver-class-path /path/to/my/jar* as
>>> well, but no luck.
>>>
>>> Thanks!
>>>
>>> On Mon, Dec 12, 2022 at 4:21 PM Hariharan 
>>> wrote:
>>>
 Hello folks,

 I have a spark app with a custom implementation of
 *fs.s3a.s3.client.factory.impl* which is packaged into the same jar.
 Output of *jar tf*

 *2620 Mon Dec 12 11:23:00 IST 2022 aws/utils/MyS3ClientFactory.class*

 However when I run the my spark app with spark-submit in cluster mode,
 it fails with the following error:

 *java.lang.RuntimeException: java.lang.RuntimeException:
 java.lang.ClassNotFoundException: Class aws.utils.MyS3ClientFactory not
 found*

 I tried:
 1. passing in the jar to the *--jars* option (with the local path)
 2. Passing in the jar to *spark.yarn.jars* option with an HDFS path

 but still the same error.

 Any suggestions on what I'm missing?

 Other pertinent details:
 Spark version: 3.3.0
 Hadoop version: 3.3.4

 Command used to run the app
 */spark/bin/spark-submit --class MyMainClass --deploy-mode cluster
 --master yarn  --conf spark.executor.instances=6   

Re: Spark-on-Yarn ClassNotFound Exception

2022-12-13 Thread Hariharan
Thanks for the response, scrypso! I will try adding the extraClassPath
option. Meanwhile, please find the full stack trace below (I have
masked/removed references to proprietary code)

java.lang.RuntimeException: java.lang.RuntimeException:
java.lang.ClassNotFoundException: Class foo.bar.MyS3ClientFactory not found
at
org.apache.hadoop.conf.Configuration.getClass(Configuration.java:2720)
at
org.apache.hadoop.fs.s3a.S3AFileSystem.bindAWSClient(S3AFileSystem.java:888)
at
org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:542)
at
org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3469)
at org.apache.hadoop.fs.FileSystem.access$300(FileSystem.java:174)
at
org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3574)
at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3521)
at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:540)
at org.apache.hadoop.fs.Path.getFileSystem(Path.java:365)
at
org.apache.spark.sql.execution.datasources.DataSource$.$anonfun$checkAndGlobPathIfNecessary$1(DataSource.scala:752)
at scala.collection.immutable.List.map(List.scala:293)
at
org.apache.spark.sql.execution.datasources.DataSource$.checkAndGlobPathIfNecessary(DataSource.scala:750)
at
org.apache.spark.sql.execution.datasources.DataSource.checkAndGlobPathIfNecessary(DataSource.scala:579)
at
org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:408)
at
org.apache.spark.sql.DataFrameReader.loadV1Source(DataFrameReader.scala:228)
at
org.apache.spark.sql.DataFrameReader.$anonfun$load$2(DataFrameReader.scala:210)
at scala.Option.getOrElse(Option.scala:189)
at
org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:210)

Thanks again!

On Tue, Dec 13, 2022 at 9:52 PM scrypso  wrote:

> Two ideas you could try:
>
> You can try spark.driver.extraClassPath as well. Spark loads the user's
> jar in a child classloader, so Spark/Yarn/Hadoop can only see your classes
> reflectively. Hadoop's Configuration should use the thread ctx classloader,
> and Spark should set that to the loader that loads your jar. The
> extraClassPath option just adds jars directly to the Java command that
> creates the driver/executor.
>
> I can't immediately tell how your error might arise, unless there is some
> timing issue with the Spark and Hadoop setup. Can you share the full
> stacktrace of the ClassNotFound exception? That might tell us when Hadoop
> is looking up this class.
>
> Good luck!
> - scrypso
>
>
> On Tue, Dec 13, 2022, 17:05 Hariharan  wrote:
>
>> Missed to mention it above, but just to add, the error is coming from the
>> driver. I tried using *--driver-class-path /path/to/my/jar* as well, but
>> no luck.
>>
>> Thanks!
>>
>> On Mon, Dec 12, 2022 at 4:21 PM Hariharan  wrote:
>>
>>> Hello folks,
>>>
>>> I have a spark app with a custom implementation of
>>> *fs.s3a.s3.client.factory.impl* which is packaged into the same jar.
>>> Output of *jar tf*
>>>
>>> *2620 Mon Dec 12 11:23:00 IST 2022 aws/utils/MyS3ClientFactory.class*
>>>
>>> However when I run the my spark app with spark-submit in cluster mode,
>>> it fails with the following error:
>>>
>>> *java.lang.RuntimeException: java.lang.RuntimeException:
>>> java.lang.ClassNotFoundException: Class aws.utils.MyS3ClientFactory not
>>> found*
>>>
>>> I tried:
>>> 1. passing in the jar to the *--jars* option (with the local path)
>>> 2. Passing in the jar to *spark.yarn.jars* option with an HDFS path
>>>
>>> but still the same error.
>>>
>>> Any suggestions on what I'm missing?
>>>
>>> Other pertinent details:
>>> Spark version: 3.3.0
>>> Hadoop version: 3.3.4
>>>
>>> Command used to run the app
>>> */spark/bin/spark-submit --class MyMainClass --deploy-mode cluster
>>> --master yarn  --conf spark.executor.instances=6   /path/to/my/jar*
>>>
>>> TIA!
>>>
>>


Re: Spark-on-Yarn ClassNotFound Exception

2022-12-13 Thread scrypso
Two ideas you could try:

You can try spark.driver.extraClassPath as well. Spark loads the user's jar
in a child classloader, so Spark/Yarn/Hadoop can only see your classes
reflectively. Hadoop's Configuration should use the thread ctx classloader,
and Spark should set that to the loader that loads your jar. The
extraClassPath option just adds jars directly to the Java command that
creates the driver/executor.

I can't immediately tell how your error might arise, unless there is some
timing issue with the Spark and Hadoop setup. Can you share the full
stacktrace of the ClassNotFound exception? That might tell us when Hadoop
is looking up this class.

Good luck!
- scrypso


On Tue, Dec 13, 2022, 17:05 Hariharan  wrote:

> Missed to mention it above, but just to add, the error is coming from the
> driver. I tried using *--driver-class-path /path/to/my/jar* as well, but
> no luck.
>
> Thanks!
>
> On Mon, Dec 12, 2022 at 4:21 PM Hariharan  wrote:
>
>> Hello folks,
>>
>> I have a spark app with a custom implementation of
>> *fs.s3a.s3.client.factory.impl* which is packaged into the same jar.
>> Output of *jar tf*
>>
>> *2620 Mon Dec 12 11:23:00 IST 2022 aws/utils/MyS3ClientFactory.class*
>>
>> However when I run the my spark app with spark-submit in cluster mode, it
>> fails with the following error:
>>
>> *java.lang.RuntimeException: java.lang.RuntimeException:
>> java.lang.ClassNotFoundException: Class aws.utils.MyS3ClientFactory not
>> found*
>>
>> I tried:
>> 1. passing in the jar to the *--jars* option (with the local path)
>> 2. Passing in the jar to *spark.yarn.jars* option with an HDFS path
>>
>> but still the same error.
>>
>> Any suggestions on what I'm missing?
>>
>> Other pertinent details:
>> Spark version: 3.3.0
>> Hadoop version: 3.3.4
>>
>> Command used to run the app
>> */spark/bin/spark-submit --class MyMainClass --deploy-mode cluster
>> --master yarn  --conf spark.executor.instances=6   /path/to/my/jar*
>>
>> TIA!
>>
>


Re: Spark-on-Yarn ClassNotFound Exception

2022-12-13 Thread Hariharan
Missed to mention it above, but just to add, the error is coming from the
driver. I tried using *--driver-class-path /path/to/my/jar* as well, but no
luck.

Thanks!

On Mon, Dec 12, 2022 at 4:21 PM Hariharan  wrote:

> Hello folks,
>
> I have a spark app with a custom implementation of
> *fs.s3a.s3.client.factory.impl* which is packaged into the same jar.
> Output of *jar tf*
>
> *2620 Mon Dec 12 11:23:00 IST 2022 aws/utils/MyS3ClientFactory.class*
>
> However when I run the my spark app with spark-submit in cluster mode, it
> fails with the following error:
>
> *java.lang.RuntimeException: java.lang.RuntimeException:
> java.lang.ClassNotFoundException: Class aws.utils.MyS3ClientFactory not
> found*
>
> I tried:
> 1. passing in the jar to the *--jars* option (with the local path)
> 2. Passing in the jar to *spark.yarn.jars* option with an HDFS path
>
> but still the same error.
>
> Any suggestions on what I'm missing?
>
> Other pertinent details:
> Spark version: 3.3.0
> Hadoop version: 3.3.4
>
> Command used to run the app
> */spark/bin/spark-submit --class MyMainClass --deploy-mode cluster
> --master yarn  --conf spark.executor.instances=6   /path/to/my/jar*
>
> TIA!
>


Spark-on-Yarn ClassNotFound Exception

2022-12-12 Thread Hariharan
Hello folks,

I have a spark app with a custom implementation of
*fs.s3a.s3.client.factory.impl* which is packaged into the same jar.
Output of *jar tf*

*2620 Mon Dec 12 11:23:00 IST 2022 aws/utils/MyS3ClientFactory.class*

However when I run the my spark app with spark-submit in cluster mode, it
fails with the following error:

*java.lang.RuntimeException: java.lang.RuntimeException:
java.lang.ClassNotFoundException: Class aws.utils.MyS3ClientFactory not
found*

I tried:
1. passing in the jar to the *--jars* option (with the local path)
2. Passing in the jar to *spark.yarn.jars* option with an HDFS path

but still the same error.

Any suggestions on what I'm missing?

Other pertinent details:
Spark version: 3.3.0
Hadoop version: 3.3.4

Command used to run the app
*/spark/bin/spark-submit --class MyMainClass --deploy-mode cluster --master
yarn  --conf spark.executor.instances=6   /path/to/my/jar*

TIA!