Databricks Spark Knowledge Base
Databricks Spark Knowledge Base
1. Knowledgebase
2. Best Practices
i. Avoid GroupByKey
ii. Don't copy all elements of a large RDD to the driver
iii. Gracefully Dealing with Bad Input Data
3. General Troubleshooting
i. Job aborted due to stage failure: Task not serializable:
ii. Missing Dependencies in Jar Files
iii. Error running start-all.sh - Connection refused
iv. Network connectivity issues between Spark components
4. Performance & Optimization
i. How Many Partitions Does An RDD Have?
ii. Data Locality
5. Spark Streaming
i. ERROR OneForOneStrategy
Best Practices
Avoid GroupByKey
Don't copy all elements of a large RDD to the driver
Gracefully Dealing with Bad Input Data
Avoid GroupByKey
Let's look at two different ways to compute word counts, one using
reduceByKey
groupByKey :
While both of these functions will produce the correct answer, the
reduceByKey
dataset. That's because Spark knows it can combine output with a common key on each partition before shuffling the
data.
Look at the diagram below to understand what happens with
reduceByKey . Notice
the same key are combined (by using the lamdba function passed into
reduceByKey )
lamdba function is called again to reduce all the values from each partition to produce one final result.
groupByKey
- all the key-value pairs are shuffled around. This is a lot of unnessary data
You can imagine that for a much larger dataset size, the difference in the amount of data you are shuffling becomes more
exaggerated and different between
reduceByKey
and
groupByKey .
groupByKey :
can be used when you are combining elements but your return type differs from your input value type.
merges the values for each key using an associative function and a neutral "zero value".
Collect will attempt to copy every single element in the RDD onto the single driver program, and then run out of memory
and crash.
Instead, you can make sure the number of elements you return is capped by calling
take
or
takeSample
, or perhaps
If you really do need every one of these values of the RDD and the data is too big to fit into memory, you can write out the
RDD to files or export the RDD to a database that is large enough to hold all the data.
filter
flatMap
map
function where you can try fixing the input but fall back to discarding
If we tried to input this set of json strings to a sqlContext, it would clearly fail due to the malformed input's.
sqlContext.jsonRDD(input_rdd).registerTempTable("valueTable")
# The above command will throw an error.
Instead, let's try fixing the input with this python function:
def try_correct_json(json_string):
try:
# First check if the json is okay.
json.loads(json_string)
return [json_string]
except ValueError:
try:
# If not, try correcting it by adding a ending brace.
try_to_correct_json = json_string + "}"
json.loads(try_to_correct_json)
return [try_to_correct_json]
except ValueError:
# The malformed json input can't be recovered, drop this input.
return []
Now, we can apply that function to fix our input and try again. This time we will succeed to read in three inputs:
corrected_input_rdd = input_rdd.flatMap(try_correct_json)
sqlContext.jsonRDD(corrected_input_rdd).registerTempTable("valueTable")
sqlContext.sql("select * from valueTable").collect()
# Returns [Row(value=1), Row(value=2), Row(value=3)]
General Troubleshooting
Job aborted due to stage failure: Task not serializable:
Missing Dependencies in Jar Files
Error running start-all.sh - Connection refused
Network connectivity issues between Spark components
org.apache.spark.SparkException: Job aborted due to stage failure: Task not serializable: java.io.NotSerializableException: ...
The above error can be triggered when you intialize a variable on the driver (master), but then try to use it on one of the
workers. In that case, Spark Streaming will try to serialize the object to send it over to the worker, and fail if the object is
not serializable. Consider the following code snippet:
This will trigger that error. Here are some ideas to fix this error:
Serializable the class
Declare the instance only within the lambda function passed in map.
Make the NotSerializable object as a static and create it once per machine.
Call rdd.forEachPartition and create the NotSerializable object in there like this:
rdd.forEachPartition(iter -> {
NotSerializable notSerializable = new NotSerializable();
// ...Now process iter
});
. Spark dependencies should be marked as provided since they are already on the Spark
cluster. You may also exclude other jars that you have installed on your worker machines.
Here is an example Maven pom.xml file that creates an uber jar with all the code in that project and includes the
common-cli dependency, but not any of the Spark libraries.:
<project>
<groupId>com.databricks.apps.logs</groupId>
<artifactId>log-analyzer</artifactId>
<modelVersion>4.0.0</modelVersion>
<name>Databricks Spark Logs Analyzer</name>
<packaging>jar</packaging>
<version>1.0</version>
<repositories>
<repository>
<id>Akka repository</id>
<url>http://repo.akka.io/releases</url>
</repository>
</repositories>
<dependencies>
<dependency> <!-- Spark -->
<groupId>org.apache.spark</groupId>
<artifactId>spark-core_2.10</artifactId>
<version>1.1.0</version>
<scope>provided</scope>
</dependency>
<dependency> <!-- Spark SQL -->
<groupId>org.apache.spark</groupId>
<artifactId>spark-sql_2.10</artifactId>
<version>1.1.0</version>
<scope>provided</scope>
</dependency>
<dependency> <!-- Spark Streaming -->
<groupId>org.apache.spark</groupId>
<artifactId>spark-streaming_2.10</artifactId>
<version>1.1.0</version>
<scope>provided</scope>
</dependency>
<dependency> <!-- Command Line Parsing -->
<groupId>commons-cli</groupId>
<artifactId>commons-cli</artifactId>
<version>1.2</version>
</dependency>
</dependencies>
<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
<version>2.3.2</version>
<configuration>
<source>1.8</source>
<target>1.8</target>
</configuration>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-shade-plugin</artifactId>
<version>2.3</version>
<executions>
<execution>
<phase>package</phase>
<goals>
<goal>shade</goal>
</goals>
</execution>
</executions>
<configuration>
<filters>
<filter>
<artifact>*:*</artifact>
<excludes>
<exclude>META-INF/*.SF</exclude>
<exclude>META-INF/*.DSA</exclude>
<exclude>META-INF/*.RSA</exclude>
</excludes>
</filter>
</filters>
<finalName>uber-${project.artifactId}-${project.version}</finalName>
</configuration>
</plugin>
</plugins>
</build>
</project>
% sh start-all.sh
starting org.apache.spark.deploy.master.Master, logging to ...
localhost: ssh: connect to host localhost port 22: Connection refused
You need to enable "Remote Login" for your machine. From System Preferences, select Sharing, and then turn on
Remote Login.
If the driver is able to connect to the master but the master is unable to communicate back to the driver, then the
Master's logs may record multiple attempts to connect even though the driver will report that it could not connect:
In this case, the master reports that it has successfully registered an application, but if the acknowledgment of this
registration fails to be received by the driver, then the driver will automatically make several attempts to re-connect
before eventually giving up and failing. As a result, the master web UI may report multiple failed applications even
though only a single SparkContext was created.
Recomendations
If you are experiencing any of the errors described above:
Check that the workers and drivers are configured to connect to the Spark master on the exact address listed in the
Spark master web UI / logs.
Set
SPARK_LOCAL_IP
spark.driver.host
SPARK_LOCAL_IP
configuration property.
environment variable is set to a hostname, then this hostname will be used. If
spark.driver.port
configuration property.
InetAddress.getLocalHost
method.
SPARK_LOCAL_IP
is
--host
The
SPARK_MASTER_HOST
If the
, or
-h
SPARK_LOCAL_IP
--ip
or
-i
Master
Master
or
Worker
process.
).
SPARK_LOCAL_IP
InetAddress.getLocalHost
method.
Port:
The
--port
The
SPARK_MASTER_PORT
, or
-p
Master
SPARK_WORKER_PORT
respectively).
An ephemeral port chosen by the OS.
or
Worker
process.
Master
and
Worker
is
In Spark's application UI, you can see from the following screenshot that the "Total Tasks" represents the number of
partitions:
scala> someRDD.setName("toy").cache
res2: someRDD.type = toy ParallelCollectionRDD[0] at parallelize at <console>:12
scala> someRDD.map(x => x).collect
res3: Array[Int] = Array(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33
Note from the screenshot that there are four partitions cached.
In the python API, there is a method for explicitly listing the number of partitions:
Note in the examples above, the number of partitions was intentionally set to 30 upon initialization.
Data Locality
Spark is a data parallel processing framework, which means it will execute tasks as close to where the data lives as
possible (i.e. minimize data transfer).
Checking Locality
The best means of checking whether a task ran locally is to inspect a given stage in the Spark UI. Notice from the
screenshot below that the "Locality Level" column displays which locality a given task ran with.
local --> node local --> rack local --> Any). For more information on these parameters, see the
the Scheduling section of the Application Configration docs.
spark.locality.*
configs in
Spark Streaming
ERROR OneForOneStrategy
ERROR OneForOneStrategy
If you enable checkpointing in Spark Streaming, then objects used in a function called in forEachRDD should be
Serializable. Otherwise, there will be an "ERROR OneForOneStrategy: ... java.io.NotSerializableException:
This code will run if you make one of these changes to it:
Turn off checkpointing by removing the
jssc.checkpoint
line.