SlideShare a Scribd company logo
Solr as a Spark SQL Datasource
Kiran Chitturi,
Lucidworks
Solr & Spark
• A few interesting things about Spark
• Overview of SparkSQL and DataFrames
• Solr as a SparkSQL DataSource in depth
• Use Lucene for text analysis in ML pipelines
• Example Use Case: Lucidworks Fusion and Spark
The standard for
enterprise search. of Fortune 500
uses Solr.
90%
What’s interesting about Spark?
• Wealth of overview / getting started resources on the Web
➢ Start here -> https://spark.apache.org/
➢ Should READ! https://www.cs.berkeley.edu/~matei/papers/2012/nsdi_spark.pdf
• Faster, more modernized alternative to MapReduce
➢ Spark running on Hadoop sorted 100TB in 23 minutes (3x faster than Yahoo’s previous record while using10x less
computing power)
• Unified platform for Big Data
➢ Great for iterative algorithms (PageRank, K-Means, Logistic regression) & interactive data mining
➢ Runs on YARN, Mesos, and plays well with HDFS
• Nice API for Java, Scala, Python, R, and sometimes SQL … REPL interface too
• >14,200 Issues in JIRA, 1000+ code contributors, 2.0 coming soon!
Spark Components
Spark Core
Spark
SQL
Spark
Streaming
MLlib
(machine
learning)
GraphX
(BSP)
Hadoop YARN Mesos Standalone
HDFS
Execution
Model
The Shuffle Caching
components
engine
cluster
mgmt
Alluxio (formerly Tachyon)
languages Scala Java Python R
shared
memory
Physical Architecture (Standalone)
Spark Master (daemon)
Spark Slave (daemon)
my-spark-job.jar
(w/ shaded deps)
My Spark App
SparkContext
(driver)
• Keeps track of live workers
• Web UI on port 8080
• Task Scheduler
• Restart failed tasks
Spark Executor (JVM process)
Tasks
Executor runs in separate
process than slave daemon
Spark Worker Node (1...N of these)
Each task works on some partition of a
data set to apply a transformation or action
Cache
Losing a master prevents new
applications from being executed
Can achieve HA using ZooKeeper
and multiple master nodes
Tasks are assigned
based on data-locality
When selecting which node to execute a task on,
the master takes into account data locality
• RDD Graph
• DAG Scheduler
• Block tracker
• Shuffle tracker
Spark SQL
• DataSource API for reading from and writing to external data sources
• DataFrame is an RDD[Row] + schema
• Secret sauce is logical plan optimizer
• SQL or relational operators on DF
• JDBC / ODBC
• UDFs!
• Machine Learning Pipelines
Solr as a Spark SQL Data Source
• Read/write data from/to Solr as DataFrame
• Use Solr Schema API to access field-level metadata
• Push predicates down into Solr query constructs, e.g. fq clause
• Deep-paging, shard partitioning, intra-shard splitting, streaming results
// Connect to Solr
val opts = Map("zkhost" -> "localhost:9983", "collection" -> "nyc_trips")
val solrDF = sqlContext.read.format("solr").options(opts).load
// Register DF as temp table
solrDF.registerTempTable("trips")
// Perform SQL queries
sqlContext.sql("SELECT avg(tip_amount), avg(fare_amount) FROM trips").show()
Shard 1
Shard 2
socialdata
collection
Partition 1
(spark task)
solr.DefaultSource
Partition 2
(spark task)
Spark
Driver
App
ZooKeeper
Read collection metadata
(num shards, replica URLs, etc)
Table	Scan	Query:	
q=*:*&rows=1000&	
distrib=false&cursorMark=*
Results streamed back from Solr
Parallelizequeryexecution
sqlContext.load(“solr”,opts)
DataFrame
(schema + RDD<Row>)
Map("collection”	->	“nyc_trips”,	
					“zkHost”	->	“…”)
SolrRDD: Reading data from Solr into Spark
Shard 1
Shard 2
socialdata
collection
Partition 1
(spark task)
solr.DefaultSource
Partition 3
(spark task)
Spark
Driver
App
ZooKeeper
Read collection metadata
(num shards, replica URLs, etc)
Table	Scan	Query:	
q=*:*&rows=1000&	
distrib=false&cursorMark=*	
&split_field:[aa	TO	ba}
Parallelizequeryexecution
sqlContext.load(“solr”,opts)
DataFrame
(schema + RDD<Row>)
Map(“collection”	->	“nyc_trips”,	
					“zkHost”	->	“…”)
