Migration Guide: Spark Core
- Upgrading from Core 3.5.3 to 3.5.4
- Upgrading from Core 3.4 to 3.5
- Upgrading from Core 3.3 to 3.4
- Upgrading from Core 3.2 to 3.3
- Upgrading from Core 3.1 to 3.2
- Upgrading from Core 3.0 to 3.1
- Upgrading from Core 2.4 to 3.0
Upgrading from Core 3.5.3 to 3.5.4
- Since Spark 3.5.4, when reading files hits
org.apache.hadoop.security.AccessControlExceptionandorg.apache.hadoop.hdfs.BlockMissingException, the exception will be thrown and fail the task, even ifspark.files.ignoreCorruptFilesis set totrue.
Upgrading from Core 3.4 to 3.5
-
Since Spark 3.5,
spark.yarn.executor.failuresValidityIntervalis deprecated. Usespark.executor.failuresValidityIntervalinstead. -
Since Spark 3.5,
spark.yarn.max.executor.failuresis deprecated. Usespark.executor.maxNumFailuresinstead.
Upgrading from Core 3.3 to 3.4
-
Since Spark 3.4, Spark driver will own
PersistentVolumnClaims and try to reuse if they are not assigned to live executors. To restore the behavior before Spark 3.4, you can setspark.kubernetes.driver.ownPersistentVolumeClaimtofalseandspark.kubernetes.driver.reusePersistentVolumeClaimtofalse. -
Since Spark 3.4, Spark driver will track shuffle data when dynamic allocation is enabled without shuffle service. To restore the behavior before Spark 3.4, you can set
spark.dynamicAllocation.shuffleTracking.enabledtofalse. -
Since Spark 3.4, Spark will try to decommission cached RDD and shuffle blocks if both
spark.decommission.enabledandspark.storage.decommission.enabledare true. To restore the behavior before Spark 3.4, you can set bothspark.storage.decommission.rddBlocks.enabledandspark.storage.decommission.shuffleBlocks.enabledtofalse. -
Since Spark 3.4, Spark will use RocksDB store if
spark.history.store.hybridStore.enabledis true. To restore the behavior before Spark 3.4, you can setspark.history.store.hybridStore.diskBackendtoLEVELDB.
Upgrading from Core 3.2 to 3.3
- Since Spark 3.3, Spark migrates its log4j dependency from 1.x to 2.x because log4j 1.x has reached end of life and is no longer supported by the community. Vulnerabilities reported after August 2015 against log4j 1.x were not checked and will not be fixed. Users should rewrite original log4j properties files using log4j2 syntax (XML, JSON, YAML, or properties format). Spark rewrites the
conf/log4j.properties.templatewhich is included in Spark distribution, toconf/log4j2.properties.templatewith log4j2 properties format.
Upgrading from Core 3.1 to 3.2
-
Since Spark 3.2,
spark.scheduler.allocation.filesupports read remote file using hadoop filesystem which means if the path has no scheme Spark will respect hadoop configuration to read it. To restore the behavior before Spark 3.2, you can specify the local scheme forspark.scheduler.allocation.filee.g.file:///path/to/file. -
Since Spark 3.2,
spark.hadoopRDD.ignoreEmptySplitsis set totrueby default which means Spark will not create empty partitions for empty input splits. To restore the behavior before Spark 3.2, you can setspark.hadoopRDD.ignoreEmptySplitstofalse. -
Since Spark 3.2,
spark.eventLog.compression.codecis set tozstdby default which means Spark will not fallback to usespark.io.compression.codecanymore. -
Since Spark 3.2,
spark.storage.replication.proactiveis enabled by default which means Spark tries to replenish in case of the loss of cached RDD block replicas due to executor failures. To restore the behavior before Spark 3.2, you can setspark.storage.replication.proactivetofalse. -
In Spark 3.2,
spark.launcher.childConectionTimeoutis deprecated (typo) though still works. Usespark.launcher.childConnectionTimeoutinstead. -
In Spark 3.2, support for Apache Mesos as a resource manager is deprecated and will be removed in a future version.
-
In Spark 3.2, Spark will delete K8s driver service resource when the application terminates by itself. To restore the behavior before Spark 3.2, you can set
spark.kubernetes.driver.service.deleteOnTerminationtofalse.
Upgrading from Core 3.0 to 3.1
-
In Spark 3.0 and below,
SparkContextcan be created in executors. Since Spark 3.1, an exception will be thrown when creatingSparkContextin executors. You can allow it by setting the configurationspark.executor.allowSparkContextwhen creatingSparkContextin executors. -
In Spark 3.0 and below, Spark propagated the Hadoop classpath from
yarn.application.classpathandmapreduce.application.classpathinto the Spark application submitted to YARN when Spark distribution is with the built-in Hadoop. Since Spark 3.1, it does not propagate anymore when the Spark distribution is with the built-in Hadoop in order to prevent the failure from the different transitive dependencies picked up from the Hadoop cluster such as Guava and Jackson. To restore the behavior before Spark 3.1, you can setspark.yarn.populateHadoopClasspathtotrue.
Upgrading from Core 2.4 to 3.0
-
The
org.apache.spark.ExecutorPlugininterface and related configuration has been replaced withorg.apache.spark.api.plugin.SparkPlugin, which adds new functionality. Plugins using the old interface must be modified to extend the new interfaces. Check the Monitoring guide for more details. -
Deprecated method
TaskContext.isRunningLocallyhas been removed. Local execution was removed and it always has returnedfalse. -
Deprecated method
shuffleBytesWritten,shuffleWriteTimeandshuffleRecordsWritteninShuffleWriteMetricshave been removed. Instead, usebytesWritten,writeTimeandrecordsWrittenrespectively. -
Deprecated method
AccumulableInfo.applyhave been removed because creatingAccumulableInfois disallowed. -
Deprecated accumulator v1 APIs have been removed and please use v2 APIs instead.
-
Event log file will be written as UTF-8 encoding, and Spark History Server will replay event log files as UTF-8 encoding. Previously Spark wrote the event log file as default charset of driver JVM process, so Spark History Server of Spark 2.x is needed to read the old event log files in case of incompatible encoding.
-
A new protocol for fetching shuffle blocks is used. It’s recommended that external shuffle services be upgraded when running Spark 3.0 apps. You can still use old external shuffle services by setting the configuration
spark.shuffle.useOldFetchProtocoltotrue. Otherwise, Spark may run into errors with messages likeIllegalArgumentException: Unexpected message type: <number>. -
SPARK_WORKER_INSTANCESis deprecated in Standalone mode. It’s recommended to launch multiple executors in one worker and launch one worker per node instead of launching multiple workers per node and launching one executor per worker.