Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SUPPORT] java.lang.NoClassDefFoundError: org/apache/hudi/com/fasterxml/jackson/module/scala/DefaultScalaModule$ when doing an Incremental CDC Query in 0.14.1 #10590

Closed
johnwilsonabartys opened this issue Jan 31, 2024 · 16 comments
Labels
class-not-found on-call-triaged priority:critical production down; pipelines stalled; Need help asap. reader-core

Comments

@johnwilsonabartys
Copy link

johnwilsonabartys commented Jan 31, 2024

Describe the problem you faced

When doing an Incremental CDC Query ('hoodie.datasource.query.incremental.format': "cdc"), the job crashes with the mentioned error, this only happens in 0.14.1 and not in 0.13.1 for the same dataset.

To Reproduce

Steps to reproduce the behavior:

  1. Write a Table with CDC Enabled (Can be either MoR or CoW, happens on both)
  2. Insert New Data (Can be upserts, inserts, or simply write in append mode)
  3. Query the Table with in Incremental Query with CDC Enabled (Default CDC Format Before and After Image)
  4. Crashes with mentioned error in 0.14.1, but successfully returns the appropriate Dataframe in 0.13.1 with the Schema:
root
 |-- op: string (nullable = true)
 |-- ts_ms: string (nullable = true)
 |-- before: string (nullable = true)
 |-- after: string (nullable = true)

Expected behavior

Should return the same Dataframe and not Crash in 0.14.1

Environment Description

  • Hudi version : 0.13.1 and 0.14.1

  • Spark version : 3.3

  • Hive version : N/A

  • Hadoop version : 3.3.2

  • Storage (HDFS/S3/GCS..) : S3 and Local MacOS File System

  • Running on Docker? (yes/no) : No

Additional context

This original came from us trying to run Hudi in AWS Glue. We were running Hudi 0.13.1 in Glue 4.0 without any problems but a new QA feature to test Deletes being captured was deployed, and it started causing errors of "There should be a CDC Log File", We found out by reading through the repo, this was a bug that was occurring that was reported here https://github.com/apache/hudi/issues/9987

And that it was fixed in 0.14.1. Since Glue runs with Hudi 0.13.1 and spark 3.3 in Glue 4.0. We went ahead and downloaded and manually ran the hudi-bundle3.3 for 0.14.1, since Glue 4.0 runs with 3.3. And that's when we got the mentioned error of the Missing Class. To start "narrowing down problems". I went ahead and attempted to recreate the problem locally with a smaller test sample. And was able to reproduce it exactly the same. using the HUDI Quickstart Utils.

I leave attached the exact same code that was ran.

Some Notes:

  1. In 0.14.1 the error also occurs when using the new "hudi_table_changes" function when trying to do it with CDC
from typing import Any

from pyspark import Row
from pyspark.sql import SparkSession
from pyspark.sql.functions import col

spark = SparkSession.builder \
    .appName("Hudi Basics") \
    .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") \
    .config("spark.jars.packages", "org.apache.hudi:hudi-spark3.3-bundle_2.12:0.13.1") \  
    .config("spark.sql.extensions", "org.apache.spark.sql.hudi.HoodieSparkSessionExtension") \
    .config("spark.sql.legacy.timeParserPolicy", "LEGACY") \
    .getOrCreate()

sc = spark.sparkContext

table_name = "hudi_trips_cdc"
base_path = "XXXXXX"  # Replace for whatever path
quickstart_utils = sc._jvm.org.apache.hudi.QuickstartUtils
dataGen = quickstart_utils.DataGenerator()

inserts = sc._jvm.org.apache.hudi.QuickstartUtils.convertToStringList(dataGen.generateInserts(10))


def create_df():
    df = spark.read.json(spark.sparkContext.parallelize(inserts, 2))
    return df


