Introduction
The data infrastructure team at Thumbtack has just completed the process of migrating all of our production Spark jobs from Spark 1.6.1 to Spark 2.0.0. We deployed Spark 2 and all of our updated jobs to production after a few weeks of testing in our development environment.
As we went through the Spark 2 migration process, we realized that many of the issues we encountered were not well documented, but had relatively straightforward solutions. In this post, we wanted to share some of the common problems we found during the migration, and the solutions that we identified.
Common Issues
Mapping over DataFrames
Issue
Spark job fails with the error:
Unable to find encoder for type stored in a Dataset. Primitive types (Int, String, etc) and Product types (case classes) are supported by importing spark.implicits._ Support for serializing other types will be added in future releases.
Resolution
You can no longer map directly over the rows of DataFrames, as they are now simply a type alias for Dataset[Row]. In Spark 2, to map over a Dataset[T], an encoder for type T must be available in the current context. Encoders are available for standard types and for case classes, but not for Row since it is a generic container type. If you wish to map over the rows of a DataFrame df, you should now convert it to an RDD first:
Before:
df.map { row: Row => … }
After:
df.rdd.map { row: Row => … }
Over time, you can likely migrate usage like the above from DataFrame to Dataset, which will give you stricter typing on loading data and improved performance. See the Spark documentation for details.
Writing Collections Types to Parquet
Issue
Spark job fails with the error:
org.apache.spark.sql.AnalysisException: cannot resolve 'wrapoption(staticinvoke(class org.apache.spark.sql.catalyst. <<huge arg list here>>)' due to data type mismatch: argument 1 requires scala.collection.immutable.Map type, however, 'staticinvoke(class org.apache.spark.sql.catalyst. <<huge arg list here>>)' is of scala.collection.Map type.;
Resolution
Spark requires all Maps that are being written to Parquet to be scala.collection.Map instances instead of immutable.Map; see SPARK-14767.
In any case class records that you will be persisting to Parquet, change instances of Map to scala.collection.Map. You may also need to add .toMap in places where you use the case class instance to convert map types.
Before:
case class Foo(obj: Map[String, String])
After:
case class Foo(obj: scala.collection.Map[String, String])
HiveContext is deprecated
In Spark 2, HiveContext is deprecated. Replace all usage with an instantiation of the singleton SparkSession:
val spark: SparkSession = SparkSession.builder
.config(conf)
.enableHiveSupport()
.getOrCreate()
Most functionality of HiveContext is now available directly on the SparkSession instance. Note that, if you need them, SparkContext and SQLContext are now properties of SparkSession:
val sc = spark.sparkContext
val sqlContext = spark.sqlContext
Address Already in Use Errors
Issue:
INFO - 16/09/07 08:03:42 WARN AbstractLifeCycle: FAILED ServerConnector@7342e05d{HTTP/1.1}{0.0.0.0:4040}: java.net.BindException: Address already in use
INFO - java.net.BindException: Address already in use
INFO - at sun.nio.ch.Net.bind0(Native Method)
INFO - at sun.nio.ch.Net.bind(Net.java:433)
This is a YARN job launching issue that was present in Spark prior to Spark 2, but the error spew was much quieter. It occurs when you start multiple Spark masters on a single host, and your Spark jobs are launched in YARN client mode.
Resolution:
Randomize the Spark port on launch. We do the following:
val conf = new SparkConf()
conf.set("spark.ui.port", (4040 + scala.util.Random.nextInt(1000)).toString)
val spark = SparkSession.builder.config(conf)
Alternatively, you can launch in YARN cluster mode. However, client mode is easiest to block on job completion with a workflow manager (we use Airflow).
The post Migrating our Scala codebase to Spark 2 appeared first on Thumbtack Engineering.