Spark sql meetup

30
Spark SQL 漫漫 Cheng Hao Oct 25, 2014 Copyright © 2014 Intel Corporation.

Transcript of Spark sql meetup

Page 1: Spark sql meetup

Spark SQL 漫谈Cheng Hao

Oct 25, 2014Copyright © 2014 Intel Corporation.

Page 2: Spark sql meetup

Agenda Spark SQL Overview Catalyst in Depth SQL Core API Introduction V.S. Shark & Hive-on-Spark Our Contributions Useful Materials

2Copyright © 2014 Intel Corporation.

Page 3: Spark sql meetup

Spark SQL Overview

Copyright © 2014 Intel Corporation.

Page 4: Spark sql meetup

Spark SQL in Spark

4

Spark Streamin

greal-time

GraphXGraph(alpha)

MLLibMachine Learning

Spark Core

Spark SQL

Spark SQL was first released in Spark 1.0 (May, 2014) Initial committed by Michael Armbrust & Reynold Xin from Databricks

Copyright © 2014 Intel Corporation.

Page 5: Spark sql meetup

5

Spark SQL Component Stack (User Perspective)

Hive-like interface(JDBC Service / CLI) SQL API support (LINQ-like) Both Hive QL & Simple SQL dialects are

Supported DDL is 100% compatible with Hive Metastore Hive QL aims to 100% compatible with Hive

DML Simple SQL dialect is now very weak in

functionality, but easy to extendSpark Core

Spark Execution OperatorsCatalyst

Hive QL Simple SQL

SQL APICLI

User Application

JDBC Service

Data Analyst

Hive Meta Store

Simple Catalog

Copyright © 2014 Intel Corporation.

Page 6: Spark sql meetup

6

Spark SQL Architecture

Catalyst

BackendFrontend

Copyright © 2014 Intel Corporation.

By Michael Armbrust @ Databricks

Page 7: Spark sql meetup

Catalyst in Depth

Copyright © 2014 Intel Corporation.

Page 8: Spark sql meetup

8

Understand Some Terminology Logical and Physical query plans

Both are trees representing query evaluation Internal nodes are operators over the data Logical plan is higher-level and algebraic Physical plan is lower-level and operational

Logical plan operators Correspond to query language constructs Conceptually describe what operation needs to be

performed Physical plan operators

Correspond to implemented access methods Physically Implement the operation described by

logical operators

Unresolved Logical Plan

Logical Plan

SQL Text

Optimized Logical Plan

Physical Plan

Parsing

Binding & Analyzing

Optimizing

Query Planning

Copyright © 2014 Intel Corporation.

Page 9: Spark sql meetup

9

Examples

• CREATE TABLE T (key: String, value: String)

• EXPLAIN EXTENDED SELECT a.key * (2 + 3), b.value FROM T a JOIN T b ON a.key=b.key AND a.key>3

We execute the following commands on Spark SQL CLI.

Copyright © 2014 Intel Corporation.

Page 10: Spark sql meetup

10 Copyright © 2014 Intel Corporation.