def write_data():
    df = create_df()

    hudi_options = {
        "hoodie.table.name": table_name,
        "hoodie.datasource.write.recordkey.field": "uuid",
        "hoodie.datasource.write.table.type": "MERGE_ON_READ",  # This can be either MoR or CoW and the error will still happen
        "hoodie.datasource.write.partitionpath.field": "partitionpath",
        "hoodie.datasource.write.table.name": table_name,
        "hoodie.datasource.write.operation": "upsert",
        "hoodie.table.cdc.enabled": "true",  # This can be left enabled, and won"t affect anything unless actually queried as CDC
        "hoodie.datasource.write.precombine.field": "ts",
        "hoodie.upsert.shuffle.parallelism": 2,
        "hoodie.insert.shuffle.parallelism": 2
    }

    df.write.format("hudi") \
        .options(**hudi_options) \
        .mode("overwrite") \
        .save(base_path)


def update_data():
    updates = quickstart_utils.convertToStringList(dataGen.generateUpdates(10))
    df = spark.read.json(spark.sparkContext.parallelize(updates, 2))
    df.write \
        .format("hudi") \
        .mode("append") \
        .save(base_path)


def incremental_query():

    ordered_rows: list[Row] = spark.read \
        .format("hudi") \
        .load(base_path) \
        .select(col("_hoodie_commit_time").alias("commit_time")) \
        .orderBy(col("commit_time")) \
        .collect()

    commits: list[Any] = list(map(lambda row: row[0], ordered_rows))
    begin_time = commits[0]

    incremental_read_options = {
        'hoodie.datasource.query.incremental.format': "cdc",  # Uncomment this line to Query as CDC, crashes in 0.14.1
        'hoodie.datasource.query.type': 'incremental',
        'hoodie.datasource.read.begin.instanttime': begin_time,
    }

    trips_incremental_df = spark.read \
        .format("hudi") \
        .options(**incremental_read_options) \
        .load(base_path)

    # Error also occurs when using the "from_hudi_table_changes" in 0.14.1
    # sql_query = f""" SELECT * FROM hudi_table_changes ('{base_path}', 'cdc', 'earliest')"""
    # trips_incremental_df = spark.sql(sql_query)

    trips_incremental_df.show()
    trips_incremental_df.printSchema()


if __name__ == "__main__":
    write_data()
    update_data()
    incremental_query()

# Source: https://hudi.apache.org/docs/quick-start-guide

As long as you have spark 3.3 installed, you should be able to reproduce/re-run this with no issues.
Just replace the org.apache.hudi:hudi-spark3.3-bundle_2.12:0.13.1 with org.apache.hudi:hudi-spark3.3-bundle_2.12:0.14.1.

And Vice-Versa.

Stacktrace

Traceback (most recent call last):
  File "/Users/johng.wilsonnegroni/PycharmProjects/hudi_testing/venv/lib/python3.10/site-packages/pyspark/sql/utils.py", line 190, in deco
    return f(*a, **kw)
  File "/Users/johng.wilsonnegroni/PycharmProjects/hudi_testing/venv/lib/python3.10/site-packages/py4j/protocol.py", line 326, in get_return_value
    raise Py4JJavaError(
py4j.protocol.Py4JJavaError: An error occurred while calling o105.showString.
: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 78.0 failed 1 times, most recent failure: Lost task 0.0 in stage 78.0 (TID 110) (192.168.4.100 executor driver): java.lang.NoClassDefFoundError: org/apache/hudi/com/fasterxml/jackson/module/scala/DefaultScalaModule$
	at org.apache.hudi.cdc.HoodieCDCRDD$CDCFileGroupIterator.<init>(HoodieCDCRDD.scala:237)
	at org.apache.hudi.cdc.HoodieCDCRDD.compute(HoodieCDCRDD.scala:101)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:365)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:329)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:365)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:329)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:365)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:329)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
	at org.apache.spark.scheduler.Task.run(Task.scala:136)
	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:548)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1504)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:551)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:750)
Caused by: java.lang.ClassNotFoundException: org.apache.hudi.com.fasterxml.jackson.module.scala.DefaultScalaModule$
	at java.net.URLClassLoader.findClass(URLClassLoader.java:387)
	at java.lang.ClassLoader.loadClass(ClassLoader.java:418)
	at java.lang.ClassLoader.loadClass(ClassLoader.java:351)
	... 18 more

