Advertisement
Advertisement

More Related Content

Slideshows for you(20)

Advertisement

More from Databricks(20)

Advertisement

Care and Feeding of Catalyst Optimizer

  1. Care and Feeding of Spark SQL Catalyst Optimizer Rose Toomey Senior Software Engineer, AI Group @ Bloomberg
  2. About me A software engineer making up for a lifetime of trying to fix data problems at the wrong end of the pipeline. Working in the AI Group @ Bloomberg since April 2020. Some talks ▪ Spark Summit Europe 2019: Apache Spark At Scale in the Cloud ▪ Scale By the Bay 2019: Serverless Spark with GraalVM ▪ ScalaDays 2020 2021: The Field Mechanic’s Guide to Integration Testing An Apache Spark App
  3. Agenda Catalyst Optimizer Extensible query optimization for Apache Spark Case Studies So this happened Query Planning Pharmacy Prescriptive measures for blinkenlights Exciting new features Spark 3 sparkle for Spark SQL
  4. First, some data Columns pruned to make examples easier to read. +--------+---------+------+---------------+------+--------------------+--------+-------------+---------+ |OBJECTID| Borough|Number| Street|PSSite| ParkName|Postcode|Congressional| PSStatus| +--------+---------+------+---------------+------+--------------------+--------+-------------+---------+ | 4969542| Queens| 138-| 102 AV|Street| null| 11435| 5|Populated| | 5119146| Bronx| null| null| Park| Virginia Park| 0| 0| Empty| | 5573655| Bronx| null| null| Park| El Flamboyan Garden| 10455| 15|Populated| | 5580543| Bronx| null| null| Park| Hispanos Unidos| 10460| 15|Populated| | 7181904| Queens| null| null| Park|Grand Central Par...| 0| 0|Populated| | 3639151| Brooklyn| 625| PUTNAM AVENUE|Street| null| 0| 0|Populated| | 5439258| Queens| null| null| Park|Hilton Holiday Ga...| 0| 0|Populated| | 7229688| Queens| null| null| Park|Hilton Holiday Ga...| 0| 0|Populated| | 5512267| Brooklyn| 557|COLUMBIA STREET|Street| null| 0| 0|Populated| | 5046117|Manhattan| null| null| Park|East River Esplanade| 0| 0|Populated| | 7231655| Queens| null| null| Park|Hilton Holiday Ga...| 0| 0|Populated| | 6478977| Brooklyn| null| null| Park|Belt Parkway/Shor...| 11214| 11|Populated| +--------+---------+------+---------------+------+--------------------+--------+-------------+---------+ NYC OpenData: Forestry Planting Spaces
  5. Represent, ingest, cache import org.apache.spark.sql.types._ val schema = new StructType() .add("OBJECTID", LongType) .add("Borough", StringType) // etc // limited columns to make query plans easier to read case class PlantingSpace( OBJECTID: Long, Borough: String, // etc ) val ds = spark .read .option("inferSchema", false) .option("header", true) .option("sep", ",") .schema(schema) .csv("/path/to/csv") .select( $"OBJECTID", $"Borough", // etc ) .as[PlantingSpace] ds.cache() ImportModel
  6. What are the “greenest” parts of NYC? import org.apache.spark.sql.functions._ val parks = ds .filter($"PSSite" === "Park") .filter($"Borough".isNotNull) .filter($"ParkName".isNotNull) .filter($"Postcode" =!= 0) .groupBy($"Borough", $"Postcode") .agg(countDistinct($"ParkName").alias("parkCount")) val streetSites = ds .filter($"PSSite" === "Street") .filter($"Borough".isNotNull) .filter($"Postcode" =!= 0) .groupBy($"Borough", $"Postcode") .agg(countDistinct($"Number", $"Street").alias("streetSiteCount")) val greenest = parks .join(streetSites, Seq("Postcode", "Borough")) .sort($"parkCount".desc, $"streetSiteCount".desc) .limit(10)
  7. But the results are the start of your questions > greenest.show() +--------+---------+---------+---------------+ |Postcode| Borough|parkCount|streetSiteCount| +--------+---------+---------+---------------+ | 11207| Brooklyn| 53| 4705| | 10009|Manhattan| 43| 1306| | 11212| Brooklyn| 37| 3970| | 11233| Brooklyn| 36| 4425| | 11211| Brooklyn| 32| 3214| | 11206| Brooklyn| 32| 2559| | 11208| Brooklyn| 31| 5740| | 10027|Manhattan| 31| 1618| | 10451| Bronx| 31| 994| | 10035|Manhattan| 30| 1044| +--------+---------+---------+---------------+ ✨ How did a declarative SQL query transform into something that could execute on a Spark cluster? That took almost 20 seconds - why?
  8. Catalyst Optimizer From “what” to “how”
  9. Catalyst goes to work val streetSites = ds .filter($"PSSite" === "Street") .filter($"Borough".isNotNull) .filter($"Postcode" =!= 0) .groupBy($"Borough", $"Postcode") .agg(countDistinct($"Number", $"Street") .alias("streetSiteCount")) .sort($"streetSiteCount".desc) .limit(10) Watch these colored pieces move through the query plans
  10. Getting the query plans // Prints the plans (logical and // physical) to the console for // debugging purposes. // If `extended` is false, prints only // the physical plan. // exciting new options for explain // available in Spark 3 💖 streetSites .explain(extended = true) Run .explain on querySpark UI > SQL tab (executed query plan)
  11. The what: parsing and analysis ▪ Logical plans are an abstraction of the computations required to perform the query that can be freely transformed. They aren’t concerned with how the cluster will execute the query. ▪ Parsed logical plan is valid syntax but unresolved references ▪ Analyzed logical plan is valid syntax and valid references Data Frame Metadata catalog Parsed Logical Plan Analyzed Logical Plan Analysis Parsing
  12. Parsed Logical Plan == Parsed Logical Plan == GlobalLimit 10 +- LocalLimit 10 +- Sort [streetSiteCount#1406L DESC NULLS LAST], true +- Aggregate [Borough#1287, Postcode#1300L], [Borough#1287, Postcode#1300L, count(distinct Number#1288, Street#1289) AS streetSiteCount#1406L] +- Filter NOT (Postcode#1300L = cast(0 as bigint)) +- Filter isnotnull(Borough#1287) +- Filter (PSSite#1290 = Street) +- Project [OBJECTID#1286L, Borough#1287, Number#1288, Street#1289, PSSite#1290, ParkName#1294, Postcode#1300L, Congressional#1304L, PSStatus#1306] +- Relation[OBJECTID#1286L,Borough#1287,Number#1288,Street#1289,PSSite#1290, PlantingSpaceOnStreet#1291,Width#1292,Length#1293,ParkName#1294,ParkZone#1295,CrossStreet1#1296,Cros sStreet2#1297,CommunityBoard#1298L,SanitationZone#1299,Postcode#1300L,CouncilDistrict#1301L,StateSen ate#1302L,StateAssembly#1303L,Congressional#1304L,PhysicalID#1305L,PSStatus#1306,Geometry#1307,Globa lID#1308,CreatedDate#1309,... 11 more fields] csv
  13. Analyzed Logical Plan == Analyzed Logical Plan == Borough: string, Postcode: bigint, streetSiteCount: bigint GlobalLimit 10 +- LocalLimit 10 +- Sort [streetSiteCount#1406L DESC NULLS LAST], true +- Aggregate [Borough#1287, Postcode#1300L], [Borough#1287, Postcode#1300L, count(distinct Number#1288, Street#1289) AS streetSiteCount#1406L] +- Filter NOT (Postcode#1300L = cast(0 as bigint)) +- Filter isnotnull(Borough#1287) +- Filter (PSSite#1290 = Street) +- Project [OBJECTID#1286L, Borough#1287, Number#1288, Street#1289, PSSite#1290, ParkName#1294, Postcode#1300L, Congressional#1304L, PSStatus#1306] +- Relation[OBJECTID#1286L,Borough#1287,Number#1288,Street#1289,PSSite#1290, PlantingSpaceOnStreet#1291,Width#1292,Length#1293,ParkName#1294,ParkZone#1295,CrossStreet1#1296,Cros sStreet2#1297,CommunityBoard#1298L,SanitationZone#1299,Postcode#1300L,CouncilDistrict#1301L,StateSen ate#1302L,StateAssembly#1303L,Congressional#1304L,PhysicalID#1305L,PSStatus#1306,Geometry#1307,Globa lID#1308,CreatedDate#1309,... 11 more fields] csv
  14. The better what: Optimization ▪ ▪ Optimized logical plan will become one or more physical plans ▪ Optimization checks for all the tasks which can be performed together in one stage (fusing map operations) ▪ If there’s more than one JOIN clause, decide how to order the execution of the query ▪ Predicate pushdown: evaluate FILTER clauses before any PROJECT Logical Plan Physical Plan Physical Plan Physical Plan Optimized Logical Plan Physical Planning Cache Manager Analysis
  15. Optimized Logical Plan == Optimized Logical Plan == GlobalLimit 10 +- LocalLimit 10 +- Sort [streetSiteCount#1406L DESC NULLS LAST], true +- Aggregate [Borough#1287, Postcode#1300L], [Borough#1287, Postcode#1300L, count(distinct Number#1288, Street#1289) AS streetSiteCount#1406L] +- Project [Borough#1287, Number#1288, Street#1289, Postcode#1300L] +- Filter ((((isnotnull(PSSite#1290) && isnotnull(Postcode#1300L)) && (PSSite#1290 = Street)) && isnotnull(Borough#1287)) && NOT (Postcode#1300L = 0)) +- InMemoryRelation [OBJECTID#1286L, Borough#1287, Number#1288, Street#1289, PSSite#1290, ParkName#1294, Postcode#1300L, Congressional#1304L, PSStatus#1306], StorageLevel(disk, memory, deserialized, 1 replicas) +- *(1) FileScan csv [OBJECTID#16L,Borough#17,Number#18,Street#19,PSSite#20,ParkName#24, Postcode#30L,Congressional#34L,PSStatus#36] Batched: false, DataFilters: [], Format: CSV, Location: InMemoryFileIndex[dbfs:/FileStore/tables/Forestry_Planting_Spaces_csv-3123a.gz], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<OBJECTID:bigint,Borough:string,Number:string,Street:string,PSSite:string,ParkName:string,P... * Whole stage code generation mark
  16. The how: physical plan to execution ▪ A Physical plan is executable: specifies how the optimized logical plan will be executed on the cluster ▪ Cost model estimates execution time and resources for each strategy ▪ Then code generation turns the best physical plan into a DAG of RDDs that can execute on the cluster RDD (DAG) RDD (DAG) RDD (DAG) RDD (DAGs) Code GenerationPlanner Physical Plan Physical Plan Physical Plan Cost Model Selected Physical Plan
  17. Physical Plan == Physical Plan == TakeOrderedAndProject(limit=10, orderBy=[streetSiteCount#1406L DESC NULLS LAST], output=[Borough#1287,Postcode#1300L,streetSiteCount#1406L]) +- *(3) HashAggregate(keys=[Borough#1287, Postcode#1300L], functions=[finalmerge_count(distinct merge count#1497L) AS count(Number#1288, Street#1289)#1405L], output=[Borough#1287, Postcode#1300L, streetSiteCount#1406L]) +- Exchange hashpartitioning(Borough#1287, Postcode#1300L, 200), [id=#1279] +- *(2) HashAggregate(keys=[Borough#1287, Postcode#1300L], functions=[partial_count(distinct Number#1288, Street#1289) AS count#1497L], output=[Borough#1287, Postcode#1300L, count#1497L]) +- *(2) HashAggregate(keys=[Borough#1287, Postcode#1300L, Number#1288, Street#1289], functions=[], output=[Borough#1287, Postcode#1300L, Number#1288, Street#1289]) +- Exchange hashpartitioning(Borough#1287, Postcode#1300L, Number#1288, Street#1289, 200), [id=#1274] +- *(1) HashAggregate(keys=[Borough#1287, Postcode#1300L, Number#1288, Street#1289], functions=[], output=[Borough#1287, Postcode#1300L, Number#1288, Street#1289]) +- *(1) Project [Borough#1287, Number#1288, Street#1289, Postcode#1300L] +- *(1) Filter ((((isnotnull(PSSite#1290) && isnotnull(Postcode#1300L)) && (PSSite#1290 = Street)) && isnotnull(Borough#1287)) && NOT (Postcode#1300L = 0)) +- InMemoryTableScan [Borough#1287, Number#1288, PSSite#1290, Postcode#1300L, Street#1289], [isnotnull(PSSite#1290), isnotnull(Postcode#1300L), (PSSite#1290 = Street), isnotnull(Borough#1287), NOT (Postcode#1300L = 0)] +- InMemoryRelation [OBJECTID#1286L, Borough#1287, Number#1288, Street#1289, PSSite#1290, ParkName#1294, Postcode#1300L, Congressional#1304L, PSStatus#1306], StorageLevel(disk, memory, deserialized, 1 replicas) +- *(1) FileScan csv [OBJECTID#16L,Borough#17,Number#18,Street#19,PSSite#20,ParkName#24, Postcode#30L,Congressional#34L,PSStatus#36] Batched: false, DataFilters: [], Format: CSV, Location: InMemoryFileIndex[dbfs:/FileStore/tables/Forestry_Planting_Spaces_csv-3123a.gz], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<OBJECTID:bigint,Borough:string,Number:string,Street:string, PSSite:string,ParkName:string,P...
  18. Spark UI SQL tab - first stage codegen details Correlates back to +- *(1) section of physical query plan ��
  19. Tales from Production Clusters So this happened.
  20. Case Studies So you know your Spark job has a problem. How do you know whether that problem is caused by query optimization? Two tales from production clusters... ▪ Groundhog Day ▪ Salmiak
  21. Case study 1: “Groundhog Day” A large Spark cluster stops working and idles for 45 minutes before coming back to life. ▪ No logs on master or executors during this time Once the cluster begins working again 🐌 ▪ CPU usage spikes, then falls ▪ Memory usage steadily rises ▪ Cluster runs out of memory ▪ Job dies The symptoms
  22. Diagnosing Groundhog Day 🕵‍♀ org.apache.spark.sql.Dataset.select(Dataset.scala:1361) org.apache.spark.sql.Dataset.withColumns(Dataset.scala:2326) org.apache.spark.sql.Dataset.withColumn(Dataset.scala:2293) ▪ Zeroing in on the issue in a thread dump from a running Spark driver requires timing and luck. So take a lot of them. ▪ Don’t give up! Most of the thread dump content will be irrelevant, innocuous, a dead end, or a red herring. ▪ But whatever is causing the issue is in there. withColumn, you say? When a Spark cluster “hangs”, take thread dumps until you get a good explanation why.
  23. Groundhog Day: the smoking gun import scala.concurrent.duration.FiniteDuration // hello friend, let's do some repetitive calculations // using a list of metrics that are columns in our dataset // and windowing back over our dataset over and over for different time periods val colNames = Seq("foo", "bar", "baz", /* etc, */ ) val periods = Seq(6.months, 12.months, 18.months, /* etc */) val metrics: List[(String, FiniteDuration)] = for { col <- colNames; p <- periods} yield (col, p) def doSomeCalc(ds: Dataframe, column: String, period: FiniteDuration): Column = // boring calc goes here // i like big query plans and i cannot lie metrics.foldLeft(ds) { case (ds, (colName, period)) => ds.withColumn(colName+"_someCalc_"+period, doSomeCalc(ds, column, period)) } TL;DR
  24. Groundhog Day: what happened The query optimizer cannot optimize chained calls to withColumn (or withColumnRenamed) the same way it optimizes .select! Each withColumn call ▪ internally checks to see if you’re renaming an existing column ▪ then calls .select with the new set of columns ▪ which creates a new Dataset ▪ which initiates an analysis of the query plan With complex query plans, repeated calls to withColumn can result in the query optimization process taking a long, long time. ⏳ The dose makes the poison
  25. Groundhog Day: practical demonstration println(s"ORIGINAL: ${streetSites.queryExecution.optimizedPlan.stats.simpleString}") val cols = ('A' to 'Z').map(_.toString) cols.foldLeft(streetSites.toDF()) { case (ds, name) => val modifiedDf = ds.withColumn(name, lit(name)) val plan = modifiedDf.queryExecution.optimizedPlan println(s"withColumn('${name}'): ${plan.stats.simpleString}") modifiedDf } ORIGINAL: sizeInBytes=10.5 MB, hints=none withColumn('A'): sizeInBytes=15.3 MB, hints=none withColumn('B'): sizeInBytes=20.1 MB, hints=none withColumn('C'): sizeInBytes=24.8 MB, hints=none // SNIP withColumn('W'): sizeInBytes=120.3 MB, hints=none withColumn('X'): sizeInBytes=125.1 MB, hints=none withColumn('Y'): sizeInBytes=129.9 MB, hints=none withColumn('Z'): sizeInBytes=134.7 MB, hints=none
  26. Bonus stacktrace: beware the mega-query ERROR AsyncEventQueue: Listener DBCEventLoggingListener threw an exception com.fasterxml.jackson.databind.JsonMappingException: Exceeded 2097152 bytes (current = 2101045) (through reference chain: org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart["physicalPlanDescr iption"]) Caused by: com.databricks.spark.util.LimitedOutputStream$LimitExceededException: Exceeded 2097152 bytes (current = 2101045) at com.databricks.spark.util.LimitedOutputStream.write(LimitedOutputStream.scala:45) The query plan so large Spark blew up trying to log it The text file for Charles Dickens’ David Copperfield, a 500+ page book, is 2,033,139 bytes on Project Gutenberg. This physical query plan was 67,906 bytes larger than David Copperfield when the logger - thankfully - gave up.
  27. Groundhog Day: the fix val ds: Dataframe = ??? val existingCols: Seq[Column] = ds.columns.toSeq // Seq[String] .map(col) // def col: String => Column val newColumns: Seq[(String, Column)] = metrics.map { case (colName, period) => val newColName = colName+"_someCalc_"+period doSomeCalc(ds, colName, period).as(newColName) } // just once :) ds.select((existingCols ++ newColumns): _*) // take a look at the query lineage going forward - caching should solve the problem // if not, truncate query lineage by checkpointing or writing out to cloud storage
  28. Case study 2: “Salmiak”* This Spark job has ▪ A very large dataset ▪ partitioned by a heavy key ▪ that has been salted to evenly distribute the workload ▪ That adds a column based on the heavy key using a UDF that is ▪ deterministic ▪ expensive ▪ Then goes on to run repeated calculations on this dataset The first run of the job paralyzes the Spark cluster and runs out of memory. The symptoms * if you weren’t expecting salty liquorice 🤯
  29. But first! Let’s talk about Spark and UDFs The Catalyst optimizer makes efficient use of UDFs that are both: ▪ Cheap ▪ Deterministic When optimizing, Catalyst could choose to: ▪ invoke the UDF multiple times per record when the original query only called the UDF once, or ▪ eliminate duplicate invocations in the original query See SPARK-17728 for a lively discussion into why.
  30. Salmiak: first try Partitions (each partition goes to a single executor) Salted keys A1 A2 An B1 B2 Z synthetic key for long tail Rows 100mm 100mm 85mm 102mm 92mm 75mm Cache hit ratio 99.9% 12% Outcome 💸 Idle 🔥 Insane skew Eventual OOM Proposal: give each executor a local cache. Expensive UDF will lazily cache its computations. We’re done here, right? 🍹💥
  31. Salmiak: second try Partitions (each partition goes to a single executor) Keys 🃏 🃏 🃏 🃏 🃏 🃏 Rows 97mm 104mm 65mm 101mm 96mm 111mm Hit ratio 45% 38% 52% 29% 😿 42% 37% 🐥 dataset in partitions of roughly equal size, but leaving key distribution to chance 💵 🔥 This approach seemed viable (cache maybe not worth it), but after ingestion the analysis bogged down. Eventually the cluster hung.
  32. Salmiak: post mortem ▪ Salting is meant to combat skew by distributing work evenly ▪ Caching is meant to amortize the work by lazily populating the results of the expensive UDF calculation ▪ When combined in this case, we got the inverse of what we wanted ▪ Despite even data distribution, skewed runtime on executors ▪ A bad cache hit rate in the UDF eats memory while adding complexity to the code ▪ If you don’t .cache the dataset after adding the column with the UDF, but before calling transformations like .filter, the expensive UDF could be called multiple times 🧨 as you continue to work with your dataset. See Apache Spark Jobs hang due to non-deterministic custom UDF.
  33. Salmiak: conclusions This UDF is Cheap Expensive Deterministic 🍒🍒🍒 Stand back and admire the optimizer doing what it does best. 💀 Cache the dataset after adding column and before transformations like .filter Non-deterministic 🤡 Mark .nonDeterministic Behaves as you’d expect. Prevents certain optimizations from being applied to query, which may affect performance (see SPARK-27969 for example). The fix
  34. Query Planning Pharmacy Prescriptive measures for blinkenlights
  35. Query Planning Pharmacy ▪ Slow query ▪ How to investigate query planning issues ▪ Code cache is full ▪ Codegen generated method >64K ▪ Debugging codegen ▪ Disabling codegen ▪ Examining number of partitions
  36. Slow query - is it a query planning problem? What to do: Examine the query plan ▪ If the query completes, look at query plan using Spark UI SQL tab ▪ If the query does not complete, use .explain What to look for: ▪ Optimized logical query plan approaches size of Dickens novel ▪ Specifically, optimized plan >>> original query How to fix it: ▪ When caused by .withColumn or similar, refer to Groundhog Day ▪ When caused by an iterative algorithm ▪ Round trip through RDD to truncate query plan (SPARK-13346)
  37. How to investigate query planning issues 👉 Search Spark Jira filtered on SQL and/or Optimizer component with version filtered. Investigate behavior of ▪ Coalesce ▪ The optimizer will push coalesce as early as possible, which limits the number of executors the rest of the job will run on! See this article for a good explanation. ▪ Joins ▪ Did it turn into a cross join? See SPARK-21380 for an example of how this can happen. ▪ UDFs ▪ Case statements (SPARK-24983) ▪ Repeated calls to cast or alias (SPARK-26626) ▪ Aggregation on decimal columns (SPARK-28067) OK, so it’s not something LOL iterative, what next?
  38. Code cache is full The symptom: Your Spark app is running very slowly and you find an error in stdout: Java HotSpot(TM) 64-Bit Server VM warning: CodeCache is full. Compiler has been disabled. The impact: Tungsten uses JIT to optimize queries at runtime. JIT stores generated code in the JVM’s code cache. So Spark will stop optimizing queries if there’s no more room. How to fix it: ▪ Give the Spark driver more memory ▪ Tune the code cache to ensure there will be enough room
  39. Code Cache: figuring out usage // pass these options to spark.driver.extraOptions // these will print useful information when JVM exits // - print code cache usage // - print compiler stats -XX:+PrintCodeCache -XX:+CITime
  40. Tuning the Code Cache // set the reserved and min free to ensure code cache flushing // frees up space in a timely manner // code cache flushing turned on by default in 1.7.0_40+ -XX:+UseCodeCacheFlushing -XX:InitialCodeCacheSize=33554432 // 32m // flushing triggered when (reserved - min free) is used -XX:ReservedCodeCacheSize=536870912 // maximum size 512m // space is reserved for code that is not compiled methods, e.g., native // adapters. -XX:CodeCacheMinimumFreeSpace=52428800 // 50m Making room for more junk in the trunk
  41. Codegen failed because generated code > 64K The symptom: Extremely large stacktrace containing something like org.codehaus.janino.JaninoRuntimeException: Code of method XXX of class YYY grows beyond 64 KB What happened: 65,535 is the largest bytecode size possible for a valid Java method (limit imposed by JVM spec). Generated code can exceed this limit. The impact: Spark logs a massive stacktrace. Then it falls back to executing the physical plan (slower).
  42. Codegen >64K: no quick fix Make the big query smaller Cost: 💰(💰💰) Effort: 💪💪(💪) Figure out why the large query is so large. Could be fixed by truncating query lineage by round tripping through RDD or checkpointing (performance impact!). Other causes could require significant work. Disable codegen for everything Cost: 💰 Effort: 💪 Set spark.sql.codegen.wholeStage to false on the Spark driver. Significant performance impact. Recommended only in the case of a critical codegen bug affecting your app.
  43. Codegen >64K: capturing generated code in logs // default value is 1000 lines (Spark 2.3.0+) // -1 to log the entire error // BEWARE: the entire error could be very large // so large that logging all the lines could cause a performance issue // 0 to disable logging the error spark.sql.codegen.logging.maxLines 10000 SQLConf.scala (see discussion in SPARK-20871) ��
  44. Codegen >64K: stop logging the error # TODO: this is suppressing errors that Spark codegen is generating code >64K # for one or more queries. When upgrading Spark versions, check if still necessary. log4j.logger.org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator=OFF Your app is running “fine” and nobody has time to fix this right now Action: Turn off this stacktrace in Spark’s log4j.properties file. Impact: Leaves codegen enabled for your app. Most queries still benefit. But now the on call engineer doesn’t get paged about an exception that didn’t halt the application.
  45. Under the hood: codegen import org.apache.spark.sql.execution.debug._ val q = ds .filter($"PSSite" === "Street") .filter($"Borough".isNotNull) .filter($"Postcode" =!= "0") .groupBy($"Borough", $"Postcode") .agg(count($"OBJECTID").alias("count")) .sort($"count".desc) // prints the codegenString q.debugCodegen() Looking at the decisions whole stage code generation is making Executing command, time = 1590099312929. Executing command, time = 1590099325406. Found 2 WholeStageCodegen subtrees. 🤔
  46. == Subtree 1 / 2 == *(1) HashAggregate(keys=[Borough#17, Postcode#30L], functions=[partial_count(OBJECTID#16L) AS count#1247L], output=[Borough#17, Postcode#30L, count#1247L]) +- *(1) Project [OBJECTID#16L, Borough#17, Postcode#30L] +- *(1) Filter ((((isnotnull(PSSite#20) && isnotnull(Postcode#30L)) && (PSSite#20 = Street)) && isnotnull(Borough#17)) && NOT (Postcode#30L = 0)) +- InMemoryTableScan [Borough#17, OBJECTID#16L, PSSite#20, Postcode#30L], [isnotnull(PSSite#20), isnotnull(Postcode#30L), (PSSite#20 = Street), isnotnull(Borough#17), NOT (Postcode#30L = 0)] +- InMemoryRelation [OBJECTID#16L, Borough#17, Number#18, Street#19, PSSite#20, ParkName#24, Postcode#30L, Congressional#34L, PSStatus#36, Latitude#44, Longitude#45], StorageLevel(disk, memory, deserialized, 1 replicas) +- *(1) FileScan csv [OBJECTID#16L, Borough#17, Number#18, Street#19, PSSite#20, ParkName#24,Postcode#30L,Congressional#34L,PSStatus#36,Latitude#44,Longitude#45] Batched: false, DataFilters: [], Format: CSV, Location: InMemoryFileIndex[dbfs:/FileStore/tables/Forestry_Planting_Spaces_csv-3123a.gz], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<OBJECTID:bigint,Borough:string,Number:string,Street:string,PSSite:string,ParkName:string,P... Generated code: /* 001 */ public Object generate(Object[] references) { /* ETC */ /* 005 */ // codegenStageId=1 /* 006 */ final class GeneratedIteratorForCodegenStage1 extends org.apache.spark.sql.execution.BufferedRowIterator { /* ETC */ /* 419 */ } Triggered by .cache
  47. == Subtree 2 / 2 == *(2) HashAggregate(keys=[Borough#17, Postcode#30L], functions=[finalmerge_count(merge count#1247L) AS count(OBJECTID#16L)#1180L], output=[Borough#17, Postcode#30L, count#1181L]) +- Exchange hashpartitioning(Borough#17, Postcode#30L, 200), [id=#953] +- *(1) HashAggregate(keys=[Borough#17, Postcode#30L], functions=[partial_count(OBJECTID#16L) AS count#1247L], output=[Borough#17, Postcode#30L, count#1247L]) +- *(1) Project [OBJECTID#16L, Borough#17, Postcode#30L] +- *(1) Filter ((((isnotnull(PSSite#20) && isnotnull(Postcode#30L)) && (PSSite#20 = Street)) && isnotnull(Borough#17)) && NOT (Postcode#30L = 0)) +- InMemoryTableScan [Borough#17, OBJECTID#16L, PSSite#20, Postcode#30L], [isnotnull(PSSite#20), isnotnull(Postcode#30L), (PSSite#20 = Street), isnotnull(Borough#17), NOT (Postcode#30L = 0)] +- InMemoryRelation [OBJECTID#16L, Borough#17, Number#18, Street#19, PSSite#20, ParkName#24, Postcode#30L, Congressional#34L, PSStatus#36, Latitude#44, Longitude#45], StorageLevel(disk, memory, deserialized, 1 replicas) +- *(1) FileScan csv [OBJECTID#16L, Borough#17, Number#18, Street#19, PSSite#20, ParkName#24,Postcode#30L,Congressional#34L,PSStatus#36,Latitude#44,Longitude#45] Batched: false, DataFilters: [], Format: CSV, Location: InMemoryFileIndex[dbfs:/FileStore/tables/Forestry_Planting_Spaces_csv-3123a.gz], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<OBJECTID:bigint,Borough:string,Number:string,Street:string,PSSite:string,ParkName:string,P… Generated code: /* 001 */ public Object generate(Object[] references) { /* ETC */ /* 005 */ // codegenStageId=2 /* 006 */ final class GeneratedIteratorForCodegenStage2 extends org.apache.spark.sql.execution.BufferedRowIterator { /* ETC */ /* 174 */ } From the physical plan SHUFFLE Sends data across network
  48. Under the hood: rule based optimization import org.apache.spark.sql.catalyst.rules.RuleExecutor // do this first or your values will be cumulative RuleExecutor.resetMetrics() // if query can’t run successfully, try this without an action ds .filter($"PSSite" === "Street") .filter($"Borough".isNotNull) .filter($"Postcode" =!= 0) .groupBy($"Borough", $"Postcode") .agg(countDistinct($"Number", $"Street").alias("streetSiteCount")) .sort($"streetSiteCount".desc) .limit(10) .show() println(RuleExecutor.dumpTimeSpent()) Source / SPARK-23170
  49. Debugging rule based optimization === Metrics of Analyzer/Optimizer Rules === Total number of runs: 8759 Total time: 1.70915788 seconds Rule Effective Time / Total Time Effective Runs / Total Runs o.a.s.s.c.expressions.codegen.ExpressionCanonicalizer$CleanExpressions 6831 / 90273880 2 /2050 o.a.s.s.c.analysis.Analyzer$ResolveDeserializer 41473051 / 61077827 12 / 73 o.a.s.s.c.optimizer.ColumnPruning 10392678 / 57856720 2 / 48 o.a.s.s.c.analysis.Analyzer$ResolveReferences 17029859 / 41812132 11 / 73 o.a.s.s.c.optimizer.RemoveRedundantAliases 619302 / 36946007 1 / 32 o.a.s.s.c.analysis.Analyzer$ResolveAliases 28314403 / 28852027 5 / 73 com.databricks.sql.optimizer.FilterReduction 5503222 / 22282239 4 / 24 o.a.s.s.c.analysis.ResolveTimeZone 14843718 / 18850069 10 / 73 o.a.s.s.c.optimizer.InferFiltersFromConstraints 11482047 / 18536102 2 / 16 o.a.s.s.c.analysis.Analyzer$ResolveUpCast 13469159 / 17480404 7 / 73 o.a.s.s.c.analysis.TypeCoercion$PromoteStrings 4365841 / 15382158 2 / 73 o.a.s.s.c.analysis.CleanupAliases 10068435 / 13143189 4 / 43 o.a.s.s.c.analysis.Analyzer$ResolveNewInstance 3774516 / 11244432 7 / 73 o.a.s.s.c.optimizer.ConstantFolding 3979019 / 8407165 2 / 24 o.a.s.s.c.optimizer.CombineFilters 4298334 / 5767849 4 / 32 o.a.s.s.c.optimizer.SimplifyCasts 222489 / 4654333 1 / 24 o.a.s.s.c.optimizer.ConvertToLocalRelation 271763 / 1721820 2 / 18 Vendor specific Logical Optimization This query is trivial, so the optimizer spent most of its time seeing if it could DRY out the code 😊
  50. Disabling codegen spark.sql.codegen.wholeStage false Separate executions - less efficient execution model Codegen has fused multiple operations together here
  51. Now look at the final HashAggregate operation This is a single operation after a shuffle. But due to its execution model, it’s still less efficient than the whole-stage codegen operation. 10x faster. But still slower than it should be. And peak memory is still 💩. Why? With codegenWithout codegen
  52. Query planning can’t fix poor partitioning choices > q.rdd.toDebugString (188) MapPartitionsRDD[43] at rdd at command-2858540230021050:1 [] | SQLExecutionRDD[42] at rdd at command-2858540230021050:1 [] | MapPartitionsRDD[41] at rdd at command-2858540230021050:1 [] | MapPartition sRDD[40] at rdd at command-2858540230021050:1 [] | ShuffledRowRDD[39] at rdd at command-2858540230021050:1 [] +-(200) MapPartitionsRDD[38] at rdd at command-2858540230021050:1 [] | MapPartitionsRDD[34] at rdd at command-2858540230021050:1 [] | ShuffledRowRDD[33] at rdd at command-2858540230021050:1 [] +-(1) MapPartitionsRDD[32] at rdd at command-2858540230021050:1 [] | MapPartitionsRDD[31] at rdd at command-2858540230021050:1 [] | MapPartitionsRDD[30] at rdd at command-2858540230021050:1 [] | MapPartitionsRDD[29] at rdd at command-2858540230021050:1 [] | MapPartitionsRDD[28] at rdd at command-2858540230021050:1 [] | MapPartitionsRDD[27] at rdd at command-2858540230021050:1 [] | FileScan csv [OBJECTID#2L,Borough#3,Number#4,Street#5,PSSite#6,ParkName#10,Postcode#16L,Congressional#20L,PSStatus#22,Latitude#30,Longit ude#31] Batched: false, DataFilters: [], Format: CSV, Location: InMemoryFileIndex[dbfs:/FileStore/tables/Forestry_Planting_Spaces_csv-3123a.gz], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<OBJECTID:bigint,Borough:string,Number:string,Street:string,PSSite:string,ParkName:string,P... MapPartitionsRDD[2] at collectResult at OutputAggregator.scala:149 [] | CachedPartitions: 1; MemorySize: 46.9 MB; ExternalBlockStoreSize: 0.0 B; DiskSize: 0.0 B | MapPartitionsRDD[1] at collectResult at OutputAggregator.scala:149 [] | FileScanRDD[0] at collectResult at OutputAggregator.scala:149 [] Wait, why are there so many partitions? 🤦‍♂ Default is 200 partitions for aggregations and joins. Clue was back in physical plan under Exchange (= shuffle) hashpartitioning
  53. Updating shuffle partitions > q.rdd.toDebugString (2) MapPartitionsRDD[37] at rdd at command-2858540230021050:1 [] | SQLExecutionRDD[36] at rdd at command-2858540230021050:1 [] | MapPartitionsRDD[35] at rdd at command-2858540230021050:1 [] | MapPartitionsRDD[34] at rdd at command-2858540230021050:1 [] | ShuffledRowRDD[33] at rdd at command-2858540230021050:1 [] +-(2) MapPartitionsRDD[32] at rdd at command-2858540230021050:1 [] | MapPartitionsRDD[28] at rdd at command-2858540230021050:1 [] | ShuffledRowRDD[27] at rdd at command-2858540230021050:1 [] +-(1) MapPartitionsRDD[26] at rdd at command-2858540230021050:1 [] | MapPartitionsRDD[25] at rdd at command-2858540230021050:1 [] | MapPartitionsRDD[24] at rdd at command-2858540230021050:1 [] | MapPartitionsRDD[23] at rdd at command-2858540230021050:1 [] | *(1) FileScan csv [OBJECTID#2L,Borough#3,Number#4,Street#5,PSSite#6,ParkName#10,Post code#16L,Congressional#20L,PSStatus#22,Latitude#30,Longitude#31] Batched: false, DataFilters: [], Format: CSV, Location: InMemoryFileIndex[dbfs:/FileStore/tables/Forestry_Planting_Spaces_ csv-3123a.gz], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<OBJECTID:bigint,Borough:string,Number:string,Street:string, PSSite:string,ParkName:string,P... spark.sql.shuffle.partitions 2 Faster, uses less memory 🚀
  54. Exciting changes
  55. Spark 3+ exciting changes ▪ JDK 11, Scala 2.12 support (SPARK-24417) (blog) ▪ Adaptive SQL Query Optimization (SPARK-31412) (blog) ▪ Dynamic Partition Pruning (SPARK-11150) ▪ New join hints (SPARK-27225) ▪ Data Source V2 API (SPARK-15689) ▪ New Pandas UDFs and PEP-484 Python type hints (SPARK-28958) (blog) ▪ Spark UI SQL tab shows query instead of callsite (SPARK-27045) ▪ RuleExecutor query metrics improvements (SPARK-31079)
  56. Alan Perlis Epigrams on Programming, 1982 Simplicity does not precede complexity, but follows it.
  57. Incredible Resources
  58. Papers ▪ Stuart, C (2020). Profiling Compiled SQL Query Pipelines in Apache Spark (Master’s thesis, University of Amsterdam). ▪ https://homepages.cwi.nl/~boncz/msc/2020-ChristianStuart.pdf ▪ Schiavio, F., Bonetta, D., & Binder, W. (2020). Dynamic speculative optimizations for SQL compilation in Apache Spark. Proceedings of the VLDB Endowment, 13(5), 754–767. ▪ http://www.vldb.org/pvldb/vol13/p754-schiavio.pdf ▪ Wroblewski, J., Ishizaki, K., Inoue, H., & Ohara, M. (2017). Accelerating Spark Datasets by Inlining Deserialization. 2017 IEEE International Parallel and Distributed Processing Symposium (IPDPS). ▪ https://www.mimuw.edu.pl/~xi/download/deserialization-inlining-ipdps2017.pdf ▪ Prokopec, A., Duboscq, G., Leopoldseder, D., & Würthinger, T (2019). An optimization-driven incremental inline substitution algorithm for just-in-time compilers. CGO 2019: Proceedings of the 2019 IEEE/ACM International Symposium on Code Generation and Optimization, February 2019, 164–179. ▪ http://aleksandar-prokopec.com/resources/docs/prio-inliner-final.pdf ▪ Neumann, T. (2011). Efficiently compiling efficient query plans for modern hardware. Proceedings of the VLDB Endowment, 4(9), 539–550. ▪ http://www.vldb.org/pvldb/vol4/p539-neumann.pdf
  59. Presentations ▪ Ishizaki, K. (2018, November 20). Looking back at Spark 2.x and forward to 3.0. Hadoop source code reading. Tokyo, Japan. ▪ https://www.slideshare.net/ishizaki/looking-back-at-spark-2x-and-forward-to-30 ▪ Li, X. (2019, March 14). An Insider’s Guide to Maximizing Spark SQL Performance. Hadoop/Spark Conference. Tokyo, Japan. ▪ https://www.slideshare.net/ueshin/an-insiders-guide-to-maximizing-spark-sql-performance ▪ Li, X. (2019, May 11). Understanding Query Plans and Spark UIs. Spark+AI Summit. San Francisco, US. ▪ https://www.slideshare.net/databricks/understanding-query-plans-and-spark-uis ▪ Ueshin, T. (2019, March 14). Deep Dive into Spark SQL with Advanced Performance Tuning. Hadoop/Spark Conference. Tokyo, Japan. (日本語) ▪ https://www.slideshare.net/ueshin/deep-dive-into-spark-sql-with-advanced-performance-tuning ▪ MacGregor, D. (2019, March 4). Graal: Not just a new JIT for the JVM. QCon London: Software Architecture Conference. London, UK. ▪ https://qconlondon.com/system/files/presentation-slides/qcon-london.pdf
  60. Presentations, continued ▪ Vrba, D. (2019, 15-17 October). Physical Plans in Spark SQL. Spark+AI Summit Europe. Amsterdam, The Netherlands. ▪ https://www.slideshare.net/databricks/physical-plans-in-spark-sql ▪ Ellison, T. (2016, May 20-21). A Java Implementer’s Guide to Boosting Apache Spark Performance. J On The Beach. Malaga, Spain. ▪ https://www.slideshare.net/JontheBeach/a-java-implementers-guide-to-boosting-apache-spark-performance-by-tim-ellison ▪ Fan, W. & Wang, G. (2018, June 6). Data Source API V2. Spark+AI Summit. San Francisco, US. ▪ https://www.slideshare.net/databricks/apache-spark-data-source-v2-with-wenchen-fan-and-gengliang-wang ▪ Xue, M., Jiang, X., & Mok, K. (2019, April 24-25). A Deep Dive into Query Execution Engine of Spark SQL. Spark+AI Summit. San Francisco, US. ▪ https://www.slideshare.net/databricks/a-deep-dive-into-query-execution-engine-of-spark-sql ▪ Fiorito, S. (2018, October 2-4). Lessons from the Field, Episode II: Applying Best Practices to Your Apache Spark Applications. Spark+AI Summit Europe. London, UK. ▪ https://www.slideshare.net/databricks/lessons-from-the-field-episode-ii-applying-best-practices-to-your-apache-spark-appli cations-with-silvio-fiorito ▪ Price, M. (2016). Hot code is faster code - addressing JVM warm-up. QCon London. London, UK. ▪ https://qconlondon.com/system/files/presentation-slides/markprice-howcodeisfastercode.pdf
  61. Articles ▪ Agarwal, S., Liu, D., & Xin, R. (2016, May 23). Apache Spark as a Compiler: Joining a Billion Rows per Second on a Laptop. ▪ https://databricks.com/blog/2016/05/23/apache-spark-as-a-compiler-joining-a-billion-rows-per-second-on-a-laptop.html ▪ Ueshin, T. & van Hövell, H. (2018, November 16). Introducing New Built-in and Higher-Order Functions for Complex Data Types in Apache Spark 2.4. ▪ https://databricks.com/blog/2018/11/16/introducing-new-built-in-functions-and-higher-order-functions-for-complex-data-typ es-in-apache-spark.html ▪ Zhang, M. (2018, July 11). The hidden cost of Spark withColumn. ▪ https://manuzhang.github.io/2018/07/11/spark-catalyst-cost.html ▪ Zhang, M. (2019, April 18). Note about Spark Python UDF. ▪ https://manuzhang.github.io/2019/04/18/spark-python-udf.html ▪ Mohamed, Y. (2019, November 2). The recipe of instability: Spark UDFs + non pure computation. ▪ https://medium.com/analytics-vidhya/the-recipe-of-instability-f2e914e31f5a ▪ Konieczny, B. (2019, September 28). The why of code generation in Apache Spark SQL. ▪ https://www.waitingforcode.com/apache-spark-sql/why-code-generation-apache-spark-sql/read
  62. Online Books ▪ Laskowski, J. (2020). Spark SQL Analyzer. In The Internals of Apache Spark. ▪ https://jaceklaskowski.gitbooks.io/mastering-spark-sql/spark-sql-Analyzer.html ▪ Laskowski, J. (2020). Debugging Query Execution. In The Internals of Apache Spark. ▪ https://jaceklaskowski.gitbooks.io/mastering-spark-sql/spark-sql-debugging-query-execution.html ▪ Laskowski, J. (2020). Nondeterministic Expression Contract. In The Internals of Apache Spark. ▪ https://jaceklaskowski.gitbooks.io/mastering-spark-sql/spark-sql-Expression-Nondeterministic.html ▪ Laskowski, J. (2020). Data Source API V2. In The Internals of Apache Spark. ▪ https://jaceklaskowski.gitbooks.io/mastering-spark-sql/spark-sql-data-source-api-v2.html ▪ Laskowski, J. (2020). CollapseCodegenStages Physical Query Optimization — Collapsing Physical Operators for Whole-Stage Java Code Generation (aka Whole-Stage CodeGen). In The Internals of Apache Spark. ▪ https://jaceklaskowski.gitbooks.io/mastering-spark-sql/spark-sql-data-source-api-v2.html ▪ Laskowski, J. (2020). Case Study: Number of Partitions for groupBy Aggregation. In The Internals of Apache Spark. ▪ https://jaceklaskowski.gitbooks.io/mastering-spark-sql/spark-sql-performance-tuning-groupBy-aggregation.html
  63. Spark issues and KB Articles ▪ Apache Spark Jobs hang due to non-deterministic custom UDF ▪ https://kb.databricks.com/jobs/spark-udf-performance.html ▪ SPARK-27761: Make UDFs non-deterministic by default (open, interesting discussion) ▪ https://issues.apache.org/jira/browse/SPARK-27761 ▪ SPARK-27969: Non-deterministic expressions in filters or projects can unnecessarily prevent all scan-time column pruning, harming performance ▪ https://issues.apache.org/jira/browse/SPARK-27969 ▪ [SPARK-27692][SQL] Add new optimizer rule to evaluate the deterministic scala udf only once if all inputs are literals 👈 unmerged PR with interesting discussion ▪ https://github.com/apache/spark/pull/24593
  64. Feedback Your feedback is important to us. Don’t forget to rate and review the sessions.
Advertisement