Understand some terminologies== Parsed Logical Plan ==Project [('a.key * (2 + 3)) AS c_0#24,'b.value] Join Inner, Some((('a.key = 'b.key) && ('a.key > 3))) UnresolvedRelation None, T, Some(a) UnresolvedRelation None, T, Some(b)

== Analyzed Logical Plan ==Project [(CAST(key#27, DoubleType) * CAST((2 + 3), DoubleType)) AS c_0#24,value#30] Join Inner, Some(((key#27 = key#29) && (CAST(key#27, DoubleType) > CAST(3, DoubleType)))) MetastoreRelation default, T, Some(a) MetastoreRelation default, T, Some(b)

== Optimized Logical Plan ==Project [(CAST(key#27, DoubleType) * 5.0) AS c_0#24,value#30] Join Inner, Some((key#27 = key#29)) Project [key#27] Filter (CAST(key#27, DoubleType) > 3.0) MetastoreRelation default, T, Some(a) MetastoreRelation default, T, Some(b)

== Physical Plan ==Project [(CAST(key#27, DoubleType) * 5.0) AS c_0#24,value#30] BroadcastHashJoin [key#27], [key#29], BuildLeft Filter (CAST(key#27, DoubleType) > 3.0) HiveTableScan [key#27], (MetastoreRelation default, T, Some(a)), None HiveTableScan [key#29,value#30], (MetastoreRelation default, T, Some(b)), None

Page 11: Spark sql meetup

11

• Catalyst essentially a extensible framework to Analyze & Optimize the logical plan, expression.

• Core Elements:• Tree Node API• Expression Optimization • Data Type & Schema• Row API• Logical Plan (Unresolved) Binding & Analyzing (Rules)• Logical Plan (Resolved) Optimizing (Rules)

• SPI (Service Provider Interface)• FunctionRegistry• Schema Catalog

Catalyst Overview

Copyright © 2014 Intel Corporation.

Page 12: Spark sql meetup

12

Data Type & Schema Primitive Type

StringType, FloatType, IntegerType, ByteType, ShortType, DoubleType, LongType, BinaryType, BooleanType, DecimalType, TimestampType, DateType, Varchar(Not Complete Supported Yet), Char(Not Complete Supported Yet)

Complex Type ArrayType

ArrayType(elementType: DataType) StructType

StructField(name: String, dataType: DataType) StructType(fields: Seq[StructField])

MapType MapType(keyType: DataType, valueType: DataType)

UnionType (Not Supported Yet)

Relation Schema

Copyright © 2014 Intel Corporation.

Page 13: Spark sql meetup

13

Row APItrait Row extends Seq[Any] with Serializable { def apply(i: Int): Any def isNullAt(i: Int): Boolean def getInt(i: Int): Int def getLong(i: Int): Long def getDouble(i: Int): Double def getFloat(i: Int): Float def getBoolean(i: Int): Boolean def getShort(i: Int): Short def getByte(i: Int): Byte def getString(i: Int): String def getAs[T](int: Int): T}

Row class is the key data structure widely used internal / external Spark SQL.

“def getAs[T]” is used for non-primitive data types

Field value represented as native language data type.

Field type represented as DataType described in last slice.

Page 14: Spark sql meetup

14

Logical Plan Binding & Analyzing • Essentially about data binding & semantic analysis• Example Rules

• Bind Attributes, Relations with concrete data.• ResolveReferences, ResolveRelation

• Expressions Analysis• Data Type Coercion (PropagateTypes, PromoteString, BooleanCasts, Division

etc.)• Bind UDF(ResolveFunctions)

• Evict / Expand the Analysis Logical Plan Operators• StarExpansion, EliminateAnalysisOperators

• Implicit Semantic Supplement• Add sort expressions into the child projection list.(ResolveSortReferences)• Convert projection into aggregation if the projection contains aggregate

function(GlobalAggregates).• UnresolvedHavingClauseAttributes

• Semantic Checking• Unresolved Function, Relation, Attributes (CheckResolution)• Illegal expressions in projection of an Aggregation (CheckAggregation)

• ….

Copyright © 2014 Intel Corporation.

Page 15: Spark sql meetup

15

Logical Plan Optimizing• Simplify the Logical Plan Tree based on Relational / Logical Algebra, Common Sense (Rule

Based)• Example Rules

• Expression Optimization.• NullPropagation, ConstantFolding, SimplifyFilters, SimplifyCasts, OptimizeIn etc.

• Filter PushDown• UnionPushdown, PushPredicateThroughProject,

PushPredicateThroughJoin,ColumnPruning• Combine Operators

• CombineFilters, CombineLimits• Concrete Example

• IsNull(‘a + null) => IsNull(null) => Literal(true)• SELECT a.key, b.key FROM a, b ON a.key=b.key AND b.key>10 => SELECT a.key, b.key FROM a, (SELECT key FROM b WHERE key>10) ON a.key=b.key

Copyright © 2014 Intel Corporation.

Page 16: Spark sql meetup

16

Spark SQL Dialects

Hive AST

Hive Parser

Logical Plan

Optimized Logical Plan

Hive+Spark Planner

Spark Planner

Execution Operators

SQL Parser

Unresolved Logical Plan

Hive Catelog

Simple Catelog

HiveContext SQLContext

DSL API

Frontend

Catalyst

Backend

XXXContext

XXX Planner

XXX Catelog

XX Parser / API

Frontend +

Catalyst + SPI+

Backend||

Tool

Copyright © 2014 Intel Corporation.

Page 17: Spark sql meetup

17

Spark Plan (Physical Plan) Root class of Spark Plan Operator (Physical Plan Operator for Spark)

Spark Plan Operators Joins: BroadcastHashJoin, CartesianProduct, HashOuterJoin, LeftSemiJoinHash etc.) Aggregate: Aggregate BasicOperators: Distinct, Except, Filter, Limit, Project, Sort, Union etc.) Shuffle: AddExchange, Exchange Commands: CacheTableCommand, DescribeCommand, ExplainCommand etc.) ..

Spark Strategy (SparkPlanner) Map the Optimized Logical Plan to Spark Plan

abstract class SparkPlan { def children: Seq[SparkPlan] /** Specifies how data is partitioned across different nodes in the cluster. */ def outputPartitioning: Partitioning = UnknownPartitioning(0) /** Specifies any partition requirements on the input data for this operator. */ def requiredChildDistribution: Seq[Distribution] = Seq.fill(children.size)(UnspecifiedDistribution)

def execute(): RDD[Row]}

Optimized Logical Plan

Spark Plan

RDD

Spark Execution

Page 18: Spark sql meetup

18

Case Study for Catalyst in Depth• StreamSQL

• Reuse the HiveContext but with different Frontend / Backend.• Frontend: Slight modification of the HiveParser• Backend: Customed Query Planner, to generate the physical plan based on

Spark DStream.• JIRA: https://issues.apache.org/jira/browse/SPARK-1363 • Source: https://github.com/thunderain-project/StreamSQL

• SQL 92 Support• Reuse the HiveContext but with different Frontend• Frontend: A modified HiveParser & Hive QL translator.• https://github.com/intel-hadoop/spark/tree/panthera

• Pig on Spark POC• Modify the SQLContext• Provide a PigParser to translate the Pig script into Catalyst unresolved logical

plan• https://github.com/databricks/pig-on-spark

Copyright © 2014 Intel Corporation.

Page 19: Spark sql meetup

SQL Core API Introduction

Copyright © 2014 Intel Corporation.

Page 20: Spark sql meetup

20

SchemaRDD• What’s SchemaRDD?• Spark SQL Core API (In Scala)

• Create SchemaRDD instance from• Plain SQL Text def sql(sqlText: String)• An existed Logical Plan def logicalPlanToSparkQuery(plan: LogicalPlan)• Spark RDD def createSchemaRDD[A <: Product: TypeTag](rdd: RDD[A])• Spark RDD with Schema def applySchema(rowRDD: RDD[Row], schema: StructType)• Frequently used format file (json, parquet, etc.) def parquetFile(path: String)

• SQL DSL• select, where, join, orderBy, limit, groupBy, unionAll, etc.

• Data Sink• Persist the data with specified storage level def persist(newLevel: StorageLevel)• Save the data as ParquetFile def saveAsParquetFile(path: String)• Save the data as a new Table def registerTempTable(tableName: String)• Insert the data into existed table def insertInto(tableName: String, overwrite:

Boolean)• ….

• Java API / Python API supported

Copyright © 2014 Intel Corporation.

class SchemaRDD( @transient val sqlContext: SQLContext, @transient val baseLogicalPlan: LogicalPlan) extends RDD[Row](sqlContext.sparkContext, Nil)

Page 21: Spark sql meetup

21

Conceptual State Transition Diagram

RDD

Schema RDD

Unresolved Logical Plan

SQL API

SQL Text / File / Table

* Unresolved Logical Plan RDD (Unresolved Logical Plan Logical Plan Optimized Logical Plan Physical Plan Spark RDD)

Copyright © 2014 Intel Corporation.

File / Memory etc.

Page 22: Spark sql meetup

22

Code Examplesbt/sbt hive/console

// HiveContext is created by default, and the object is imported, so we can call the object methods directly.sql("CREATE TABLE IF NOT EXISTS kv_text(key INT, value STRING)")sql("LOAD DATA LOCAL INPATH '/tmp/kv1.txt' INTO TABLE kv_text") // create a Hive table and load data into it

case class KV(key: Int, value: String)val kvRdd = sparkContext.parallelize((1 to 100).map(i => KV(i, s"val_$i"))) // create a normal RDD// implicitly convert the kvRDD into a SchemaRDDkvRdd.where('key >= 1).where('key <=5).registerTempTable("kv_rdd") // create a Hive Table from a SchemaRDD

jsonFile("/tmp/file2.json").registerTempTable("kv_json") // load json file and register as a Hive Tableval result = sql("SELECT a.key, b.value, c.key from kv_text a join kv_rdd b join kv_json c")

result.collect().foreach(row => { val f0 = if(row.isNullAt(0)) "null" else row.getInt(0) val f1 = if(row.isNullAt(1)) "null" else row.getString(1) val f2 = if(row.isNullAt(2)) "null" else row.getInt(2) println(s"result:$f0, $f1, $f2")}) Copyright © 2014 Intel Corporation.

Page 23: Spark sql meetup

V.S. Shark & Hive

Copyright © 2014 Intel Corporation.

Page 24: Spark sql meetup

24

Background of Shark/Hive-on-Spark/Spark SQL Shark is the first SQL on Spark product, based on the earlier versions of Hive (with a re-write

QueryPlanner to generate Spark RDD-based Physicial Plan); Shark is retired now and replaced by Spark SQL.

Hive-on-Spark is an QueryPlanner extension of Hive, it focus on the SparkPlanner and Spark RDD-based physical operators implementation. Spark users will automatically get the whole set of Hive’s rich features, including any new features that Hive might introduce in the future.

Spark SQL is a new SQL engine on Spark developed from scratch. Functionality

Spark SQL almost support all of the functionalities that Hive provided from the perspective of data analysts.

SQL API on Spark Shell V.S. Pig latin. Spark SQL is an extensible / flexible framework for developers (based on Catalyst), new extensions are

very easy to be integrated. Implementation Philosophy of Spark SQL (Simple & Nature)

Largely employs the Scala features (Pattern Matching, Implicit Conversion, Partial Function etc.) Large small pieces of simple rule to bind, analyze, optimize logical plan & expression tree, and also the

physical plan generation. In-memory Computing & Maximize the Memory Usage (Cache related SQL API & Command). Spark SQL benefits a lot from Hive by reusing its components (Hive QL Parser, Metatore, SerDe,

StorageHandler etc.) Stability

Hive is the defacto standard for SQL on big data so far, and it has been proven as a productive tool for couple of years in practices, many corner cases are covered in its continuous enhancements.

Spark SQL just start its journey ( ~0.5 year), we need more time to prove / improve it.

Copyright © 2014 Intel Corporation.

Page 25: Spark sql meetup

Our Contributions

Copyright © 2014 Intel Corporation.

Page 26: Spark sql meetup

26

Totally 60+ PRs, 50+ Merged on Spark SQL Features

Add serde support for CTAS (PR2570) Support the Grouping Set (PR1567) Support EXTENDED for EXPLAIN (PR1982) Cross join support in HiveQL (PR2124) Add support for left semi join (PR837) Add Date type support (PR2344) Add Timestamp type support (PR275) Add Expression RLike & Like support (PR224) ..

Performance Enhancement / Improvement Avoid table creation in logical plan analyzing for CTAS (PR1846) Extract the joinkeys from join condition (PR1190) Reduce the Expression tree object creations for aggregation function (min/max) (PR2113) Pushdown the join filter & predication for outer join (PR1015) Constant Folding for Expression Optimization (PR482) Fix Performance Issue in data type casting (PR679) Not limit argument type for hive simple udf (PR2506) Use GenericUDFUtils.ConversionHelper for Simple UDF type conversions (PR2407) Select null from table would throw a MatchError (PR2396) Type Coercion should support every type to have null value (PR2246) ….

Bugs Fixing ….

Copyright © 2014 Intel Corporation.

Page 27: Spark sql meetup

Useful Materials

Copyright © 2014 Intel Corporation.

Page 28: Spark sql meetup

28

References http://

spark-summit.org/wp-content/uploads/2013/10/J-Michael-Armburst-catalyst-spark-summit-dec-2013.pptx

http://spark-summit.org/wp-content/uploads/2014/07/Performing-Advanced-Analytics-on-Relational-Data-with-Spark-SQL-Michael-Armbrust.pdf

https://www.youtube.com/watch?v=GQSNJAzxOr8 http://

www.slideshare.net/ueshin/20140908-spark-sql-catalyst?qid=3bb8abf4-3d8d-433f-9397-c24c5256841d https://cwiki.apache.org/confluence/display/Hive/Hive+on+Spark http://web.stanford.edu/class/cs346/qpnotes.html http://www.cse.ohio-state.edu/hpcs/WWW/HTML/publications/papers/TR-11-7.pdf http://codex.cs.yale.edu/avi/db-book/db6/slide-dir/PDF-dir/ch13.pdf https://courses.cs.washington.edu/courses/cse444/12sp/lectures/ http://www.cs.uiuc.edu/class/sp06/cs411/lectures.html

• User Mail List [email protected]

• Dev Mail List [email protected]

• Jira https://issues.apache.org/jira/browse/SPARK/component/12322623

• DevDoc https://spark.apache.org/docs/latest/sql-programming-guide.html

• Github https://github.com/apache/spark/tree/master/sql

Copyright © 2014 Intel Corporation.

Page 29: Spark sql meetup

Notice and Disclaimers: Intel, the Intel logo are trademarks of Intel Corporation in the U.S. and/or other countries. *Other names and brands may

be claimed as the property of others.See Trademarks on intel.com for full list of Intel trademarks.

Optimization Notice:Intel's compilers may or may not optimize to the same degree for non-Intel microprocessors for optimizations that are not unique to Intel microprocessors. These optimizations include SSE2, SSE3, and SSSE3 instruction sets and other optimizations. Intel does not guarantee the availability, functionality, or effectiveness of any optimization on microprocessors not manufactured by Intel.Microprocessor-dependent optimizations in this product are intended for use with Intel microprocessors. Certain optimizations not specific to Intel microarchitecture are reserved for Intel microprocessors. Please refer to the applicable product User and Reference Guides for more information regarding the specific instruction sets covered by this notice.

Intel technologies may require enabled hardware, specific software, or services activation. Check with your system manufacturer or retailer.

No computer system can be absolutely secure. Intel does not assume any liability for lost or stolen data or systems or any damages resulting from such losses.

You may not use or facilitate the use of this document in connection with any infringement or other legal analysis concerning Intel products described herein. You agree to grant Intel a non-exclusive, royalty-free license to any patent claim thereafter drafted which includes subject matter disclosed herein.

No license (express or implied, by estoppel or otherwise) to any intellectual property rights is granted by this document. The products described may contain design defects or errors known as errata which may cause the product to deviate

from publish.

Copyright © 2014 Intel Corporation.

Page 30: Spark sql meetup

Copyright © 2014 Intel Corporation.