Driver stacktrace:
	at org.apache.spark.scheduler.DAGScheduler.failJobAndIndependentStages(DAGScheduler.scala:2672)
	at org.apache.spark.scheduler.DAGScheduler.$anonfun$abortStage$2(DAGScheduler.scala:2608)
	at org.apache.spark.scheduler.DAGScheduler.$anonfun$abortStage$2$adapted(DAGScheduler.scala:2607)
	at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62)
	at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55)
	at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49)
	at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:2607)
	at org.apache.spark.scheduler.DAGScheduler.$anonfun$handleTaskSetFailed$1(DAGScheduler.scala:1182)
	at org.apache.spark.scheduler.DAGScheduler.$anonfun$handleTaskSetFailed$1$adapted(DAGScheduler.scala:1182)
	at scala.Option.foreach(Option.scala:407)
	at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:1182)
	at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:2860)
	at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2802)
	at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2791)
	at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:49)
	at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:952)
	at org.apache.spark.SparkContext.runJob(SparkContext.scala:2228)
	at org.apache.spark.SparkContext.runJob(SparkContext.scala:2249)
	at org.apache.spark.SparkContext.runJob(SparkContext.scala:2268)
	at org.apache.spark.sql.execution.SparkPlan.executeTake(SparkPlan.scala:506)
	at org.apache.spark.sql.execution.SparkPlan.executeTake(SparkPlan.scala:459)
	at org.apache.spark.sql.execution.CollectLimitExec.executeCollect(limit.scala:48)
	at org.apache.spark.sql.Dataset.collectFromPlan(Dataset.scala:3868)
	at org.apache.spark.sql.Dataset.$anonfun$head$1(Dataset.scala:2863)
	at org.apache.spark.sql.Dataset.$anonfun$withAction$2(Dataset.scala:3858)
	at org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:510)
	at org.apache.spark.sql.Dataset.$anonfun$withAction$1(Dataset.scala:3856)
	at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$6(SQLExecution.scala:109)
	at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:169)
	at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:95)
	at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:779)
	at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
	at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3856)
	at org.apache.spark.sql.Dataset.head(Dataset.scala:2863)
	at org.apache.spark.sql.Dataset.take(Dataset.scala:3084)
	at org.apache.spark.sql.Dataset.getRows(Dataset.scala:288)
	at org.apache.spark.sql.Dataset.showString(Dataset.scala:327)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:498)
	at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)
	at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)
	at py4j.Gateway.invoke(Gateway.java:282)
	at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
	at py4j.commands.CallCommand.execute(CallCommand.java:79)
	at py4j.ClientServerConnection.waitForCommands(ClientServerConnection.java:182)
	at py4j.ClientServerConnection.run(ClientServerConnection.java:106)
	at java.lang.Thread.run(Thread.java:750)
Caused by: java.lang.NoClassDefFoundError: org/apache/hudi/com/fasterxml/jackson/module/scala/DefaultScalaModule$
	at org.apache.hudi.cdc.HoodieCDCRDD$CDCFileGroupIterator.<init>(HoodieCDCRDD.scala:237)
	at org.apache.hudi.cdc.HoodieCDCRDD.compute(HoodieCDCRDD.scala:101)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:365)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:329)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:365)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:329)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:365)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:329)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
	at org.apache.spark.scheduler.Task.run(Task.scala:136)
	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:548)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1504)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:551)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	... 1 more
Caused by: java.lang.ClassNotFoundException: org.apache.hudi.com.fasterxml.jackson.module.scala.DefaultScalaModule$
	at java.net.URLClassLoader.findClass(URLClassLoader.java:387)
	at java.lang.ClassLoader.loadClass(ClassLoader.java:418)
	at java.lang.ClassLoader.loadClass(ClassLoader.java:351)
	... 18 more
@ad1happy2go
Copy link
Collaborator

