SlideShare a Scribd company logo
1 of 22
• Spark Overview / High-level Architecture
• Indexing from Spark
• Reading data from Solr
+ term vectors & Spark SQL
• Document Matching
• Q&A
Solr & Spark
• Solr user since 2010, committer since April 2014, work for
Lucidworks
• Focus mainly on SolrCloud features … and bin/solr!
• Release manager for Lucene / Solr 5.1
• Co-author of Solr in Action
• Several years experience working with Hadoop, Pig, Hive,
ZooKeeper, but only started using Spark about 6 months ago
…
• Other contributions include Solr on YARN, Solr Scale Toolkit,
and Spark-Solr integration project on github
About Me …
Spark Overview
• 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
• Write code in Java, Scala, or Python … REPL interface too
• Runs on YARN (or Mesos), plays well with HDFS
Spark Components
Spark Core
Spark
SQL
Spark
Streaming
MLlib
(machine
learning)
GraphX
(BSP)
Hadoop YARN Mesos Standalone
HDFS
Execution
Model
The Shuffle Caching
UI / API
engine
cluster
mgmt
Can combine all of these together in the same app!
Physical Architecture
Spark Master (daemon)
Spark Slave (daemon)
spark-solr-1.0.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
val file = spark.textFile("hdfs://...")
val counts = file.flatMap(line => line.split(" "))
.map(word => (word, 1))
.reduceByKey(_ + _)
counts.saveAsTextFile("hdfs://...")
RDD Illustrated: Word count
file RDD from HDFS
val file =
spark.textFile("hdfs://...")
HDFS
file.flatMap(line => line.split(" "))
Split lines into words Map words into
pairs with count of 1
map(word => (word, 1))
quick brown fox jumped … quick
brown
fox
(quick,1)
(brown,1)
(fox,1)
reduceByKey(_ + _)
quick brownie recipe … quick (quick,1)
Send all keys to same
reducer and sum
(quick,1)
(quick,1)
quick drying glue … quick (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
Understanding Resilient Distributed Datasets (RDD)
• Read-only partitioned collection of records with smart(er) 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; default is based on the input source
Spark & Solr Integration
• https://github.com/LucidWorks/spark-solr/
• Streaming applications
 Real-time, streaming ETL jobs
 Solr as sink for Spark job
 Real-time document matching against stored queries
• Distributed computations (interactive data mining, machine learning)
 Expose results from Solr query as Spark RDD (resilient distributed dataset)
 Optionally process results from each shard in parallel
 Read millions of rows efficiently using deep paging
 SparkSQL DataFrame support (uses Solr schema API) and Term Vectors too!
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
 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
Spark Streaming Example: Solr as Sink
// start receiving a stream of tweets ...
JavaReceiverInputDStream<Status> tweets =
TwitterUtils.createStream(jssc, null, filters);
// map incoming tweets into SolrInputDocument objects for indexing in Solr
JavaDStream<SolrInputDocument> docs = tweets.map(
new Function<Status,SolrInputDocument>() {
public SolrInputDocument call(Status status) {
SolrInputDocument doc =
SolrSupport.autoMapToSolrInputDoc("tweet-"+status.getId(), status, null);
doc.setField("provider_s", "twitter");
doc.setField("author_s", status.getUser().getScreenName());
doc.setField("type_s", status.isRetweet() ? "echo" : "post");
return doc;
}
}
);
// when ready, send the docs into a SolrCloud cluster
SolrSupport.indexDStreamOfDocs(zkHost, collection, docs);
com.lucidworks.spark.SolrSupport
public static void indexDStreamOfDocs(final String zkHost, final String collection, final int batchSize,
JavaDStream<SolrInputDocument> docs)
{
docs.foreachRDD(
new Function<JavaRDD<SolrInputDocument>, Void>() {
public Void call(JavaRDD<SolrInputDocument> solrInputDocumentJavaRDD) throws Exception {
solrInputDocumentJavaRDD.foreachPartition(
new VoidFunction<Iterator<SolrInputDocument>>() {
public void call(Iterator<SolrInputDocument> solrInputDocumentIterator) throws Exception {
final SolrServer solrServer = getSolrServer(zkHost);
List<SolrInputDocument> batch = new ArrayList<SolrInputDocument>();
while (solrInputDocumentIterator.hasNext()) {
batch.add(solrInputDocumentIterator.next());
if (batch.size() >= batchSize)
sendBatchToSolr(solrServer, collection, batch);
}
if (!batch.isEmpty())
sendBatchToSolr(solrServer, collection, batch);
}
}
);
return null;
}
}
);
}
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
com.lucidworks.spark.ShardPartitioner
• Custom partitioning scheme for RDD using Solr’s DocRouter
• Stream docs directly to each shard leader using metadata from ZooKeeper, do
cument shard assignment, and ConcurrentUpdateSolrClient
final ShardPartitioner shardPartitioner = new ShardPartitioner(zkHost, collection);
pairs.partitionBy(shardPartitioner).foreachPartition(
new VoidFunction<Iterator<Tuple2<String, SolrInputDocument>>>() {
public void call(Iterator<Tuple2<String, SolrInputDocument>> tupleIter) throws Exception {
ConcurrentUpdateSolrClient cuss = null;
while (tupleIter.hasNext()) {
// ... Initialize ConcurrentUpdateSolrClient once per partition
cuss.add(doc);
}
}
});
SolrRDD: Reading data from Solr into Spark
• Can execute any query and expose as an RDD
• SolrRDD produces JavaRDD<SolrDocument>
• Use deep-paging if needed (cursorMark)
• For reading full result sets where global sort order doesn’t matter,
parallelize query execution by distributing requests across the Spark cluster
JavaRDD<SolrDocument> results =
solrRDD.queryShards(jsc, solrQuery);
Reading Term Vectors from Solr
• Pull TF/IDF (or just TF) for each term in a field for each document in query
results from Solr
• Can be used to construct RDD<Vector> which can then be passed to MLLib:
SolrRDD solrRDD = new SolrRDD(zkHost, collection);
JavaRDD<Vector> vectors =
solrRDD.queryTermVectors(jsc, solrQuery, field, numFeatures);
vectors.cache();
KMeansModel clusters =
KMeans.train(vectors.rdd(), numClusters, numIterations);
// Evaluate clustering by computing Within Set Sum of Squared Errors
double WSSSE = clusters.computeCost(vectors.rdd());
Spark SQL
• Query Solr, then expose results as a SQL table
SolrQuery solrQuery = new SolrQuery(...);
solrQuery.setFields("text_t","type_s");
SolrRDD solrRDD = new SolrRDD(zkHost, collection);
JavaRDD<SolrDocument> solrJavaRDD = solrRDD.queryShards(jsc, solrQuery);
SQLContext sqlContext = new SQLContext(jsc);
DataFrame df =
solrRDD.applySchema(sqlContext, solrQuery, solrJavaRDD, zkHost, collection);
df.registerTempTable("tweets");
JavaSchemaRDD results =
sqlContext.sql("SELECT COUNT(type_s) FROM tweets WHERE type_s='echo'");
List<Long> count = results.javaRDD().map(new Function<Row, Long>() {
public Long call(Row row) {
return row.getLong(0);
}
}).collect();
System.out.println("# of echos : "+count.get(0));
Wrap-up and Q & A
• Reference implementation of Solr and Spark on YARN
• Formal benchmarks for reads and writes to Solr
• Checkout SOLR-6816 – improving replication performance
• Add Spark support to Solr Scale Toolkit
• Integrate metrics to give visibility into performance
• More use cases …
Feel free to reach out to me with questions:
tim.potter@lucidworks.com / @thelabdude
ApacheCon NA 2015 Spark / Solr Integration

More Related Content

What's hot

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
 
Data Science with Solr and Spark
Data Science with Solr and SparkData Science with Solr and Spark
Data Science with Solr and SparkLucidworks
 
Faster Data Analytics with Apache Spark using Apache Solr - Kiran Chitturi, L...
Faster Data Analytics with Apache Spark using Apache Solr - Kiran Chitturi, L...Faster Data Analytics with Apache Spark using Apache Solr - Kiran Chitturi, L...
Faster Data Analytics with Apache Spark using Apache Solr - Kiran Chitturi, L...Lucidworks
 
"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
 
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 TechnologyLucidworks
 
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 6Lucidworks
 
Sparkcamp @ Strata CA: Intro to Apache Spark with Hands-on Tutorials
Sparkcamp @ Strata CA: Intro to Apache Spark with Hands-on TutorialsSparkcamp @ Strata CA: Intro to Apache Spark with Hands-on Tutorials
Sparkcamp @ Strata CA: Intro to Apache Spark with Hands-on TutorialsDatabricks
 
Benchmarking Solr Performance at Scale
Benchmarking Solr Performance at ScaleBenchmarking Solr Performance at Scale
Benchmarking Solr Performance at Scalethelabdude
 
LuceneRDD for (Geospatial) Search and Entity Linkage
LuceneRDD for (Geospatial) Search and Entity LinkageLuceneRDD for (Geospatial) Search and Entity Linkage
LuceneRDD for (Geospatial) Search and Entity Linkagezouzias
 
The Pushdown of Everything by Stephan Kessler and Santiago Mola
The Pushdown of Everything by Stephan Kessler and Santiago MolaThe Pushdown of Everything by Stephan Kessler and Santiago Mola
The Pushdown of Everything by Stephan Kessler and Santiago MolaSpark Summit
 
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
 
Ingesting and Manipulating Data with JavaScript
Ingesting and Manipulating Data with JavaScriptIngesting and Manipulating Data with JavaScript
Ingesting and Manipulating Data with JavaScriptLucidworks
 
Spark Programming
Spark ProgrammingSpark Programming
Spark ProgrammingTaewook Eom
 
Spark DataFrames: Simple and Fast Analytics on Structured Data at Spark Summi...
Spark DataFrames: Simple and Fast Analytics on Structured Data at Spark Summi...Spark DataFrames: Simple and Fast Analytics on Structured Data at Spark Summi...
Spark DataFrames: Simple and Fast Analytics on Structured Data at Spark Summi...Databricks
 
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
 
Understanding Lucene Search Performance
Understanding Lucene Search PerformanceUnderstanding Lucene Search Performance
Understanding Lucene Search PerformanceLucidworks (Archived)
 
Building a Large Scale SEO/SEM Application with Apache Solr: Presented by Rah...
Building a Large Scale SEO/SEM Application with Apache Solr: Presented by Rah...Building a Large Scale SEO/SEM Application with Apache Solr: Presented by Rah...
Building a Large Scale SEO/SEM Application with Apache Solr: Presented by Rah...Lucidworks
 
Apache Spark RDDs
Apache Spark RDDsApache Spark RDDs
Apache Spark RDDsDean Chen
 

What's hot (20)

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...
 
Data Science with Solr and Spark
Data Science with Solr and SparkData Science with Solr and Spark
Data Science with Solr and Spark
 
Faster Data Analytics with Apache Spark using Apache Solr - Kiran Chitturi, L...
Faster Data Analytics with Apache Spark using Apache Solr - Kiran Chitturi, L...Faster Data Analytics with Apache Spark using Apache Solr - Kiran Chitturi, L...
Faster Data Analytics with Apache Spark using Apache Solr - Kiran Chitturi, L...
 
"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...
 
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
 
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
 
Sparkcamp @ Strata CA: Intro to Apache Spark with Hands-on Tutorials
Sparkcamp @ Strata CA: Intro to Apache Spark with Hands-on TutorialsSparkcamp @ Strata CA: Intro to Apache Spark with Hands-on Tutorials
Sparkcamp @ Strata CA: Intro to Apache Spark with Hands-on Tutorials
 
Benchmarking Solr Performance at Scale
Benchmarking Solr Performance at ScaleBenchmarking Solr Performance at Scale
Benchmarking Solr Performance at Scale
 
LuceneRDD for (Geospatial) Search and Entity Linkage
LuceneRDD for (Geospatial) Search and Entity LinkageLuceneRDD for (Geospatial) Search and Entity Linkage
LuceneRDD for (Geospatial) Search and Entity Linkage
 
The Pushdown of Everything by Stephan Kessler and Santiago Mola
The Pushdown of Everything by Stephan Kessler and Santiago MolaThe Pushdown of Everything by Stephan Kessler and Santiago Mola
The Pushdown of Everything by Stephan Kessler and Santiago Mola
 
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
 
Ingesting and Manipulating Data with JavaScript
Ingesting and Manipulating Data with JavaScriptIngesting and Manipulating Data with JavaScript
Ingesting and Manipulating Data with JavaScript
 
Spark Programming
Spark ProgrammingSpark Programming
Spark Programming
 
Spark DataFrames: Simple and Fast Analytics on Structured Data at Spark Summi...
Spark DataFrames: Simple and Fast Analytics on Structured Data at Spark Summi...Spark DataFrames: Simple and Fast Analytics on Structured Data at Spark Summi...
Spark DataFrames: Simple and Fast Analytics on Structured Data at Spark Summi...
 
High Performance Solr
High Performance SolrHigh Performance Solr
High Performance Solr
 
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 - ...
 
Understanding Lucene Search Performance
Understanding Lucene Search PerformanceUnderstanding Lucene Search Performance
Understanding Lucene Search Performance
 
Building a Large Scale SEO/SEM Application with Apache Solr: Presented by Rah...
Building a Large Scale SEO/SEM Application with Apache Solr: Presented by Rah...Building a Large Scale SEO/SEM Application with Apache Solr: Presented by Rah...
Building a Large Scale SEO/SEM Application with Apache Solr: Presented by Rah...
 
Introduction to Apache Spark
Introduction to Apache SparkIntroduction to Apache Spark
Introduction to Apache Spark
 
Apache Spark RDDs
Apache Spark RDDsApache Spark RDDs
Apache Spark RDDs
 

Viewers also liked

Solr As A SparkSQL DataSource
Solr As A SparkSQL DataSourceSolr As A SparkSQL DataSource
Solr As A SparkSQL DataSourceSpark Summit
 
Boosting Documents in Solr (Lucene Revolution 2011)
Boosting Documents in Solr (Lucene Revolution 2011)Boosting Documents in Solr (Lucene Revolution 2011)
Boosting Documents in Solr (Lucene Revolution 2011)thelabdude
 
Adding Search to the Hadoop Ecosystem
Adding Search to the Hadoop EcosystemAdding Search to the Hadoop Ecosystem
Adding Search to the Hadoop EcosystemCloudera, Inc.
 
個人的にAmazon EMR5.0.0でSpark 2.0を使ってZeppelinでSQL集計してみる
個人的にAmazon EMR5.0.0でSpark 2.0を使ってZeppelinでSQL集計してみる個人的にAmazon EMR5.0.0でSpark 2.0を使ってZeppelinでSQL集計してみる
個人的にAmazon EMR5.0.0でSpark 2.0を使ってZeppelinでSQL集計してみるEiji Shinohara
 
Deploying and managing SolrCloud in the cloud using the Solr Scale Toolkit
Deploying and managing SolrCloud in the cloud using the Solr Scale ToolkitDeploying and managing SolrCloud in the cloud using the Solr Scale Toolkit
Deploying and managing SolrCloud in the cloud using the Solr Scale Toolkitthelabdude
 
Solr Exchange: Introduction to SolrCloud
Solr Exchange: Introduction to SolrCloudSolr Exchange: Introduction to SolrCloud
Solr Exchange: Introduction to SolrCloudthelabdude
 
Leveraging the Power of Solr with Spark
Leveraging the Power of Solr with SparkLeveraging the Power of Solr with Spark
Leveraging the Power of Solr with SparkQAware GmbH
 
Apache spark - History and market overview
Apache spark - History and market overviewApache spark - History and market overview
Apache spark - History and market overviewMartin Zapletal
 
Red hat enterprise_virtualization_load
Red hat enterprise_virtualization_loadRed hat enterprise_virtualization_load
Red hat enterprise_virtualization_loadsilviucojocaru
 
Big data insights with Red Hat JBoss Data Virtualization
Big data insights with Red Hat JBoss Data VirtualizationBig data insights with Red Hat JBoss Data Virtualization
Big data insights with Red Hat JBoss Data VirtualizationKenneth Peeples
 
Big Data and Data Virtualization
Big Data and Data VirtualizationBig Data and Data Virtualization
Big Data and Data VirtualizationKenneth Peeples
 
Red Hat JBOSS Data Virtualization
Red Hat JBOSS Data VirtualizationRed Hat JBOSS Data Virtualization
Red Hat JBOSS Data VirtualizationDLT Solutions
 
Prottとsketchとzeplinのススメ
ProttとsketchとzeplinのススメProttとsketchとzeplinのススメ
ProttとsketchとzeplinのススメAsami Yamamoto
 

Viewers also liked (14)

Solr As A SparkSQL DataSource
Solr As A SparkSQL DataSourceSolr As A SparkSQL DataSource
Solr As A SparkSQL DataSource
 
Boosting Documents in Solr (Lucene Revolution 2011)
Boosting Documents in Solr (Lucene Revolution 2011)Boosting Documents in Solr (Lucene Revolution 2011)
Boosting Documents in Solr (Lucene Revolution 2011)
 
Adding Search to the Hadoop Ecosystem
Adding Search to the Hadoop EcosystemAdding Search to the Hadoop Ecosystem
Adding Search to the Hadoop Ecosystem
 
個人的にAmazon EMR5.0.0でSpark 2.0を使ってZeppelinでSQL集計してみる
個人的にAmazon EMR5.0.0でSpark 2.0を使ってZeppelinでSQL集計してみる個人的にAmazon EMR5.0.0でSpark 2.0を使ってZeppelinでSQL集計してみる
個人的にAmazon EMR5.0.0でSpark 2.0を使ってZeppelinでSQL集計してみる
 
Deploying and managing SolrCloud in the cloud using the Solr Scale Toolkit
Deploying and managing SolrCloud in the cloud using the Solr Scale ToolkitDeploying and managing SolrCloud in the cloud using the Solr Scale Toolkit
Deploying and managing SolrCloud in the cloud using the Solr Scale Toolkit
 
Solr Exchange: Introduction to SolrCloud
Solr Exchange: Introduction to SolrCloudSolr Exchange: Introduction to SolrCloud
Solr Exchange: Introduction to SolrCloud
 
Leveraging the Power of Solr with Spark
Leveraging the Power of Solr with SparkLeveraging the Power of Solr with Spark
Leveraging the Power of Solr with Spark
 
Apache spark - History and market overview
Apache spark - History and market overviewApache spark - History and market overview
Apache spark - History and market overview
 
Red hat enterprise_virtualization_load
Red hat enterprise_virtualization_loadRed hat enterprise_virtualization_load
Red hat enterprise_virtualization_load
 
Big data insights with Red Hat JBoss Data Virtualization
Big data insights with Red Hat JBoss Data VirtualizationBig data insights with Red Hat JBoss Data Virtualization
Big data insights with Red Hat JBoss Data Virtualization
 
Big Data and Data Virtualization
Big Data and Data VirtualizationBig Data and Data Virtualization
Big Data and Data Virtualization
 
Red Hat JBOSS Data Virtualization
Red Hat JBOSS Data VirtualizationRed Hat JBOSS Data Virtualization
Red Hat JBOSS Data Virtualization
 
Prottとsketchとzeplinのススメ
ProttとsketchとzeplinのススメProttとsketchとzeplinのススメ
Prottとsketchとzeplinのススメ
 
Yahoo compares Storm and Spark
Yahoo compares Storm and SparkYahoo compares Storm and Spark
Yahoo compares Storm and Spark
 

Similar to ApacheCon NA 2015 Spark / Solr Integration

TriHUG talk on Spark and Shark
TriHUG talk on Spark and SharkTriHUG talk on Spark and Shark
TriHUG talk on Spark and Sharktrihug
 
Dive into spark2
Dive into spark2Dive into spark2
Dive into spark2Gal Marder
 
Introduction to Apache Spark :: Lagos Scala Meetup session 2
Introduction to Apache Spark :: Lagos Scala Meetup session 2 Introduction to Apache Spark :: Lagos Scala Meetup session 2
Introduction to Apache Spark :: Lagos Scala Meetup session 2 Olalekan Fuad Elesin
 
Introduction to Apache Spark
Introduction to Apache SparkIntroduction to Apache Spark
Introduction to Apache SparkRahul Jain
 
Real time Analytics with Apache Kafka and Apache Spark
Real time Analytics with Apache Kafka and Apache SparkReal time Analytics with Apache Kafka and Apache Spark
Real time Analytics with Apache Kafka and Apache SparkRahul Jain
 
An Introduction to Spark
An Introduction to SparkAn Introduction to Spark
An Introduction to Sparkjlacefie
 
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 Meetupjlacefie
 
Intro to Apache Spark by CTO of Twingo
Intro to Apache Spark by CTO of TwingoIntro to Apache Spark by CTO of Twingo
Intro to Apache Spark by CTO of TwingoMapR Technologies
 
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
 
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 DatabricksDatabricks
 
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 SparkMartin Toshev
 
Apache Spark™ is a multi-language engine for executing data-S5.ppt
Apache Spark™ is a multi-language engine for executing data-S5.pptApache Spark™ is a multi-language engine for executing data-S5.ppt
Apache Spark™ is a multi-language engine for executing data-S5.pptbhargavi804095
 
Apache Spark Overview @ ferret
Apache Spark Overview @ ferretApache Spark Overview @ ferret
Apache Spark Overview @ ferretAndrii Gakhov
 
Learning spark ch09 - Spark SQL
Learning spark ch09 - Spark SQLLearning spark ch09 - Spark SQL
Learning spark ch09 - Spark SQLphanleson
 
Spark Saturday: Spark SQL & DataFrame Workshop with Apache Spark 2.3
Spark Saturday: Spark SQL & DataFrame Workshop with Apache Spark 2.3Spark Saturday: Spark SQL & DataFrame Workshop with Apache Spark 2.3
Spark Saturday: Spark SQL & DataFrame Workshop with Apache Spark 2.3Databricks
 
Introduction to apache spark
Introduction to apache sparkIntroduction to apache spark
Introduction to apache sparkMuktadiur Rahman
 
Introduction to apache spark
Introduction to apache sparkIntroduction to apache spark
Introduction to apache sparkJUGBD
 

Similar to ApacheCon NA 2015 Spark / Solr Integration (20)

TriHUG talk on Spark and Shark
TriHUG talk on Spark and SharkTriHUG talk on Spark and Shark
TriHUG talk on Spark and Shark
 
Spark core
Spark coreSpark core
Spark core
 
Dive into spark2
Dive into spark2Dive into spark2
Dive into spark2
 
Meetup ml spark_ppt
Meetup ml spark_pptMeetup ml spark_ppt
Meetup ml spark_ppt
 
Introduction to Apache Spark :: Lagos Scala Meetup session 2
Introduction to Apache Spark :: Lagos Scala Meetup session 2 Introduction to Apache Spark :: Lagos Scala Meetup session 2
Introduction to Apache Spark :: Lagos Scala Meetup session 2
 
Introduction to Apache Spark
Introduction to Apache SparkIntroduction to Apache Spark
Introduction to Apache Spark
 
Real time Analytics with Apache Kafka and Apache Spark
Real time Analytics with Apache Kafka and Apache SparkReal time Analytics with Apache Kafka and Apache Spark
Real time Analytics with Apache Kafka and Apache Spark
 
An Introduction to Spark
An Introduction to SparkAn Introduction to Spark
An Introduction to Spark
 
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
 
Intro to Apache Spark by CTO of Twingo
Intro to Apache Spark by CTO of TwingoIntro to Apache Spark by CTO of Twingo
Intro to Apache Spark by CTO of Twingo
 
20170126 big data processing
20170126 big data processing20170126 big data processing
20170126 big data processing
 
Introduction to Spark (Intern Event Presentation)
Introduction to Spark (Intern Event Presentation)Introduction to Spark (Intern Event Presentation)
Introduction to Spark (Intern Event Presentation)
 
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
 
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
 
Apache Spark™ is a multi-language engine for executing data-S5.ppt
Apache Spark™ is a multi-language engine for executing data-S5.pptApache Spark™ is a multi-language engine for executing data-S5.ppt
Apache Spark™ is a multi-language engine for executing data-S5.ppt
 
Apache Spark Overview @ ferret
Apache Spark Overview @ ferretApache Spark Overview @ ferret
Apache Spark Overview @ ferret
 
Learning spark ch09 - Spark SQL
Learning spark ch09 - Spark SQLLearning spark ch09 - Spark SQL
Learning spark ch09 - Spark SQL
 
Spark Saturday: Spark SQL & DataFrame Workshop with Apache Spark 2.3
Spark Saturday: Spark SQL & DataFrame Workshop with Apache Spark 2.3Spark Saturday: Spark SQL & DataFrame Workshop with Apache Spark 2.3
Spark Saturday: Spark SQL & DataFrame Workshop with Apache Spark 2.3
 
Introduction to apache spark
Introduction to apache sparkIntroduction to apache spark
Introduction to apache spark
 
Introduction to apache spark
Introduction to apache sparkIntroduction to apache spark
Introduction to apache spark
 

Recently uploaded

VIP Model Call Girls Hinjewadi ( Pune ) Call ON 8005736733 Starting From 5K t...
VIP Model Call Girls Hinjewadi ( Pune ) Call ON 8005736733 Starting From 5K t...VIP Model Call Girls Hinjewadi ( Pune ) Call ON 8005736733 Starting From 5K t...
VIP Model Call Girls Hinjewadi ( Pune ) Call ON 8005736733 Starting From 5K t...SUHANI PANDEY
 
Best VIP Call Girls Noida Sector 22 Call Me: 8448380779
Best VIP Call Girls Noida Sector 22 Call Me: 8448380779Best VIP Call Girls Noida Sector 22 Call Me: 8448380779
Best VIP Call Girls Noida Sector 22 Call Me: 8448380779Delhi Call girls
 
Junnasandra Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore...
Junnasandra Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore...Junnasandra Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore...
Junnasandra Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore...amitlee9823
 
Log Analysis using OSSEC sasoasasasas.pptx
Log Analysis using OSSEC sasoasasasas.pptxLog Analysis using OSSEC sasoasasasas.pptx
Log Analysis using OSSEC sasoasasasas.pptxJohnnyPlasten
 
April 2024 - Crypto Market Report's Analysis
April 2024 - Crypto Market Report's AnalysisApril 2024 - Crypto Market Report's Analysis
April 2024 - Crypto Market Report's Analysismanisha194592
 
Call Girls Bannerghatta Road Just Call 👗 7737669865 👗 Top Class Call Girl Ser...
Call Girls Bannerghatta Road Just Call 👗 7737669865 👗 Top Class Call Girl Ser...Call Girls Bannerghatta Road Just Call 👗 7737669865 👗 Top Class Call Girl Ser...
Call Girls Bannerghatta Road Just Call 👗 7737669865 👗 Top Class Call Girl Ser...amitlee9823
 
Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...
Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...
Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...amitlee9823
 
BabyOno dropshipping via API with DroFx.pptx
BabyOno dropshipping via API with DroFx.pptxBabyOno dropshipping via API with DroFx.pptx
BabyOno dropshipping via API with DroFx.pptxolyaivanovalion
 
Edukaciniai dropshipping via API with DroFx
Edukaciniai dropshipping via API with DroFxEdukaciniai dropshipping via API with DroFx
Edukaciniai dropshipping via API with DroFxolyaivanovalion
 
Data-Analysis for Chicago Crime Data 2023
Data-Analysis for Chicago Crime Data  2023Data-Analysis for Chicago Crime Data  2023
Data-Analysis for Chicago Crime Data 2023ymrp368
 
Midocean dropshipping via API with DroFx
Midocean dropshipping via API with DroFxMidocean dropshipping via API with DroFx
Midocean dropshipping via API with DroFxolyaivanovalion
 
Schema on read is obsolete. Welcome metaprogramming..pdf
Schema on read is obsolete. Welcome metaprogramming..pdfSchema on read is obsolete. Welcome metaprogramming..pdf
Schema on read is obsolete. Welcome metaprogramming..pdfLars Albertsson
 
Determinants of health, dimensions of health, positive health and spectrum of...
Determinants of health, dimensions of health, positive health and spectrum of...Determinants of health, dimensions of health, positive health and spectrum of...
Determinants of health, dimensions of health, positive health and spectrum of...shambhavirathore45
 
FESE Capital Markets Fact Sheet 2024 Q1.pdf
FESE Capital Markets Fact Sheet 2024 Q1.pdfFESE Capital Markets Fact Sheet 2024 Q1.pdf
FESE Capital Markets Fact Sheet 2024 Q1.pdfMarinCaroMartnezBerg
 
Call me @ 9892124323 Cheap Rate Call Girls in Vashi with Real Photo 100% Secure
Call me @ 9892124323  Cheap Rate Call Girls in Vashi with Real Photo 100% SecureCall me @ 9892124323  Cheap Rate Call Girls in Vashi with Real Photo 100% Secure
Call me @ 9892124323 Cheap Rate Call Girls in Vashi with Real Photo 100% SecurePooja Nehwal
 
Vip Model Call Girls (Delhi) Karol Bagh 9711199171✔️Body to body massage wit...
Vip Model  Call Girls (Delhi) Karol Bagh 9711199171✔️Body to body massage wit...Vip Model  Call Girls (Delhi) Karol Bagh 9711199171✔️Body to body massage wit...
Vip Model Call Girls (Delhi) Karol Bagh 9711199171✔️Body to body massage wit...shivangimorya083
 
Carero dropshipping via API with DroFx.pptx
Carero dropshipping via API with DroFx.pptxCarero dropshipping via API with DroFx.pptx
Carero dropshipping via API with DroFx.pptxolyaivanovalion
 
100-Concepts-of-AI by Anupama Kate .pptx
100-Concepts-of-AI by Anupama Kate .pptx100-Concepts-of-AI by Anupama Kate .pptx
100-Concepts-of-AI by Anupama Kate .pptxAnupama Kate
 
Introduction-to-Machine-Learning (1).pptx
Introduction-to-Machine-Learning (1).pptxIntroduction-to-Machine-Learning (1).pptx
Introduction-to-Machine-Learning (1).pptxfirstjob4
 

Recently uploaded (20)

VIP Model Call Girls Hinjewadi ( Pune ) Call ON 8005736733 Starting From 5K t...
VIP Model Call Girls Hinjewadi ( Pune ) Call ON 8005736733 Starting From 5K t...VIP Model Call Girls Hinjewadi ( Pune ) Call ON 8005736733 Starting From 5K t...
VIP Model Call Girls Hinjewadi ( Pune ) Call ON 8005736733 Starting From 5K t...
 
Best VIP Call Girls Noida Sector 22 Call Me: 8448380779
Best VIP Call Girls Noida Sector 22 Call Me: 8448380779Best VIP Call Girls Noida Sector 22 Call Me: 8448380779
Best VIP Call Girls Noida Sector 22 Call Me: 8448380779
 
Junnasandra Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore...
Junnasandra Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore...Junnasandra Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore...
Junnasandra Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore...
 
Log Analysis using OSSEC sasoasasasas.pptx
Log Analysis using OSSEC sasoasasasas.pptxLog Analysis using OSSEC sasoasasasas.pptx
Log Analysis using OSSEC sasoasasasas.pptx
 
April 2024 - Crypto Market Report's Analysis
April 2024 - Crypto Market Report's AnalysisApril 2024 - Crypto Market Report's Analysis
April 2024 - Crypto Market Report's Analysis
 
Call Girls Bannerghatta Road Just Call 👗 7737669865 👗 Top Class Call Girl Ser...
Call Girls Bannerghatta Road Just Call 👗 7737669865 👗 Top Class Call Girl Ser...Call Girls Bannerghatta Road Just Call 👗 7737669865 👗 Top Class Call Girl Ser...
Call Girls Bannerghatta Road Just Call 👗 7737669865 👗 Top Class Call Girl Ser...
 
Abortion pills in Doha Qatar (+966572737505 ! Get Cytotec
Abortion pills in Doha Qatar (+966572737505 ! Get CytotecAbortion pills in Doha Qatar (+966572737505 ! Get Cytotec
Abortion pills in Doha Qatar (+966572737505 ! Get Cytotec
 
Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...
Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...
Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...
 
BabyOno dropshipping via API with DroFx.pptx
BabyOno dropshipping via API with DroFx.pptxBabyOno dropshipping via API with DroFx.pptx
BabyOno dropshipping via API with DroFx.pptx
 
Edukaciniai dropshipping via API with DroFx
Edukaciniai dropshipping via API with DroFxEdukaciniai dropshipping via API with DroFx
Edukaciniai dropshipping via API with DroFx
 
Data-Analysis for Chicago Crime Data 2023
Data-Analysis for Chicago Crime Data  2023Data-Analysis for Chicago Crime Data  2023
Data-Analysis for Chicago Crime Data 2023
 
Midocean dropshipping via API with DroFx
Midocean dropshipping via API with DroFxMidocean dropshipping via API with DroFx
Midocean dropshipping via API with DroFx
 
Schema on read is obsolete. Welcome metaprogramming..pdf
Schema on read is obsolete. Welcome metaprogramming..pdfSchema on read is obsolete. Welcome metaprogramming..pdf
Schema on read is obsolete. Welcome metaprogramming..pdf
 
Determinants of health, dimensions of health, positive health and spectrum of...
Determinants of health, dimensions of health, positive health and spectrum of...Determinants of health, dimensions of health, positive health and spectrum of...
Determinants of health, dimensions of health, positive health and spectrum of...
 
FESE Capital Markets Fact Sheet 2024 Q1.pdf
FESE Capital Markets Fact Sheet 2024 Q1.pdfFESE Capital Markets Fact Sheet 2024 Q1.pdf
FESE Capital Markets Fact Sheet 2024 Q1.pdf
 
Call me @ 9892124323 Cheap Rate Call Girls in Vashi with Real Photo 100% Secure
Call me @ 9892124323  Cheap Rate Call Girls in Vashi with Real Photo 100% SecureCall me @ 9892124323  Cheap Rate Call Girls in Vashi with Real Photo 100% Secure
Call me @ 9892124323 Cheap Rate Call Girls in Vashi with Real Photo 100% Secure
 
Vip Model Call Girls (Delhi) Karol Bagh 9711199171✔️Body to body massage wit...
Vip Model  Call Girls (Delhi) Karol Bagh 9711199171✔️Body to body massage wit...Vip Model  Call Girls (Delhi) Karol Bagh 9711199171✔️Body to body massage wit...
Vip Model Call Girls (Delhi) Karol Bagh 9711199171✔️Body to body massage wit...
 
Carero dropshipping via API with DroFx.pptx
Carero dropshipping via API with DroFx.pptxCarero dropshipping via API with DroFx.pptx
Carero dropshipping via API with DroFx.pptx
 
100-Concepts-of-AI by Anupama Kate .pptx
100-Concepts-of-AI by Anupama Kate .pptx100-Concepts-of-AI by Anupama Kate .pptx
100-Concepts-of-AI by Anupama Kate .pptx
 
Introduction-to-Machine-Learning (1).pptx
Introduction-to-Machine-Learning (1).pptxIntroduction-to-Machine-Learning (1).pptx
Introduction-to-Machine-Learning (1).pptx
 

ApacheCon NA 2015 Spark / Solr Integration

  • 1.
  • 2. • Spark Overview / High-level Architecture • Indexing from Spark • Reading data from Solr + term vectors & Spark SQL • Document Matching • Q&A Solr & Spark
  • 3. • Solr user since 2010, committer since April 2014, work for Lucidworks • Focus mainly on SolrCloud features … and bin/solr! • Release manager for Lucene / Solr 5.1 • Co-author of Solr in Action • Several years experience working with Hadoop, Pig, Hive, ZooKeeper, but only started using Spark about 6 months ago … • Other contributions include Solr on YARN, Solr Scale Toolkit, and Spark-Solr integration project on github About Me …
  • 4. Spark Overview • 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 • Write code in Java, Scala, or Python … REPL interface too • Runs on YARN (or Mesos), plays well with HDFS
  • 5. Spark Components Spark Core Spark SQL Spark Streaming MLlib (machine learning) GraphX (BSP) Hadoop YARN Mesos Standalone HDFS Execution Model The Shuffle Caching UI / API engine cluster mgmt Can combine all of these together in the same app!
  • 6. Physical Architecture Spark Master (daemon) Spark Slave (daemon) spark-solr-1.0.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
  • 7. Spark vs. Hadoop’s Map/Reduce val file = spark.textFile("hdfs://...") val counts = file.flatMap(line => line.split(" ")) .map(word => (word, 1)) .reduceByKey(_ + _) counts.saveAsTextFile("hdfs://...")
  • 8. RDD Illustrated: Word count file RDD from HDFS val file = spark.textFile("hdfs://...") HDFS file.flatMap(line => line.split(" ")) Split lines into words Map words into pairs with count of 1 map(word => (word, 1)) quick brown fox jumped … quick brown fox (quick,1) (brown,1) (fox,1) reduceByKey(_ + _) quick brownie recipe … quick (quick,1) Send all keys to same reducer and sum (quick,1) (quick,1) quick drying glue … quick (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
  • 9. Understanding Resilient Distributed Datasets (RDD) • Read-only partitioned collection of records with smart(er) 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; default is based on the input source
  • 10. Spark & Solr Integration • https://github.com/LucidWorks/spark-solr/ • Streaming applications  Real-time, streaming ETL jobs  Solr as sink for Spark job  Real-time document matching against stored queries • Distributed computations (interactive data mining, machine learning)  Expose results from Solr query as Spark RDD (resilient distributed dataset)  Optionally process results from each shard in parallel  Read millions of rows efficiently using deep paging  SparkSQL DataFrame support (uses Solr schema API) and Term Vectors too!
  • 11. 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  Haven’t found any unbiased, reproducible performance comparisons between Storm / Spark
  • 12. 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
  • 13. Spark Streaming Example: Solr as Sink // start receiving a stream of tweets ... JavaReceiverInputDStream<Status> tweets = TwitterUtils.createStream(jssc, null, filters); // map incoming tweets into SolrInputDocument objects for indexing in Solr JavaDStream<SolrInputDocument> docs = tweets.map( new Function<Status,SolrInputDocument>() { public SolrInputDocument call(Status status) { SolrInputDocument doc = SolrSupport.autoMapToSolrInputDoc("tweet-"+status.getId(), status, null); doc.setField("provider_s", "twitter"); doc.setField("author_s", status.getUser().getScreenName()); doc.setField("type_s", status.isRetweet() ? "echo" : "post"); return doc; } } ); // when ready, send the docs into a SolrCloud cluster SolrSupport.indexDStreamOfDocs(zkHost, collection, docs);
  • 14. com.lucidworks.spark.SolrSupport public static void indexDStreamOfDocs(final String zkHost, final String collection, final int batchSize, JavaDStream<SolrInputDocument> docs) { docs.foreachRDD( new Function<JavaRDD<SolrInputDocument>, Void>() { public Void call(JavaRDD<SolrInputDocument> solrInputDocumentJavaRDD) throws Exception { solrInputDocumentJavaRDD.foreachPartition( new VoidFunction<Iterator<SolrInputDocument>>() { public void call(Iterator<SolrInputDocument> solrInputDocumentIterator) throws Exception { final SolrServer solrServer = getSolrServer(zkHost); List<SolrInputDocument> batch = new ArrayList<SolrInputDocument>(); while (solrInputDocumentIterator.hasNext()) { batch.add(solrInputDocumentIterator.next()); if (batch.size() >= batchSize) sendBatchToSolr(solrServer, collection, batch); } if (!batch.isEmpty()) sendBatchToSolr(solrServer, collection, batch); } } ); return null; } } ); }
  • 15. 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
  • 16. 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
  • 17. com.lucidworks.spark.ShardPartitioner • Custom partitioning scheme for RDD using Solr’s DocRouter • Stream docs directly to each shard leader using metadata from ZooKeeper, do cument shard assignment, and ConcurrentUpdateSolrClient final ShardPartitioner shardPartitioner = new ShardPartitioner(zkHost, collection); pairs.partitionBy(shardPartitioner).foreachPartition( new VoidFunction<Iterator<Tuple2<String, SolrInputDocument>>>() { public void call(Iterator<Tuple2<String, SolrInputDocument>> tupleIter) throws Exception { ConcurrentUpdateSolrClient cuss = null; while (tupleIter.hasNext()) { // ... Initialize ConcurrentUpdateSolrClient once per partition cuss.add(doc); } } });
  • 18. SolrRDD: Reading data from Solr into Spark • Can execute any query and expose as an RDD • SolrRDD produces JavaRDD<SolrDocument> • Use deep-paging if needed (cursorMark) • For reading full result sets where global sort order doesn’t matter, parallelize query execution by distributing requests across the Spark cluster JavaRDD<SolrDocument> results = solrRDD.queryShards(jsc, solrQuery);
  • 19. Reading Term Vectors from Solr • Pull TF/IDF (or just TF) for each term in a field for each document in query results from Solr • Can be used to construct RDD<Vector> which can then be passed to MLLib: SolrRDD solrRDD = new SolrRDD(zkHost, collection); JavaRDD<Vector> vectors = solrRDD.queryTermVectors(jsc, solrQuery, field, numFeatures); vectors.cache(); KMeansModel clusters = KMeans.train(vectors.rdd(), numClusters, numIterations); // Evaluate clustering by computing Within Set Sum of Squared Errors double WSSSE = clusters.computeCost(vectors.rdd());
  • 20. Spark SQL • Query Solr, then expose results as a SQL table SolrQuery solrQuery = new SolrQuery(...); solrQuery.setFields("text_t","type_s"); SolrRDD solrRDD = new SolrRDD(zkHost, collection); JavaRDD<SolrDocument> solrJavaRDD = solrRDD.queryShards(jsc, solrQuery); SQLContext sqlContext = new SQLContext(jsc); DataFrame df = solrRDD.applySchema(sqlContext, solrQuery, solrJavaRDD, zkHost, collection); df.registerTempTable("tweets"); JavaSchemaRDD results = sqlContext.sql("SELECT COUNT(type_s) FROM tweets WHERE type_s='echo'"); List<Long> count = results.javaRDD().map(new Function<Row, Long>() { public Long call(Row row) { return row.getLong(0); } }).collect(); System.out.println("# of echos : "+count.get(0));
  • 21. Wrap-up and Q & A • Reference implementation of Solr and Spark on YARN • Formal benchmarks for reads and writes to Solr • Checkout SOLR-6816 – improving replication performance • Add Spark support to Solr Scale Toolkit • Integrate metrics to give visibility into performance • More use cases … Feel free to reach out to me with questions: tim.potter@lucidworks.com / @thelabdude

Editor's Notes

  1. Solr 5 – overview: http://www.slideshare.net/lucidworks/webinar-inside-apache-solr-5 Who is using Solr in production? Anyone currently evaluating Solr and other technologies for a search project? Anyone using Spark?
  2. Started out as a research project at UC Berkeley – platform for exploring new areas of research in distributed systems / Big Data Shorter paper: http://people.csail.mit.edu/matei/papers/2010/hotcloud_spark.pdf Spark running on Hadoop sorted 100TB in 23 minutes (3x faster than Yahoo’s previous record) http://www.datanami.com/2014/10/10/spark-smashes-mapreduce-big-data-benchmark/ Highly optimized shuffle code and new network transport sub-system Key abstraction – Resilient Distributed Dataset Other projects using / moving to Spark: Mahout - https://www.mapr.com/blog/mahout-spark-what%E2%80%99s-new-recommenders#.VI5CBWTF9kA Hive Pig
  3. Internals talk: https://www.youtube.com/watch?v=dmL0N3qfSc8 Spark has all the same basic concepts around optimizing the shuffle stage (custom partitioning, combiners, etc) Recently overhauled the shuffle and network transport subsystem to use Netty and zero-copy techniques
  4. Can have multiple master nodes deployed for HA (leader is elected using ZooKeeper) Akka and Netty under the covers Execution Model: Create a DAG of RDDs Create logical execution plan for the DAG Schedule and execute individual tasks across the cluster Spark organizes tasks into stages; boundaries between stages are when the data needs to be re-organized (such as doing a groupBy or reduce) Stages are super operations that happen locally A task is data + computation Tasks get scheduled based on data locality
  5. Great presentation by Spark founder: https://www.usenix.org/conference/nsdi12/technical-sessions/presentation/zaharia MapReduce suffers from having to write intermediate data to disk to be used by other jobs or iterations; no good way to share data across jobs / iterations Data locality is still important Spark chooses to share data across iterations / interactive queries – the hard part is fault-tolerance, which it achieves using an RDD Less boilerplate code One way to think about Spark is it is a more intelligent optimizer that’s very good at keeping data that is reused in memory reliance on persistent storage to provide fault tolerance and its one-pass computation model parallel programs look very much like sequential programs, which make them easier to develop and reason about
  6. Different color boxes indicate partitions of the same RDD Some text data in HDFS, partitioned by HDFS blocks Spark assigns tasks to process the blocks based on data locality Narrow transformations occur in the same executor (no shuffling across machines)
  7. Spark RDD: https://www.cs.berkeley.edu/~matei/papers/2012/nsdi_spark.pdf Parallel computations using a restricted set of high-level operators Applied to *ALL* elements of a dataset at once Log one operation that is applied to many elements coarse-grained updates that apply the same operation to many data items Lineage + partition == low overhead recovery Achieve fault-tolerance by exposing coarse-grained transformations (steps are logged, which can be re-played if needed). If a partition is lost, RDDs contain enough information to re-compute the data Parallel applications apply the same transformations to many data items Persist – says to keep the RDD in-memory (probably because we’re going to be reusing it) Lazy execution: Spark will generate a DAG of stages to compute the result of an action
  8. The two technologies combined together provide near real-time processing, ad hoc queries, batch processing / deep analytics, machine learning, and horizontal scaling Aims to be a framework to help reduce boilerplate and get you started quickly, but you still have to write some code!
  9. Basically, split a stream into very small discretized batches (1 second is typical) and then all the other Spark RDD goodies apply AMP Camp Tathagata Das Probably on-par with Storm Trident (micro-batching)
  10. A series of very small deterministic batch jobs http://www.slideshare.net/pacoid/tiny-batches-in-the-wine-shiny-new-bits-in-spark-streaming http://www.cs.duke.edu/~kmoses/cps516/dstream.html Don’t have to have a separate stack for streaming apps e.g. instead of having Storm for streaming and Spark for interactive data mining, you just have Spark Spark chops live stream up into small batches of N seconds (each batch being an RDD) DStream is batch of records to be processed DStream is processed in micro-batches (controlled when the job is configured) map() step converts Twitter4J Status objects into SolrInputDocuments OR we could just send JSON directly to a Fusion pipeline and then do the mapping in the pipeline.
  11. This slide is here to show some ugliness that our Solr framework hides from end-users SolrSupport – removes need to worry about Spark boilerplate for sending a stream of docs to Solr
  12. Spark RDD: https://www.cs.berkeley.edu/~matei/papers/2012/nsdi_spark.pdf Parallel computations using a restricted set of high-level operators Achieve fault-tolerance by exposing coarse-grained transformations (steps are logged, which can be re-played if needed). If a partition is lost, RDDs contain enough information to re-compute the data Parallel applications apply the same transformations to many data items When nodes fail, Spark can recover quickly by rebuilding only the lost RDD partitions
  13. Spark RDD: https://www.cs.berkeley.edu/~matei/papers/2012/nsdi_spark.pdf Parallel computations using a restricted set of high-level operators Achieve fault-tolerance by exposing coarse-grained transformations (steps are logged, which can be re-played if needed). If a partition is lost, RDDs contain enough information to re-compute the data Parallel applications apply the same transformations to many data items When nodes fail, Spark can recover quickly by rebuilding only the lost RDD partitions
  14. Need to fix SOLR-3382 to get better error reporting when streaming docs to Solr using CUSS
  15. You can also get a Spark vector by doing: Vector vector = SolrTermVector.newInstance(String docId, HashingTF hashingTF, String rawText) // uses the Lucene StandardAnalyzer
  16. Basic process is to query Solr, expose Results as a JavaSchemaRDD, register as a temp table, perform queries Use Solr’s SchemaAPI to get metadata about fields in the query