SolrRDD: Reading data from Solr into Spark
Partition 2
(spark task) &split_field:[ba	TO	bz]
Partition 4
(spark task)
&split_field:[aa	TO	ac}
&split_field:[ac	TO	ae]
Data-locality Hint
• SolrRDD extends RDD[SolrDocument] (written in Scala)
• Give hint to Spark task scheduler about where data lives
override	def	getPreferredLocations(split:	Partition):	Seq[String]	=	{	
			//	return	preferred	hostname	for	a	Solr	partition

}	
• Useful when Spark executor and Solr replicas live on same physical
host, as we do in Fusion
• Query to a shard has a “preferred” replica; can fallback to other
replicas if the preferred goes down (will be in 2.1)
Solr Streaming API for fast reads
• Contributed to spark-solr by Bloomberg team (we PRs)
• Extremely fast “table scans” over large result sets in Solr
• Relies on a column-oriented data structure in Lucene: docValues
• DocValues help speed up faceting and sorting too!
• Coming soon! Push SQL predicates down into Solr’s Parallel SQL
engine available in Solr 6.x
Writing to Solr (aka indexing)
• Cloud-aware client sends updates to shard leaders in parallel
• Solr Schema API used to create fields on-the-fly using the DataFrame
schema
• Better parallelism than traditional approaches like Solr DIH
val dbOpts = Map(
"url" -> "jdbc:postgresql:mydb",
"dbtable" -> “schema.table",
"partitionColumn" -> "foo",
"numPartitions" -> "10")
val jdbcDF = sqlContext.read.format("jdbc").options(dbOpts).load
val solrOpts = Map("zkhost" -> "localhost:9983", "collection" -> "mycoll")
jdbcDF.write.format("solr").options(solrOpts).mode(SaveMode.Overwrite).save
Solr / Lucene Analyzers for Spark ML Pipelines
• Spark ML Pipeline provides nice API for defining stages to train / predict ML models
• Crazy idea ~ use battle-hardened Lucene for text analysis in Spark
• Pipelines support import/export (work in progress, more coming in Spark 2.0)
• Can try different text analysis techniques during cross-validation
DF
Spark ML Pipeline
Lucene
Analyzer
HashingTF
Standard
Scaler
Trained
Model
(SVM)
save
https://lucidworks.com/blog/2016/04/13/spark-solr-lucenetextanalyzer/
Fusion & Spark
• spark-solr 2.0.1 released, built into Fusion 2.4
• Users leave evidence of their needs & experience as they use your app
• Fusion closes the feedback loop to improve results based on user “signals”
• Train and serve ML Pipeline and mllib based Machine Learning models
• Run custom Scala “script” jobs in the background in Fusion
➢Complex aggregation jobs (see next slide)
➢Unsupervised learning (LDA topic modeling)
➢Re-train supervised models as new training data flows in
Scheduled Fusion job to compute stats for user sessions
val	opts	=	Map("zkhost"	->	"localhost:9983”,	"collection"	->	"apachelogs”)	
var	logEvents	=	sqlContext.read.format("solr").options(opts).load	
logEvents.registerTempTable("logs”)	
sqlContext.udf.register("ts2ms",	(d:	java.sql.Timestamp)	=>	d.getTime)	
sqlContext.udf.register("asInt",	(b:	String)	=>	b.toInt)	
val	sessions	=	sqlContext.sql("""	
				|SELECT	*,	sum(IF(diff_ms	>	30000,	1,	0))	
				|OVER	(PARTITION	BY	clientip	ORDER	BY	ts)	session_id	
				|FROM	(SELECT	*,	ts2ms(ts)	-	lag(ts2ms(ts))	
				|OVER	(PARTITION	BY	clientip	ORDER	BY	ts)	as	diff_ms	FROM	logs)	tmp	
		""".stripMargin)	
sessions.registerTempTable("sessions")	
var	sessionsAgg	=	sqlContext.sql("""	
								|SELECT	concat_ws('||',	clientip,session_id)	as	id,	
								|							first(clientip)	as	clientip,	
								|							min(ts)	as	session_start,	
								|							max(ts)	as	session_end,	
								|							(ts2ms(max(ts))	-	ts2ms(min(ts)))	as	session_len_ms_l,	
								|							count(*)	as	total_requests_l	
								|FROM	sessions	
								|GROUP	BY	clientip,session_id	
		""".stripMargin)	
sessionsAgg.write.format("solr").options(Map("zkhost"	->	"localhost:9983",	"collection"	->	"apachelogs_signals_aggr"))	
			.mode(org.apache.spark.sql.SaveMode.Overwrite).save
Getting started with spark-solr
• Import package via maven
./bin/spark-shell --packages "com.lucidworks.solr:spark-solr:2.0.1"
• Build from source
git clone https://github.com/LucidWorks/spark-solr
cd spark-solr
mvn clean package -DskipTests
./bin/spark-shell --jars 2.1.0-SNAPSHOT.jar
Example : Deep paging via shards
// Connect to Solr
val opts = Map(
"zkhost" -> "localhost:9983",
"collection" -> "nyc_trips")
val solrDF = sqlContext.read.format("solr").options(opts).load
// Register DF as temp table
solrDF.registerTempTable("trips")
sqlContext.sql("SELECT * FROM trips LIMIT 2").show()
Example : Deep paging with intra shard splitting
// Connect to Solr
val opts = Map(
"zkhost" -> "localhost:9983",
"collection" -> "nyc_trips",
"splits" -> "true")
val solrDF = sqlContext.read.format("solr").options(opts).load
// Register DF as temp table
solrDF.registerTempTable("trips")
sqlContext.sql("SELECT * FROM trips").count()
Example : Streaming API (/export handler)
// Connect to Solr
val opts = Map(
"zkhost" -> "localhost:9983",
"collection" -> "nyc_trips")
val solrDF = sqlContext.read.format("solr").options(opts).load
// Register DF as temp table
solrDF.registerTempTable("trips")
sqlContext.sql("SELECT avg(tip_amount), avg(fare_amount) FROM
trips").show()
Performance test
• NYC taxi data (30 months - 91.7M rows)
• Dataset loaded in to AWS RDS instance (Postgres)
• 3 EC2 nodes of r3.2x large instances
• Solr and Spark instances co-located together
• Collection ‘nyc-taxi’ created with 6 shards, 1 replication
• Deployed using solr-scale-tk (https://github.com/LucidWorks/solr-scale-tk)
• Dataset link: https://github.com/toddwschneider/nyc-taxi-data
• More details: https://gist.github.com/kiranchitturi/
0be62fc13e4ec7f9ae5def53180ed181
• Query - simple aggregation query to calculate averages
• Streaming expressions took 2.3 mins across 6 tasks
• Deep paging took 20 minutes across 120 tasks
Query performance
Index performance
• 91.4M rows imported to Solr in 49 minutes
• Docs per second: 31K
• JDBC batch size: 5000
• Indexing batch size: 50000
• Partitions: 200
Wrap-up and Q & A
Download Fusion: http://lucidworks.com/fusion/download/
Feel free to reach out to me with questions:
kiran.chitturi@lucidworks.com / @chitturikiran
Spark Streaming: Nuts & Bolts
• Transform a stream of records into small, deterministic batches
✓ Discretized stream: sequence of RDDs
✓ Once you have an RDD, you can use all the other Spark libs (MLlib, etc)
✓ Low-latency micro batches
✓ Time to process a batch must be less than the batch interval time
• Two types of operators:
✓ Transformations (group by, join, etc)
✓ Output (send to some external sink, e.g. Solr)
• Impressive performance!
✓ 4GB/s (40M records/s) on 100 node cluster with less than 1 second latency (note: not indexing rate)
✓ Haven’t found any unbiased, reproducible performance comparisons between Storm / Spark
Spark Streaming Example: Solr as Sink
Twitter
./spark-submit	--master	MASTER	--class	com.lucidworks.spark.SparkApp	spark-solr-1.0.jar		
					twitter-to-solr	-zkHost	localhost:2181	–collection	social
Solr
JavaReceiverInputDStream<Status> tweets =
TwitterUtils.createStream(jssc, null, filters);
Various transformations / enrichments
on each tweet (e.g. sentiment analysis,
language detection)
JavaDStream<SolrInputDocument> docs = tweets.map(
new Function<Status,SolrInputDocument>() {
// Convert a twitter4j Status object into a SolrInputDocument
public SolrInputDocument call(Status status) {
SolrInputDocument doc = new SolrInputDocument();
…
return doc;
map()
class TwitterToSolrStreamProcessor
extends SparkApp.StreamProcessor
SolrSupport.indexDStreamOfDocs(zkHost, collection, 100, docs);
Slide Legend
Provided by Spark
Custom Java / Scala code
Provided by Lucidworks
Document Matching using Stored Queries
• For each document, determine which of a large set of stored queries
matches.
• Useful for alerts, alternative flow paths through a stream, etc
• Index a micro-batch into an embedded (in-memory) Solr instance and
then determine which queries match
• Matching framework; you have to decide where to load the stored
queries from and what to do when matches are found
• Scale it using Spark … need to scale to many queries, checkout Luwak
Document Matching using Stored Queries
Stored Queries
DocFilterContext
Twitter map()
Slide Legend
Provided by Spark
Custom Java / Scala code
Provided by Lucidworks
JavaReceiverInputDStream<Status> tweets =
TwitterUtils.createStream(jssc, null, filters);
JavaDStream<SolrInputDocument> docs = tweets.map(
new Function<Status,SolrInputDocument>() {
// Convert a twitter4j Status object into a SolrInputDocument
public SolrInputDocument call(Status status) {
SolrInputDocument doc = new SolrInputDocument();
…
return doc;
}});
JavaDStream<SolrInputDocument> enriched =

SolrSupport.filterDocuments(docFilterContext, …);
Get queries
Index docs into an
EmbeddedSolrServer
Initialized from configs
stored in ZooKeeper
…
ZooKeeper
Key abstraction to allow
you to plug-in how to
store the queries and
what action to take when
docs match
RDD Illustrated: Word count
map(word	=>	(word,	1))
Map words into
pairs with count of 1
(quick,1)
(brown,1)
(fox,1)
(quick,1)
(quick,1)
val	file	=		
		spark.textFile("hdfs://...")
HDFS
file RDD from HDFS
quick	brown	fox	jumped	…
quick	brownie	recipe	…
quick	drying	glue	…
………
file.flatMap(line	=>	line.split("	"))
Split lines into words
quick
brown
fox
quick
quick
……
reduceByKey(_	+	_)
Send all keys to same
reducer and sum
(quick,1)
(quick,1)
(quick,1)
(quick,3)
Shuffle
across
machine
boundaries
Executors assigned based on data-locality if possible, narrow transformations occur in same executor
Spark keeps track of the transformations made to generate each RDD
Partition 1
Partition 2
Partition 3
x
val	file	=	spark.textFile("hdfs://...")	
val	counts	=	file.flatMap(line	=>	line.split("	"))	
																	.map(word	=>	(word,	1))	
																	.reduceByKey(_	+	_)	
counts.saveAsTextFile("hdfs://...")
Understanding Resilient Distributed Datasets (RDD)
• Read-only partitioned collection of records with fault-tolerance
• Created from external system OR using a transformation of another RDD
• RDDs track the lineage of coarse-grained transformations (map, join, filter, etc)
• If a partition is lost, RDDs can be re-computed by re-playing the transformations
• User can choose to persist an RDD (for reusing during interactive data-mining)
• User can control partitioning scheme
Physical Architecture
Spark Master (daemon)
Spark Slave (daemon)
my-spark-job.jar
(w/ shaded deps)
My Spark App
SparkContext
(driver)
• Keeps track of live workers
• Web UI on port 8080
• Task Scheduler
• Restart failed tasks
Spark Executor (JVM process)
Tasks
Executor runs in separate
process than slave daemon
Spark Worker Node (1...N of these)
Each task works on some partition of a
data set to apply a transformation or action
Cache
Losing a master prevents new
applications from being executed
Can achieve HA using ZooKeeper
and multiple master nodes
Tasks are assigned
based on data-locality
When selecting which node to execute a task on,
the master takes into account data locality
• RDD Graph
• DAG Scheduler
• Block tracker
• Shuffle tracker
Spark vs. Hadoop’s Map/Reduce
Operators File System Fault-Tolerance Ecosystem
Hadoop Map-Reduce HDFS
S3
Replicated
blocks
Java API
Pig
Hive
Spark filter, map,
flatMap, join,
groupByKey,
reduce,
sortByKey,
count, distinct,
top, cogroup,
etc …
HDFS
S3
Tachyon
Immutable RDD
lineage
Python / Scala /
Java API
SparkSQL
GraphX
MLlib
SparkR

More Related Content

What's hot

Webinar: What's New in Solr 7
Webinar: What's New in Solr 7 Webinar: What's New in Solr 7
Webinar: What's New in Solr 7
Lucidworks
 
Streaming Aggregation in Solr - New Horizons for Search: Presented by Erick E...
Streaming Aggregation in Solr - New Horizons for Search: Presented by Erick E...Streaming Aggregation in Solr - New Horizons for Search: Presented by Erick E...
Streaming Aggregation in Solr - New Horizons for Search: Presented by Erick E...
Lucidworks
 
Spark Programming
Spark ProgrammingSpark Programming
Spark Programming
Taewook Eom
 
Webinar: What's New in Solr 6
Webinar: What's New in Solr 6Webinar: What's New in Solr 6
Webinar: What's New in Solr 6
Lucidworks
 
Rebuilding Solr 6 Examples - Layer by Layer: Presented by Alexandre Rafalovit...
Rebuilding Solr 6 Examples - Layer by Layer: Presented by Alexandre Rafalovit...Rebuilding Solr 6 Examples - Layer by Layer: Presented by Alexandre Rafalovit...
Rebuilding Solr 6 Examples - Layer by Layer: Presented by Alexandre Rafalovit...
Lucidworks
 
Automated Spark Deployment With Declarative Infrastructure
Automated Spark Deployment With Declarative InfrastructureAutomated Spark Deployment With Declarative Infrastructure
Automated Spark Deployment With Declarative Infrastructure
Spark Summit
 
Rapid Prototyping with Solr
Rapid Prototyping with SolrRapid Prototyping with Solr
Rapid Prototyping with SolrErik Hatcher
 
Simple Fuzzy Name Matching in Solr: Presented by Chris Mack, Basis Technology
Simple Fuzzy Name Matching in Solr: Presented by Chris Mack, Basis TechnologySimple Fuzzy Name Matching in Solr: Presented by Chris Mack, Basis Technology
Simple Fuzzy Name Matching in Solr: Presented by Chris Mack, Basis Technology
Lucidworks
 
Easy, scalable, fault tolerant stream processing with structured streaming - ...
Easy, scalable, fault tolerant stream processing with structured streaming - ...Easy, scalable, fault tolerant stream processing with structured streaming - ...
Easy, scalable, fault tolerant stream processing with structured streaming - ...
Databricks
 
DataEngConf SF16 - Spark SQL Workshop
DataEngConf SF16 - Spark SQL WorkshopDataEngConf SF16 - Spark SQL Workshop
DataEngConf SF16 - Spark SQL Workshop
Hakka Labs
 
"Spark Search" - In-memory, Distributed Search with Lucene, Spark, and Tachyo...
"Spark Search" - In-memory, Distributed Search with Lucene, Spark, and Tachyo..."Spark Search" - In-memory, Distributed Search with Lucene, Spark, and Tachyo...
"Spark Search" - In-memory, Distributed Search with Lucene, Spark, and Tachyo...
Lucidworks
 
Solr 6 Feature Preview
Solr 6 Feature PreviewSolr 6 Feature Preview
Solr 6 Feature Preview
Yonik Seeley
 
Rapid Prototyping with Solr
Rapid Prototyping with SolrRapid Prototyping with Solr
Rapid Prototyping with Solr
Erik Hatcher
 
Introduction to Apache Solr
Introduction to Apache SolrIntroduction to Apache Solr
Introduction to Apache Solr
Christos Manios
 
Introduce to Spark sql 1.3.0
Introduce to Spark sql 1.3.0 Introduce to Spark sql 1.3.0
Introduce to Spark sql 1.3.0
Bryan Yang
 
SQL to Hive Cheat Sheet
SQL to Hive Cheat SheetSQL to Hive Cheat Sheet
SQL to Hive Cheat Sheet
Hortonworks
 
Apache Solr crash course
Apache Solr crash courseApache Solr crash course
Apache Solr crash courseTommaso Teofili
 
Beyond SQL: Speeding up Spark with DataFrames
Beyond SQL: Speeding up Spark with DataFramesBeyond SQL: Speeding up Spark with DataFrames
Beyond SQL: Speeding up Spark with DataFrames
Databricks
 
Analytics and Graph Traversal with Solr - Yonik Seeley, Cloudera
Analytics and Graph Traversal with Solr - Yonik Seeley, ClouderaAnalytics and Graph Traversal with Solr - Yonik Seeley, Cloudera
Analytics and Graph Traversal with Solr - Yonik Seeley, Cloudera
Lucidworks
 
Leveraging the Power of Solr with Spark: Presented by Johannes Weigend, QAware
Leveraging the Power of Solr with Spark: Presented by Johannes Weigend, QAwareLeveraging the Power of Solr with Spark: Presented by Johannes Weigend, QAware
Leveraging the Power of Solr with Spark: Presented by Johannes Weigend, QAware
Lucidworks
 

What's hot (20)

Webinar: What's New in Solr 7
Webinar: What's New in Solr 7 Webinar: What's New in Solr 7
Webinar: What's New in Solr 7
 
Streaming Aggregation in Solr - New Horizons for Search: Presented by Erick E...
Streaming Aggregation in Solr - New Horizons for Search: Presented by Erick E...Streaming Aggregation in Solr - New Horizons for Search: Presented by Erick E...
Streaming Aggregation in Solr - New Horizons for Search: Presented by Erick E...
 
Spark Programming
Spark ProgrammingSpark Programming
Spark Programming
 
Webinar: What's New in Solr 6
Webinar: What's New in Solr 6Webinar: What's New in Solr 6
Webinar: What's New in Solr 6
 
Rebuilding Solr 6 Examples - Layer by Layer: Presented by Alexandre Rafalovit...
Rebuilding Solr 6 Examples - Layer by Layer: Presented by Alexandre Rafalovit...Rebuilding Solr 6 Examples - Layer by Layer: Presented by Alexandre Rafalovit...
Rebuilding Solr 6 Examples - Layer by Layer: Presented by Alexandre Rafalovit...
 
Automated Spark Deployment With Declarative Infrastructure
Automated Spark Deployment With Declarative InfrastructureAutomated Spark Deployment With Declarative Infrastructure
Automated Spark Deployment With Declarative Infrastructure
 
Rapid Prototyping with Solr
Rapid Prototyping with SolrRapid Prototyping with Solr
Rapid Prototyping with Solr
 
Simple Fuzzy Name Matching in Solr: Presented by Chris Mack, Basis Technology
Simple Fuzzy Name Matching in Solr: Presented by Chris Mack, Basis TechnologySimple Fuzzy Name Matching in Solr: Presented by Chris Mack, Basis Technology
Simple Fuzzy Name Matching in Solr: Presented by Chris Mack, Basis Technology
 
Easy, scalable, fault tolerant stream processing with structured streaming - ...
Easy, scalable, fault tolerant stream processing with structured streaming - ...Easy, scalable, fault tolerant stream processing with structured streaming - ...
Easy, scalable, fault tolerant stream processing with structured streaming - ...
 
DataEngConf SF16 - Spark SQL Workshop
DataEngConf SF16 - Spark SQL WorkshopDataEngConf SF16 - Spark SQL Workshop
DataEngConf SF16 - Spark SQL Workshop
 
"Spark Search" - In-memory, Distributed Search with Lucene, Spark, and Tachyo...
"Spark Search" - In-memory, Distributed Search with Lucene, Spark, and Tachyo..."Spark Search" - In-memory, Distributed Search with Lucene, Spark, and Tachyo...
"Spark Search" - In-memory, Distributed Search with Lucene, Spark, and Tachyo...
 
Solr 6 Feature Preview
Solr 6 Feature PreviewSolr 6 Feature Preview
Solr 6 Feature Preview
 
Rapid Prototyping with Solr
Rapid Prototyping with SolrRapid Prototyping with Solr
Rapid Prototyping with Solr
 
Introduction to Apache Solr
Introduction to Apache SolrIntroduction to Apache Solr
Introduction to Apache Solr
 
Introduce to Spark sql 1.3.0
Introduce to Spark sql 1.3.0 Introduce to Spark sql 1.3.0
Introduce to Spark sql 1.3.0
 
SQL to Hive Cheat Sheet
SQL to Hive Cheat SheetSQL to Hive Cheat Sheet
SQL to Hive Cheat Sheet
 
Apache Solr crash course
Apache Solr crash courseApache Solr crash course
Apache Solr crash course
 
Beyond SQL: Speeding up Spark with DataFrames
Beyond SQL: Speeding up Spark with DataFramesBeyond SQL: Speeding up Spark with DataFrames
Beyond SQL: Speeding up Spark with DataFrames
 
Analytics and Graph Traversal with Solr - Yonik Seeley, Cloudera
Analytics and Graph Traversal with Solr - Yonik Seeley, ClouderaAnalytics and Graph Traversal with Solr - Yonik Seeley, Cloudera
Analytics and Graph Traversal with Solr - Yonik Seeley, Cloudera
 
Leveraging the Power of Solr with Spark: Presented by Johannes Weigend, QAware
Leveraging the Power of Solr with Spark: Presented by Johannes Weigend, QAwareLeveraging the Power of Solr with Spark: Presented by Johannes Weigend, QAware
Leveraging the Power of Solr with Spark: Presented by Johannes Weigend, QAware
 

Similar to Solr as a Spark SQL Datasource

Solr As A SparkSQL DataSource
Solr As A SparkSQL DataSourceSolr As A SparkSQL DataSource
Solr As A SparkSQL DataSource
Spark Summit
 
NYC Lucene/Solr Meetup: Spark / Solr
NYC Lucene/Solr Meetup: Spark / SolrNYC Lucene/Solr Meetup: Spark / Solr
NYC Lucene/Solr Meetup: Spark / Solrthelabdude
 
Berlin Buzz Words - Apache Drill by Ted Dunning & Michael Hausenblas
Berlin Buzz Words - Apache Drill by Ted Dunning & Michael HausenblasBerlin Buzz Words - Apache Drill by Ted Dunning & Michael Hausenblas
Berlin Buzz Words - Apache Drill by Ted Dunning & Michael Hausenblas
MapR Technologies
 
Apache Spark RDDs
Apache Spark RDDsApache Spark RDDs
Apache Spark RDDs
Dean Chen
 
20170126 big data processing
20170126 big data processing20170126 big data processing
20170126 big data processing
Vienna Data Science Group
 
Jump Start with Apache Spark 2.0 on Databricks
Jump Start with Apache Spark 2.0 on DatabricksJump Start with Apache Spark 2.0 on Databricks
Jump Start with Apache Spark 2.0 on Databricks
Databricks
 
Intro to Spark
Intro to SparkIntro to Spark
Intro to Spark
Kyle Burke
 
Large scale, interactive ad-hoc queries over different datastores with Apache...
Large scale, interactive ad-hoc queries over different datastores with Apache...Large scale, interactive ad-hoc queries over different datastores with Apache...
Large scale, interactive ad-hoc queries over different datastores with Apache...
jaxLondonConference
 
xPatterns on Spark, Tachyon and Mesos - Bucharest meetup
xPatterns on Spark, Tachyon and Mesos - Bucharest meetupxPatterns on Spark, Tachyon and Mesos - Bucharest meetup
xPatterns on Spark, Tachyon and Mesos - Bucharest meetup
Radu Chilom
 
Building highly scalable data pipelines with Apache Spark
Building highly scalable data pipelines with Apache SparkBuilding highly scalable data pipelines with Apache Spark
Building highly scalable data pipelines with Apache Spark
Martin Toshev
 
Shark
SharkShark
Shark
Alex Ivy
 
Dive into spark2
Dive into spark2Dive into spark2
Dive into spark2
Gal Marder
 
Ingesting hdfs intosolrusingsparktrimmed
Ingesting hdfs intosolrusingsparktrimmedIngesting hdfs intosolrusingsparktrimmed
Ingesting hdfs intosolrusingsparktrimmed
whoschek
 
Data Engineering with Solr and Spark
Data Engineering with Solr and SparkData Engineering with Solr and Spark
Data Engineering with Solr and Spark
Lucidworks
 
Introduction to Spark (Intern Event Presentation)
Introduction to Spark (Intern Event Presentation)Introduction to Spark (Intern Event Presentation)
Introduction to Spark (Intern Event Presentation)
Databricks
 
20130912 YTC_Reynold Xin_Spark and Shark
20130912 YTC_Reynold Xin_Spark and Shark20130912 YTC_Reynold Xin_Spark and Shark
20130912 YTC_Reynold Xin_Spark and Shark
YahooTechConference
 
An Introduct to Spark - Atlanta Spark Meetup
An Introduct to Spark - Atlanta Spark MeetupAn Introduct to Spark - Atlanta Spark Meetup
An Introduct to Spark - Atlanta Spark Meetup
jlacefie
 
An Introduction to Spark
An Introduction to SparkAn Introduction to Spark
An Introduction to Sparkjlacefie
 
Intro to Spark and Spark SQL
Intro to Spark and Spark SQLIntro to Spark and Spark SQL
Intro to Spark and Spark SQL
jeykottalam
 
Leveraging Hadoop in your PostgreSQL Environment
Leveraging Hadoop in your PostgreSQL EnvironmentLeveraging Hadoop in your PostgreSQL Environment
Leveraging Hadoop in your PostgreSQL Environment
Jim Mlodgenski
 

Similar to Solr as a Spark SQL Datasource (20)

Solr As A SparkSQL DataSource
Solr As A SparkSQL DataSourceSolr As A SparkSQL DataSource
Solr As A SparkSQL DataSource
 
NYC Lucene/Solr Meetup: Spark / Solr
NYC Lucene/Solr Meetup: Spark / SolrNYC Lucene/Solr Meetup: Spark / Solr
NYC Lucene/Solr Meetup: Spark / Solr
 
Berlin Buzz Words - Apache Drill by Ted Dunning & Michael Hausenblas
Berlin Buzz Words - Apache Drill by Ted Dunning & Michael HausenblasBerlin Buzz Words - Apache Drill by Ted Dunning & Michael Hausenblas
Berlin Buzz Words - Apache Drill by Ted Dunning & Michael Hausenblas
 
Apache Spark RDDs
Apache Spark RDDsApache Spark RDDs
Apache Spark RDDs
 
20170126 big data processing
20170126 big data processing20170126 big data processing
20170126 big data processing
 
Jump Start with Apache Spark 2.0 on Databricks
Jump Start with Apache Spark 2.0 on DatabricksJump Start with Apache Spark 2.0 on Databricks
Jump Start with Apache Spark 2.0 on Databricks
 
Intro to Spark
Intro to SparkIntro to Spark
Intro to Spark
 
Large scale, interactive ad-hoc queries over different datastores with Apache...
Large scale, interactive ad-hoc queries over different datastores with Apache...Large scale, interactive ad-hoc queries over different datastores with Apache...
Large scale, interactive ad-hoc queries over different datastores with Apache...
 
xPatterns on Spark, Tachyon and Mesos - Bucharest meetup
xPatterns on Spark, Tachyon and Mesos - Bucharest meetupxPatterns on Spark, Tachyon and Mesos - Bucharest meetup
xPatterns on Spark, Tachyon and Mesos - Bucharest meetup
 
Building highly scalable data pipelines with Apache Spark
Building highly scalable data pipelines with Apache SparkBuilding highly scalable data pipelines with Apache Spark
Building highly scalable data pipelines with Apache Spark
 
Shark
SharkShark
Shark
 
Dive into spark2
Dive into spark2Dive into spark2
Dive into spark2
 
Ingesting hdfs intosolrusingsparktrimmed
Ingesting hdfs intosolrusingsparktrimmedIngesting hdfs intosolrusingsparktrimmed
Ingesting hdfs intosolrusingsparktrimmed
 
Data Engineering with Solr and Spark
Data Engineering with Solr and SparkData Engineering with Solr and Spark
Data Engineering with Solr and Spark
 
Introduction to Spark (Intern Event Presentation)
Introduction to Spark (Intern Event Presentation)Introduction to Spark (Intern Event Presentation)
Introduction to Spark (Intern Event Presentation)
 
20130912 YTC_Reynold Xin_Spark and Shark
20130912 YTC_Reynold Xin_Spark and Shark20130912 YTC_Reynold Xin_Spark and Shark
20130912 YTC_Reynold Xin_Spark and Shark
 
An Introduct to Spark - Atlanta Spark Meetup
An Introduct to Spark - Atlanta Spark MeetupAn Introduct to Spark - Atlanta Spark Meetup
An Introduct to Spark - Atlanta Spark Meetup
 
An Introduction to Spark
An Introduction to SparkAn Introduction to Spark
An Introduction to Spark
 
Intro to Spark and Spark SQL
Intro to Spark and Spark SQLIntro to Spark and Spark SQL
Intro to Spark and Spark SQL
 
Leveraging Hadoop in your PostgreSQL Environment
Leveraging Hadoop in your PostgreSQL EnvironmentLeveraging Hadoop in your PostgreSQL Environment
Leveraging Hadoop in your PostgreSQL Environment
 

Recently uploaded

English lab ppt no titlespecENG PPTt.pdf
English lab ppt no titlespecENG PPTt.pdfEnglish lab ppt no titlespecENG PPTt.pdf
English lab ppt no titlespecENG PPTt.pdf
BrazilAccount1
 
ethical hacking-mobile hacking methods.ppt
ethical hacking-mobile hacking methods.pptethical hacking-mobile hacking methods.ppt
ethical hacking-mobile hacking methods.ppt
Jayaprasanna4
 
HYDROPOWER - Hydroelectric power generation
HYDROPOWER - Hydroelectric power generationHYDROPOWER - Hydroelectric power generation
HYDROPOWER - Hydroelectric power generation
Robbie Edward Sayers
 
The Benefits and Techniques of Trenchless Pipe Repair.pdf
The Benefits and Techniques of Trenchless Pipe Repair.pdfThe Benefits and Techniques of Trenchless Pipe Repair.pdf
The Benefits and Techniques of Trenchless Pipe Repair.pdf
Pipe Restoration Solutions
 
Governing Equations for Fundamental Aerodynamics_Anderson2010.pdf
Governing Equations for Fundamental Aerodynamics_Anderson2010.pdfGoverning Equations for Fundamental Aerodynamics_Anderson2010.pdf
Governing Equations for Fundamental Aerodynamics_Anderson2010.pdf
WENKENLI1
 
Fundamentals of Electric Drives and its applications.pptx
Fundamentals of Electric Drives and its applications.pptxFundamentals of Electric Drives and its applications.pptx
Fundamentals of Electric Drives and its applications.pptx
manasideore6
 
power quality voltage fluctuation UNIT - I.pptx
power quality voltage fluctuation UNIT - I.pptxpower quality voltage fluctuation UNIT - I.pptx
power quality voltage fluctuation UNIT - I.pptx
ViniHema
 
CME397 Surface Engineering- Professional Elective
CME397 Surface Engineering- Professional ElectiveCME397 Surface Engineering- Professional Elective
CME397 Surface Engineering- Professional Elective
karthi keyan
 
road safety engineering r s e unit 3.pdf
road safety engineering  r s e unit 3.pdfroad safety engineering  r s e unit 3.pdf
road safety engineering r s e unit 3.pdf
VENKATESHvenky89705
 
在线办理(ANU毕业证书)澳洲国立大学毕业证录取通知书一模一样
在线办理(ANU毕业证书)澳洲国立大学毕业证录取通知书一模一样在线办理(ANU毕业证书)澳洲国立大学毕业证录取通知书一模一样
在线办理(ANU毕业证书)澳洲国立大学毕业证录取通知书一模一样
obonagu
 
The role of big data in decision making.
The role of big data in decision making.The role of big data in decision making.
The role of big data in decision making.
ankuprajapati0525
 
block diagram and signal flow graph representation
block diagram and signal flow graph representationblock diagram and signal flow graph representation
block diagram and signal flow graph representation
Divya Somashekar
 
DESIGN A COTTON SEED SEPARATION MACHINE.docx
DESIGN A COTTON SEED SEPARATION MACHINE.docxDESIGN A COTTON SEED SEPARATION MACHINE.docx
DESIGN A COTTON SEED SEPARATION MACHINE.docx
FluxPrime1
 
Hybrid optimization of pumped hydro system and solar- Engr. Abdul-Azeez.pdf
Hybrid optimization of pumped hydro system and solar- Engr. Abdul-Azeez.pdfHybrid optimization of pumped hydro system and solar- Engr. Abdul-Azeez.pdf
Hybrid optimization of pumped hydro system and solar- Engr. Abdul-Azeez.pdf
fxintegritypublishin
 
Planning Of Procurement o different goods and services
Planning Of Procurement o different goods and servicesPlanning Of Procurement o different goods and services
Planning Of Procurement o different goods and services
JoytuBarua2
 
Standard Reomte Control Interface - Neometrix
Standard Reomte Control Interface - NeometrixStandard Reomte Control Interface - Neometrix
Standard Reomte Control Interface - Neometrix
Neometrix_Engineering_Pvt_Ltd
 
Gen AI Study Jams _ For the GDSC Leads in India.pdf
Gen AI Study Jams _ For the GDSC Leads in India.pdfGen AI Study Jams _ For the GDSC Leads in India.pdf
Gen AI Study Jams _ For the GDSC Leads in India.pdf
gdsczhcet
 
AKS UNIVERSITY Satna Final Year Project By OM Hardaha.pdf
AKS UNIVERSITY Satna Final Year Project By OM Hardaha.pdfAKS UNIVERSITY Satna Final Year Project By OM Hardaha.pdf
AKS UNIVERSITY Satna Final Year Project By OM Hardaha.pdf
SamSarthak3
 
ethical hacking in wireless-hacking1.ppt
ethical hacking in wireless-hacking1.pptethical hacking in wireless-hacking1.ppt
ethical hacking in wireless-hacking1.ppt
Jayaprasanna4
 
RAT: Retrieval Augmented Thoughts Elicit Context-Aware Reasoning in Long-Hori...
RAT: Retrieval Augmented Thoughts Elicit Context-Aware Reasoning in Long-Hori...RAT: Retrieval Augmented Thoughts Elicit Context-Aware Reasoning in Long-Hori...
RAT: Retrieval Augmented Thoughts Elicit Context-Aware Reasoning in Long-Hori...
thanhdowork
 

Recently uploaded (20)

English lab ppt no titlespecENG PPTt.pdf
English lab ppt no titlespecENG PPTt.pdfEnglish lab ppt no titlespecENG PPTt.pdf
English lab ppt no titlespecENG PPTt.pdf
 
ethical hacking-mobile hacking methods.ppt
ethical hacking-mobile hacking methods.pptethical hacking-mobile hacking methods.ppt
ethical hacking-mobile hacking methods.ppt
 
HYDROPOWER - Hydroelectric power generation
HYDROPOWER - Hydroelectric power generationHYDROPOWER - Hydroelectric power generation
HYDROPOWER - Hydroelectric power generation
 
The Benefits and Techniques of Trenchless Pipe Repair.pdf
The Benefits and Techniques of Trenchless Pipe Repair.pdfThe Benefits and Techniques of Trenchless Pipe Repair.pdf
The Benefits and Techniques of Trenchless Pipe Repair.pdf
 
Governing Equations for Fundamental Aerodynamics_Anderson2010.pdf
Governing Equations for Fundamental Aerodynamics_Anderson2010.pdfGoverning Equations for Fundamental Aerodynamics_Anderson2010.pdf
Governing Equations for Fundamental Aerodynamics_Anderson2010.pdf
 
Fundamentals of Electric Drives and its applications.pptx
Fundamentals of Electric Drives and its applications.pptxFundamentals of Electric Drives and its applications.pptx
Fundamentals of Electric Drives and its applications.pptx
 
power quality voltage fluctuation UNIT - I.pptx
power quality voltage fluctuation UNIT - I.pptxpower quality voltage fluctuation UNIT - I.pptx
power quality voltage fluctuation UNIT - I.pptx
 
CME397 Surface Engineering- Professional Elective
CME397 Surface Engineering- Professional ElectiveCME397 Surface Engineering- Professional Elective
CME397 Surface Engineering- Professional Elective
 
road safety engineering r s e unit 3.pdf
road safety engineering  r s e unit 3.pdfroad safety engineering  r s e unit 3.pdf
road safety engineering r s e unit 3.pdf
 
在线办理(ANU毕业证书)澳洲国立大学毕业证录取通知书一模一样
在线办理(ANU毕业证书)澳洲国立大学毕业证录取通知书一模一样在线办理(ANU毕业证书)澳洲国立大学毕业证录取通知书一模一样
在线办理(ANU毕业证书)澳洲国立大学毕业证录取通知书一模一样
 
The role of big data in decision making.
The role of big data in decision making.The role of big data in decision making.
The role of big data in decision making.
 
block diagram and signal flow graph representation
block diagram and signal flow graph representationblock diagram and signal flow graph representation
block diagram and signal flow graph representation
 
DESIGN A COTTON SEED SEPARATION MACHINE.docx
DESIGN A COTTON SEED SEPARATION MACHINE.docxDESIGN A COTTON SEED SEPARATION MACHINE.docx
DESIGN A COTTON SEED SEPARATION MACHINE.docx
 
Hybrid optimization of pumped hydro system and solar- Engr. Abdul-Azeez.pdf
Hybrid optimization of pumped hydro system and solar- Engr. Abdul-Azeez.pdfHybrid optimization of pumped hydro system and solar- Engr. Abdul-Azeez.pdf
Hybrid optimization of pumped hydro system and solar- Engr. Abdul-Azeez.pdf
 
Planning Of Procurement o different goods and services
Planning Of Procurement o different goods and servicesPlanning Of Procurement o different goods and services
Planning Of Procurement o different goods and services
 
Standard Reomte Control Interface - Neometrix
Standard Reomte Control Interface - NeometrixStandard Reomte Control Interface - Neometrix
Standard Reomte Control Interface - Neometrix
 
Gen AI Study Jams _ For the GDSC Leads in India.pdf
Gen AI Study Jams _ For the GDSC Leads in India.pdfGen AI Study Jams _ For the GDSC Leads in India.pdf
Gen AI Study Jams _ For the GDSC Leads in India.pdf
 
AKS UNIVERSITY Satna Final Year Project By OM Hardaha.pdf
AKS UNIVERSITY Satna Final Year Project By OM Hardaha.pdfAKS UNIVERSITY Satna Final Year Project By OM Hardaha.pdf
AKS UNIVERSITY Satna Final Year Project By OM Hardaha.pdf
 
ethical hacking in wireless-hacking1.ppt
ethical hacking in wireless-hacking1.pptethical hacking in wireless-hacking1.ppt
ethical hacking in wireless-hacking1.ppt
 
RAT: Retrieval Augmented Thoughts Elicit Context-Aware Reasoning in Long-Hori...
RAT: Retrieval Augmented Thoughts Elicit Context-Aware Reasoning in Long-Hori...RAT: Retrieval Augmented Thoughts Elicit Context-Aware Reasoning in Long-Hori...
RAT: Retrieval Augmented Thoughts Elicit Context-Aware Reasoning in Long-Hori...
 

Solr as a Spark SQL Datasource

  • 1.
  • 2. Solr as a Spark SQL Datasource Kiran Chitturi, Lucidworks
  • 3. Solr & Spark • A few interesting things about Spark • Overview of SparkSQL and DataFrames • Solr as a SparkSQL DataSource in depth • Use Lucene for text analysis in ML pipelines • Example Use Case: Lucidworks Fusion and Spark
  • 4. The standard for enterprise search. of Fortune 500 uses Solr. 90%
  • 5. What’s interesting about Spark? • Wealth of overview / getting started resources on the Web ➢ Start here -> https://spark.apache.org/ ➢ Should READ! https://www.cs.berkeley.edu/~matei/papers/2012/nsdi_spark.pdf • Faster, more modernized alternative to MapReduce ➢ Spark running on Hadoop sorted 100TB in 23 minutes (3x faster than Yahoo’s previous record while using10x less computing power) • Unified platform for Big Data ➢ Great for iterative algorithms (PageRank, K-Means, Logistic regression) & interactive data mining ➢ Runs on YARN, Mesos, and plays well with HDFS • Nice API for Java, Scala, Python, R, and sometimes SQL … REPL interface too • >14,200 Issues in JIRA, 1000+ code contributors, 2.0 coming soon!
  • 6. Spark Components Spark Core Spark SQL Spark Streaming MLlib (machine learning) GraphX (BSP) Hadoop YARN Mesos Standalone HDFS Execution Model The Shuffle Caching components engine cluster mgmt Alluxio (formerly Tachyon) languages Scala Java Python R shared memory
  • 7. Physical Architecture (Standalone) Spark Master (daemon) Spark Slave (daemon) my-spark-job.jar (w/ shaded deps) My Spark App SparkContext (driver) • Keeps track of live workers • Web UI on port 8080 • Task Scheduler • Restart failed tasks Spark Executor (JVM process) Tasks Executor runs in separate process than slave daemon Spark Worker Node (1...N of these) Each task works on some partition of a data set to apply a transformation or action Cache Losing a master prevents new applications from being executed Can achieve HA using ZooKeeper and multiple master nodes Tasks are assigned based on data-locality When selecting which node to execute a task on, the master takes into account data locality • RDD Graph • DAG Scheduler • Block tracker • Shuffle tracker
  • 8. Spark SQL • DataSource API for reading from and writing to external data sources • DataFrame is an RDD[Row] + schema • Secret sauce is logical plan optimizer • SQL or relational operators on DF • JDBC / ODBC • UDFs! • Machine Learning Pipelines
  • 9. Solr as a Spark SQL Data Source • Read/write data from/to Solr as DataFrame • Use Solr Schema API to access field-level metadata • Push predicates down into Solr query constructs, e.g. fq clause • Deep-paging, shard partitioning, intra-shard splitting, streaming results // Connect to Solr val opts = Map("zkhost" -> "localhost:9983", "collection" -> "nyc_trips") val solrDF = sqlContext.read.format("solr").options(opts).load // Register DF as temp table solrDF.registerTempTable("trips") // Perform SQL queries sqlContext.sql("SELECT avg(tip_amount), avg(fare_amount) FROM trips").show()
  • 10. Shard 1 Shard 2 socialdata collection Partition 1 (spark task) solr.DefaultSource Partition 2 (spark task) Spark Driver App ZooKeeper Read collection metadata (num shards, replica URLs, etc) Table Scan Query: q=*:*&rows=1000& distrib=false&cursorMark=* Results streamed back from Solr Parallelizequeryexecution sqlContext.load(“solr”,opts) DataFrame (schema + RDD<Row>) Map("collection” -> “nyc_trips”, “zkHost” -> “…”) SolrRDD: Reading data from Solr into Spark
  • 11. Shard 1 Shard 2 socialdata collection Partition 1 (spark task) solr.DefaultSource Partition 3 (spark task) Spark Driver App ZooKeeper Read collection metadata (num shards, replica URLs, etc) Table Scan Query: q=*:*&rows=1000& distrib=false&cursorMark=* &split_field:[aa TO ba} Parallelizequeryexecution sqlContext.load(“solr”,opts) DataFrame (schema + RDD<Row>) Map(“collection” -> “nyc_trips”, “zkHost” -> “…”) SolrRDD: Reading data from Solr into Spark Partition 2 (spark task) &split_field:[ba TO bz] Partition 4 (spark task) &split_field:[aa TO ac} &split_field:[ac TO ae]
  • 12. Data-locality Hint • SolrRDD extends RDD[SolrDocument] (written in Scala) • Give hint to Spark task scheduler about where data lives override def getPreferredLocations(split: Partition): Seq[String] = { // return preferred hostname for a Solr partition
 } • Useful when Spark executor and Solr replicas live on same physical host, as we do in Fusion • Query to a shard has a “preferred” replica; can fallback to other replicas if the preferred goes down (will be in 2.1)
  • 13. Solr Streaming API for fast reads • Contributed to spark-solr by Bloomberg team (we PRs) • Extremely fast “table scans” over large result sets in Solr • Relies on a column-oriented data structure in Lucene: docValues • DocValues help speed up faceting and sorting too! • Coming soon! Push SQL predicates down into Solr’s Parallel SQL engine available in Solr 6.x
  • 14. Writing to Solr (aka indexing) • Cloud-aware client sends updates to shard leaders in parallel • Solr Schema API used to create fields on-the-fly using the DataFrame schema • Better parallelism than traditional approaches like Solr DIH val dbOpts = Map( "url" -> "jdbc:postgresql:mydb", "dbtable" -> “schema.table", "partitionColumn" -> "foo", "numPartitions" -> "10") val jdbcDF = sqlContext.read.format("jdbc").options(dbOpts).load val solrOpts = Map("zkhost" -> "localhost:9983", "collection" -> "mycoll") jdbcDF.write.format("solr").options(solrOpts).mode(SaveMode.Overwrite).save
  • 15. Solr / Lucene Analyzers for Spark ML Pipelines • Spark ML Pipeline provides nice API for defining stages to train / predict ML models • Crazy idea ~ use battle-hardened Lucene for text analysis in Spark • Pipelines support import/export (work in progress, more coming in Spark 2.0) • Can try different text analysis techniques during cross-validation DF Spark ML Pipeline Lucene Analyzer HashingTF Standard Scaler Trained Model (SVM) save https://lucidworks.com/blog/2016/04/13/spark-solr-lucenetextanalyzer/
  • 16.
  • 17. Fusion & Spark • spark-solr 2.0.1 released, built into Fusion 2.4 • Users leave evidence of their needs & experience as they use your app • Fusion closes the feedback loop to improve results based on user “signals” • Train and serve ML Pipeline and mllib based Machine Learning models • Run custom Scala “script” jobs in the background in Fusion ➢Complex aggregation jobs (see next slide) ➢Unsupervised learning (LDA topic modeling) ➢Re-train supervised models as new training data flows in
  • 18. Scheduled Fusion job to compute stats for user sessions val opts = Map("zkhost" -> "localhost:9983”, "collection" -> "apachelogs”) var logEvents = sqlContext.read.format("solr").options(opts).load logEvents.registerTempTable("logs”) sqlContext.udf.register("ts2ms", (d: java.sql.Timestamp) => d.getTime) sqlContext.udf.register("asInt", (b: String) => b.toInt) val sessions = sqlContext.sql(""" |SELECT *, sum(IF(diff_ms > 30000, 1, 0)) |OVER (PARTITION BY clientip ORDER BY ts) session_id |FROM (SELECT *, ts2ms(ts) - lag(ts2ms(ts)) |OVER (PARTITION BY clientip ORDER BY ts) as diff_ms FROM logs) tmp """.stripMargin) sessions.registerTempTable("sessions") var sessionsAgg = sqlContext.sql(""" |SELECT concat_ws('||', clientip,session_id) as id, | first(clientip) as clientip, | min(ts) as session_start, | max(ts) as session_end, | (ts2ms(max(ts)) - ts2ms(min(ts))) as session_len_ms_l, | count(*) as total_requests_l |FROM sessions |GROUP BY clientip,session_id """.stripMargin) sessionsAgg.write.format("solr").options(Map("zkhost" -> "localhost:9983", "collection" -> "apachelogs_signals_aggr")) .mode(org.apache.spark.sql.SaveMode.Overwrite).save
  • 19. Getting started with spark-solr • Import package via maven ./bin/spark-shell --packages "com.lucidworks.solr:spark-solr:2.0.1" • Build from source git clone https://github.com/LucidWorks/spark-solr cd spark-solr mvn clean package -DskipTests ./bin/spark-shell --jars 2.1.0-SNAPSHOT.jar
  • 20. Example : Deep paging via shards // Connect to Solr val opts = Map( "zkhost" -> "localhost:9983", "collection" -> "nyc_trips") val solrDF = sqlContext.read.format("solr").options(opts).load // Register DF as temp table solrDF.registerTempTable("trips") sqlContext.sql("SELECT * FROM trips LIMIT 2").show()
  • 21. Example : Deep paging with intra shard splitting // Connect to Solr val opts = Map( "zkhost" -> "localhost:9983", "collection" -> "nyc_trips", "splits" -> "true") val solrDF = sqlContext.read.format("solr").options(opts).load // Register DF as temp table solrDF.registerTempTable("trips") sqlContext.sql("SELECT * FROM trips").count()
  • 22. Example : Streaming API (/export handler) // Connect to Solr val opts = Map( "zkhost" -> "localhost:9983", "collection" -> "nyc_trips") val solrDF = sqlContext.read.format("solr").options(opts).load // Register DF as temp table solrDF.registerTempTable("trips") sqlContext.sql("SELECT avg(tip_amount), avg(fare_amount) FROM trips").show()
  • 23. Performance test • NYC taxi data (30 months - 91.7M rows) • Dataset loaded in to AWS RDS instance (Postgres) • 3 EC2 nodes of r3.2x large instances • Solr and Spark instances co-located together • Collection ‘nyc-taxi’ created with 6 shards, 1 replication • Deployed using solr-scale-tk (https://github.com/LucidWorks/solr-scale-tk) • Dataset link: https://github.com/toddwschneider/nyc-taxi-data • More details: https://gist.github.com/kiranchitturi/ 0be62fc13e4ec7f9ae5def53180ed181
  • 24. • Query - simple aggregation query to calculate averages • Streaming expressions took 2.3 mins across 6 tasks • Deep paging took 20 minutes across 120 tasks Query performance
  • 25. Index performance • 91.4M rows imported to Solr in 49 minutes • Docs per second: 31K • JDBC batch size: 5000 • Indexing batch size: 50000 • Partitions: 200
  • 26. Wrap-up and Q & A Download Fusion: http://lucidworks.com/fusion/download/ Feel free to reach out to me with questions: kiran.chitturi@lucidworks.com / @chitturikiran
  • 27. Spark Streaming: Nuts & Bolts • Transform a stream of records into small, deterministic batches ✓ Discretized stream: sequence of RDDs ✓ Once you have an RDD, you can use all the other Spark libs (MLlib, etc) ✓ Low-latency micro batches ✓ Time to process a batch must be less than the batch interval time • Two types of operators: ✓ Transformations (group by, join, etc) ✓ Output (send to some external sink, e.g. Solr) • Impressive performance! ✓ 4GB/s (40M records/s) on 100 node cluster with less than 1 second latency (note: not indexing rate) ✓ Haven’t found any unbiased, reproducible performance comparisons between Storm / Spark
  • 28. Spark Streaming Example: Solr as Sink Twitter ./spark-submit --master MASTER --class com.lucidworks.spark.SparkApp spark-solr-1.0.jar twitter-to-solr -zkHost localhost:2181 –collection social Solr JavaReceiverInputDStream<Status> tweets = TwitterUtils.createStream(jssc, null, filters); Various transformations / enrichments on each tweet (e.g. sentiment analysis, language detection) JavaDStream<SolrInputDocument> docs = tweets.map( new Function<Status,SolrInputDocument>() { // Convert a twitter4j Status object into a SolrInputDocument public SolrInputDocument call(Status status) { SolrInputDocument doc = new SolrInputDocument(); … return doc; map() class TwitterToSolrStreamProcessor extends SparkApp.StreamProcessor SolrSupport.indexDStreamOfDocs(zkHost, collection, 100, docs); Slide Legend Provided by Spark Custom Java / Scala code Provided by Lucidworks
  • 29. Document Matching using Stored Queries • For each document, determine which of a large set of stored queries matches. • Useful for alerts, alternative flow paths through a stream, etc • Index a micro-batch into an embedded (in-memory) Solr instance and then determine which queries match • Matching framework; you have to decide where to load the stored queries from and what to do when matches are found • Scale it using Spark … need to scale to many queries, checkout Luwak
  • 30. Document Matching using Stored Queries Stored Queries DocFilterContext Twitter map() Slide Legend Provided by Spark Custom Java / Scala code Provided by Lucidworks JavaReceiverInputDStream<Status> tweets = TwitterUtils.createStream(jssc, null, filters); JavaDStream<SolrInputDocument> docs = tweets.map( new Function<Status,SolrInputDocument>() { // Convert a twitter4j Status object into a SolrInputDocument public SolrInputDocument call(Status status) { SolrInputDocument doc = new SolrInputDocument(); … return doc; }}); JavaDStream<SolrInputDocument> enriched =
 SolrSupport.filterDocuments(docFilterContext, …); Get queries Index docs into an EmbeddedSolrServer Initialized from configs stored in ZooKeeper … ZooKeeper Key abstraction to allow you to plug-in how to store the queries and what action to take when docs match
  • 31. RDD Illustrated: Word count map(word => (word, 1)) Map words into pairs with count of 1 (quick,1) (brown,1) (fox,1) (quick,1) (quick,1) val file = spark.textFile("hdfs://...") HDFS file RDD from HDFS quick brown fox jumped … quick brownie recipe … quick drying glue … ……… file.flatMap(line => line.split(" ")) Split lines into words quick brown fox quick quick …… reduceByKey(_ + _) Send all keys to same reducer and sum (quick,1) (quick,1) (quick,1) (quick,3) Shuffle across machine boundaries Executors assigned based on data-locality if possible, narrow transformations occur in same executor Spark keeps track of the transformations made to generate each RDD Partition 1 Partition 2 Partition 3 x val file = spark.textFile("hdfs://...") val counts = file.flatMap(line => line.split(" ")) .map(word => (word, 1)) .reduceByKey(_ + _) counts.saveAsTextFile("hdfs://...")
  • 32. Understanding Resilient Distributed Datasets (RDD) • Read-only partitioned collection of records with fault-tolerance • Created from external system OR using a transformation of another RDD • RDDs track the lineage of coarse-grained transformations (map, join, filter, etc) • If a partition is lost, RDDs can be re-computed by re-playing the transformations • User can choose to persist an RDD (for reusing during interactive data-mining) • User can control partitioning scheme
  • 33. Physical Architecture Spark Master (daemon) Spark Slave (daemon) my-spark-job.jar (w/ shaded deps) My Spark App SparkContext (driver) • Keeps track of live workers • Web UI on port 8080 • Task Scheduler • Restart failed tasks Spark Executor (JVM process) Tasks Executor runs in separate process than slave daemon Spark Worker Node (1...N of these) Each task works on some partition of a data set to apply a transformation or action Cache Losing a master prevents new applications from being executed Can achieve HA using ZooKeeper and multiple master nodes Tasks are assigned based on data-locality When selecting which node to execute a task on, the master takes into account data locality • RDD Graph • DAG Scheduler • Block tracker • Shuffle tracker
  • 34. Spark vs. Hadoop’s Map/Reduce Operators File System Fault-Tolerance Ecosystem Hadoop Map-Reduce HDFS S3 Replicated blocks Java API Pig Hive Spark filter, map, flatMap, join, groupByKey, reduce, sortByKey, count, distinct, top, cogroup, etc … HDFS S3 Tachyon Immutable RDD lineage Python / Scala / Java API SparkSQL GraphX MLlib SparkR