Thanks for raising this @johnwilsonabartys . You are correct. With 0.14.1 version we are facing this issue while doing incremental CDC read. Interestingly with 0.14.0, we are not facing this issue.

Raised JIRA to track - https://issues.apache.org/jira/browse/HUDI-7360

@codope codope added priority:critical production down; pipelines stalled; Need help asap. reader-core class-not-found on-call-triaged labels Jan 31, 2024
@VitoMakarevich
Copy link
Contributor

VitoMakarevich commented Feb 5, 2024

The same happens with streaming source - since HoodieSourceOffset has
import com.fasterxml.jackson.module.scala.DefaultScalaModule.
As for 0.14.1 bundle - it has only
com.fasterxml.jackson.module.afterburner from jackson.

basically these 2 places - is where this module is used

@Tyler-Rendina
Copy link

Is there a way to manually add the class after importing the spark bundle?

@blackcheckren
Copy link

blackcheckren commented Mar 8, 2024

I also encountered the same problem. Under the error log information and the suggestion of that friend, I saw that the problem seemed to be that there was a configuration missing in pom.xml when packaging and compiling. I added the following line in the pom.xml file, and no problem occurred after testing.
image
image

@Tyler-Rendina
Copy link

I got it to compile, bootstrapped the spark bundle, hive sync, and aws bundle to emr. Now getting java.lang.ClassNotFoundException: Class com.amazonaws.glue.catalog.metastore.AWSGlueDataCatalogHiveClientFactory not found

@VitoMakarevich
Copy link
Contributor

VitoMakarevich commented Mar 8, 2024 via email

@Tyler-Rendina
Copy link

Tyler-Rendina commented Mar 12, 2024

Final note, apologies for the amount of posts, but this may help EMR users with Glue as their Hive service.

Make sure to build Hudi using Java 8, if you are on ARM use something like Azul OpenJDK and export $JAVA_HOME as the provided path, i.e., /Library/Java/JavaVirtualMachines/zulu-8.jdk/Contents/Home.

Once you upload your jars to s3, bootstrap such as:

sudo chown -R hadoop:root /usr/lib/hudi
sudo chmod -R ugo+rw /usr/lib/hudi
aws s3 cp s3://BUCKET/jars/hudi-aws-bundle-0.14.1.jar /usr/lib/hudi
aws s3 cp s3://BUCKET/jars/hudi-spark3.3-bundle_2.12-0.14.1.jar /usr/lib/hudi
sudo ln -sf /usr/lib/hudi/hudi-aws-bundle-0.14.1.jar hudi-aws-bundle.jar
sudo ln -sf /usr/lib/hudi/hudi-spark3.3-bundle_2.12-0.14.1.jar hudi-spark3.3-bundle.jar

To use your custom built Hudi package, conform to your bootstrap paths in the following spark submit command element:

"--jars",
"/usr/lib/hudi/hudi-aws-bundle-0.14.1.jar,/usr/lib/hudi/hudi-spark3.3-bundle_2.12-0.14.1.jar,",
"--conf",
"spark.driver.extraClassPath=/usr/lib/hudi/hudi-aws-bundle-0.14.1.jar:/usr/lib/hudi/hudi-spark3.3-bundle_2.12-0.14.1.jar",
"--conf",
"spark.executor.extraClassPath=/usr/lib/hudi/hudi-aws-bundle-0.14.1.jar:/usr/lib/hudi/hudi-spark3.3-bundle_2.12-0.14.1.jar",

Now, you have overwritten your extraClassPath, append the following to both configurations:
:/usr/lib/hadoop-lzo/lib/*:/usr/lib/hadoop/hadoop-aws.jar:/usr/share/aws/aws-java-sdk/*:/usr/share/aws/emr/goodies/lib/emr-spark-goodies.jar:/usr/share/aws/emr/security/conf:/usr/share/aws/emr/security/lib/*:/usr/share/aws/redshift/jdbc/RedshiftJDBC.jar:/usr/share/aws/redshift/spark-redshift/lib/*:/usr/share/aws/hmclient/lib/aws-glue-datacatalog-spark-client.jar:/usr/share/java/Hive-JSON-Serde/hive-openx-serde.jar:/usr/share/aws/sagemaker-spark-sdk/lib/sagemaker-spark-sdk.jar:/usr/share/aws/emr/s3select/lib/emr-s3-select-spark-connector.jar:/docker/usr/lib/hadoop-lzo/lib/*:/docker/usr/lib/hadoop/hadoop-aws.jar:/docker/usr/share/aws/aws-java-sdk/*:/docker/usr/share/aws/emr/goodies/lib/emr-spark-goodies.jar:/docker/usr/share/aws/emr/security/conf:/docker/usr/share/aws/emr/security/lib/*:/docker/usr/share/aws/redshift/jdbc/RedshiftJDBC.jar:/docker/usr/share/aws/redshift/spark-redshift/lib/*:/docker/usr/share/aws/hmclient/lib/aws-glue-datacatalog-spark-client.jar:/docker/usr/share/java/Hive-JSON-Serde/hive-openx-serde.jar:/docker/usr/share/aws/sagemaker-spark-sdk/lib/sagemaker-spark-sdk.jar:/docker/usr/share/aws/emr/s3select/lib/emr-s3-select-spark-connector.jar

@Tyler-Rendina
Copy link

Tyler-Rendina commented Mar 12, 2024

While I can kick off backfills, they eventually fail along side streams with java.lang.NoSuchMethodError: com.amazonaws.transform.JsonUnmarshallerContext.getCurrentToken()Lcom/amazonaws/thirdparty/jackson/core/JsonToken;

Per #5053
I just added /usr/share/aws/aws-java-sdk/aws-java-sdk-bundle-1.12.446.jar to my jars, this is for emr 6.11.1, you can look up which bundle version is right for your emr version.

@VitoMakarevich
Copy link
Contributor

This is fixed in
#10877

@codope codope closed this as completed May 15, 2024
@Gatsby-Lee
Copy link

why is it required to set these?
is it really required?

> "--conf",
> "spark.driver.extraClassPath=/usr/lib/hudi/hudi-aws-bundle-0.14.1.jar:/usr/lib/hudi/hudi-spark3.3-bundle_2.12-0.14.1.jar",
> "--conf",
> "spark.executor.extraClassPath=/usr/lib/hudi/hudi-aws-bundle-0.14.1.jar:/usr/lib/hudi/hudi-spark3.3-bundle_2.12-0.14.1.jar",

@VitoMakarevich
Copy link
Contributor

why is it required to set these? is it really required?

> "--conf",
> "spark.driver.extraClassPath=/usr/lib/hudi/hudi-aws-bundle-0.14.1.jar:/usr/lib/hudi/hudi-spark3.3-bundle_2.12-0.14.1.jar",
> "--conf",
> "spark.executor.extraClassPath=/usr/lib/hudi/hudi-aws-bundle-0.14.1.jar:/usr/lib/hudi/hudi-spark3.3-bundle_2.12-0.14.1.jar",

You can try without it. Basically /usr/lib/hudi/hudi-aws-bundle-0.14.1.jar includes minor things like AwsGlueCatalogSyncTool which behaves better then default if changed(https://hudi.apache.org/docs/configurations/#hoodiemetasyncclienttoolclass) on huge number of partitions.

@Tyler-Rendina
Copy link

why is it required to set these? is it really required?

> "--conf",
> "spark.driver.extraClassPath=/usr/lib/hudi/hudi-aws-bundle-0.14.1.jar:/usr/lib/hudi/hudi-spark3.3-bundle_2.12-0.14.1.jar",
> "--conf",
> "spark.executor.extraClassPath=/usr/lib/hudi/hudi-aws-bundle-0.14.1.jar:/usr/lib/hudi/hudi-spark3.3-bundle_2.12-0.14.1.jar",

You can try without it. Basically /usr/lib/hudi/hudi-aws-bundle-0.14.1.jar includes minor things like AwsGlueCatalogSyncTool which behaves better then default if changed(https://hudi.apache.org/docs/configurations/#hoodiemetasyncclienttoolclass) on huge number of partitions.

On the spark config side: Any dependencies off of the default spark path will require setting the extra class path in two places so both the driver and executor containers can see them. You can use blob statements to make this more concise if there are no conflicts.

My comments on this thread are based around the EMR environment so the long class path after the fact is also required if using Glue metastore.

@Gatsby-Lee
Copy link

Gatsby-Lee commented Jun 12, 2024

@Tyler-Rendina

why is it required to set these? is it really required?

> "--conf",
> "spark.driver.extraClassPath=/usr/lib/hudi/hudi-aws-bundle-0.14.1.jar:/usr/lib/hudi/hudi-spark3.3-bundle_2.12-0.14.1.jar",
> "--conf",
> "spark.executor.extraClassPath=/usr/lib/hudi/hudi-aws-bundle-0.14.1.jar:/usr/lib/hudi/hudi-spark3.3-bundle_2.12-0.14.1.jar",

You can try without it. Basically /usr/lib/hudi/hudi-aws-bundle-0.14.1.jar includes minor things like AwsGlueCatalogSyncTool which behaves better then default if changed(https://hudi.apache.org/docs/configurations/#hoodiemetasyncclienttoolclass) on huge number of partitions.

On the spark config side: Any dependencies off of the default spark path will require setting the extra class path in two places so both the driver and executor containers can see them. You can use blob statements to make this more concise if there are no conflicts.

My comments on this thread are based around the EMR environment so the long class path after the fact is also required if using Glue metastore.

Thank you very much for your comment.

I am on Amazon EMR on EKS env as well.
I've been using --jar option only to load and use the Hudi bundles from either Maven or Amazon EMR.
( The JARs bundles either on S3 or in the Amazon EMR Image. )

What I am curious is --conf is required along with --jar?

Thank you

@Gatsby-Lee
Copy link

@VitoMakarevich

Thank you for your response.

I've been using --jar only to load Hudi bundles ( haven't modified the "spark.driver.extraClassPath" or "spark.executor.extraClassPath" when using other Hudi bundles which are different from the shipped one in the Amazon EMR.

That's why I am curios if setting "spark.driver.extraClassPath" or "spark.executor.extraClassPath" are required along with --jar

Thank you

@Tyler-Rendina
Copy link

EMR on EKS gave me issues and I switched to EMR on EC2 about a year ago, probably needed to do the same thing done here. Planning to use something like kubeflow in the future.

When you spin up a job and look at the environment vars in the spark UI you can ctrl + f for 'classpath' and see what the key value options are. If the location of the jar is not on the classpath then it must be specified in addition to the initial classpath specified in that var of the spark UI.

In this case I was using --packages to get them directly from Maven, then I tried --jar and ran into version issues, hence the custom build above. This led to using the custom jars from s3. I imagine writing a more robust bootstrap script may have mitigated the issue. I also wanted to make sure the jar selected for hudi was not the preinstalled version in the particular case.

@Gatsby-Lee
Copy link

@Tyler-Rendina

EMR on EKS gave me issues and I switched to EMR on EC2 about a year ago, probably needed to do the same thing done here. Planning to use something like kubeflow in the future.

When you spin up a job and look at the environment vars in the spark UI you can ctrl + f for 'classpath' and see what the key value options are. If the location of the jar is not on the classpath then it must be specified in addition to the initial classpath specified in that var of the spark UI.

In this case I was using --packages to get them directly from Maven, then I tried --jar and ran into version issues, hence the custom build above. This led to using the custom jars from s3. I imagine writing a more robust bootstrap script may have mitigated the issue. I also wanted to make sure the jar selected for hudi was not the preinstalled version in the particular case.

Thank you very much for the detailed explanation.
Let me check the environment vars in the spark UI you can ctrl + f for 'classpath'

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
class-not-found on-call-triaged priority:critical production down; pipelines stalled; Need help asap. reader-core
Projects
Archived in project
Development

No branches or pull requests

7 participants