SlideShare ist ein Scribd-Unternehmen logo
1 von 28
Downloaden Sie, um offline zu lesen
GC Tuning
lucenerevolution.org
October 13-16 Ÿ Austin, TX
Solr & Spark
•  Spark Overview
•  Resilient Distributed Dataset (RDD)
•  Solr as a SparkSQL DataSource
•  Interacting with Solr from the Spark shell
•  Indexing from Spark Streaming
•  Lucidworks Fusion and Spark
•  Other goodies (document matching, term
vectors, etc.)
•  Solr user since 2010, committer since April 2014, work for
Lucidworks, PMC member ~ May 2015
•  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 … working with Spark for about 1 year
•  Other contributions include Solr on YARN, Solr Scale
Toolkit, and Solr/Storm integration project on github
About Me …
About Solr
•  Vibrant, thriving open source community
•  Solr 5.3 just released!
ü  Authentication and authorization framework; plugins for Kerberos and Ranger
ü  ~2x indexing performance w/ replication
http://lucidworks.com/blog/indexing-performance-solr-5-2-now-twice-fast/
ü  Field cardinality estimation using HyperLogLog
ü  Rule-based replica placement strategy
•  Deploy to YARN cluster using Slider
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: Memory-centric Distributed Computation
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
Tachyon
languages Scala Java Python R
shared
memory
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
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
Solr as a Spark SQL Data Source
•  DataFrame is a DSL for distributed data manipulation
•  Data source provides a DataFrame
•  Uniform way of working with data from multiple sources
•  Hive, JDBC, Solr, Cassandra, etc.
•  Seamless integration with other Spark technologies: SparkR, Python,
MLlib …
Map<String,	
  String>	
  options	
  =	
  new	
  HashMap<String,	
  String>();	
  
options.put("zkhost",	
  zkHost);	
  
options.put("collection”,	
  "tweets");	
  
	
  
DataFrame	
  df	
  =	
  sqlContext.read().format("solr").options(options).load();	
  
count	
  =	
  df.filter(df.col("type_s").equalTo(“echo")).count();	
  
Spark SQL
Query Solr, then expose results as a SQL table
Map<String,	
  String>	
  options	
  =	
  new	
  HashMap<String,	
  String>();	
  
options.put("zkhost",	
  zkHost);	
  
options.put("collection”,	
  "tweets");	
  
	
  
DataFrame	
  df	
  =	
  sqlContext.read().format("solr").options(options).load();	
  
	
  
df.registerTempTable("tweets");	
  
	
  
sqlContext.sql("SELECT	
  count(*)	
  FROM	
  tweets	
  WHERE	
  type_s='echo'");	
  
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)
•  Stream docs from Solr (vs. building lists on the server-side)
•  More parallelism using a range filter on a numeric field (_version_)
e.g. 10 shards x 10 splits per shard == 100 concurrent Spark tasks
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”	
  -­‐>	
  “socialdata”,	
  
	
  	
  	
  	
  	
  “zkHost”	
  -­‐>	
  “…”)	
  
SolrRDD: Reading data from Solr into Spark
Query Solr from the Spark Shell
Interactive data mining with the full power of Solr queries
ADD_JARS=$PROJECT_HOME/target/spark-­‐solr-­‐1.0-­‐SNAPSHOT.jar	
  bin/spark-­‐shell	
  
	
  
val	
  solrDF	
  =	
  sqlContext.load("solr",	
  Map(	
  
	
  	
  "zkHost"	
  -­‐>	
  "localhost:9983",	
  
	
  	
  "collection"	
  -­‐>	
  "gettingstarted")).filter("provider_s='twitter'")	
  
	
  
solrDF.registerTempTable("tweets")	
  
	
  
sqlContext.sql("SELECT	
  COUNT(type_s)	
  FROM	
  tweets	
  WHERE	
  type_s='echo'").show()	
  
	
  
tweets.filter("type_s='post'").groupBy("author_s").count().	
  
	
  orderBy(desc("count")).limit(10).show()	
  
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;	
  
	
  	
  	
  	
  	
  	
  	
  	
  }	
  
	
  	
  	
  	
  	
  	
  }	
  
	
  	
  	
  	
  );	
  
	
  	
  }	
  
com.lucidworks.spark.ShardPartitioner
•  Custom partitioning scheme for RDD using Solr’s DocRouter
•  Stream docs directly to each shard leader using metadata from ZooKeeper,
document 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);	
  
	
  	
  	
  	
  	
  	
  }	
  
	
  	
  	
  }	
  
});	
  
Lucidworks Fusion
Fusion & Spark
•  Users leave evidence of their needs & experience as they use your app
•  Fusion closes the feedback loop to improve results based on user “signals”
•  Scale out distributed aggregation jobs across Spark cluster
•  Recommendations at scale
Extra goodies: 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());	
  
	
  
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
Getting Started
git clone https://github.com/LucidWorks/spark-solr/
cd spark-solr
mvn clean package -DskipTests
See: https://lucidworks.com/blog/solr-spark-sql-datasource/
Wrap-up and Q & A
Download Fusion: http://lucidworks.com/fusion/download/
Feel free to reach out to me with questions:
tim.potter@lucidworks.com / @thelabdude

Weitere ähnliche Inhalte

Was ist angesagt?

Data Engineering with Solr and Spark
Data Engineering with Solr and SparkData Engineering with Solr and Spark
Data Engineering 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
 
Ingesting and Manipulating Data with JavaScript
Ingesting and Manipulating Data with JavaScriptIngesting and Manipulating Data with JavaScript
Ingesting and Manipulating Data with JavaScriptLucidworks
 
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
 
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
 
Spark Programming
Spark ProgrammingSpark Programming
Spark ProgrammingTaewook Eom
 
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
 
"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
 
Spark after Dark by Chris Fregly of Databricks
Spark after Dark by Chris Fregly of DatabricksSpark after Dark by Chris Fregly of Databricks
Spark after Dark by Chris Fregly of DatabricksData Con LA
 
Benchmarking Solr Performance at Scale
Benchmarking Solr Performance at ScaleBenchmarking Solr Performance at Scale
Benchmarking Solr Performance at Scalethelabdude
 
Rapid Prototyping with Solr
Rapid Prototyping with SolrRapid Prototyping with Solr
Rapid Prototyping with SolrErik Hatcher
 
Introduction to Spark Internals
Introduction to Spark InternalsIntroduction to Spark Internals
Introduction to Spark InternalsPietro Michiardi
 
Apache Solr crash course
Apache Solr crash courseApache Solr crash course
Apache Solr crash courseTommaso Teofili
 
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
 
Rapid Prototyping with Solr
Rapid Prototyping with SolrRapid Prototyping with Solr
Rapid Prototyping with SolrErik Hatcher
 
Lifecycle of a Solr Search Request - Chris "Hoss" Hostetter, Lucidworks
Lifecycle of a Solr Search Request - Chris "Hoss" Hostetter, LucidworksLifecycle of a Solr Search Request - Chris "Hoss" Hostetter, Lucidworks
Lifecycle of a Solr Search Request - Chris "Hoss" Hostetter, LucidworksLucidworks
 
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
 

Was ist angesagt? (20)

Data Engineering with Solr and Spark
Data Engineering with Solr and SparkData Engineering with Solr and Spark
Data Engineering 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...
 
Ingesting and Manipulating Data with JavaScript
Ingesting and Manipulating Data with JavaScriptIngesting and Manipulating Data with JavaScript
Ingesting and Manipulating Data with JavaScript
 
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...
 
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
 
Spark Programming
Spark ProgrammingSpark Programming
Spark Programming
 
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
 
"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
 
Spark after Dark by Chris Fregly of Databricks
Spark after Dark by Chris Fregly of DatabricksSpark after Dark by Chris Fregly of Databricks
Spark after Dark by Chris Fregly of Databricks
 
Benchmarking Solr Performance at Scale
Benchmarking Solr Performance at ScaleBenchmarking Solr Performance at Scale
Benchmarking Solr Performance at Scale
 
Rapid Prototyping with Solr
Rapid Prototyping with SolrRapid Prototyping with Solr
Rapid Prototyping with Solr
 
Introduction to Spark Internals
Introduction to Spark InternalsIntroduction to Spark Internals
Introduction to Spark Internals
 
Apache Solr crash course
Apache Solr crash courseApache Solr crash course
Apache Solr crash course
 
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...
 
Rapid Prototyping with Solr
Rapid Prototyping with SolrRapid Prototyping with Solr
Rapid Prototyping with Solr
 
Lifecycle of a Solr Search Request - Chris "Hoss" Hostetter, Lucidworks
Lifecycle of a Solr Search Request - Chris "Hoss" Hostetter, LucidworksLifecycle of a Solr Search Request - Chris "Hoss" Hostetter, Lucidworks
Lifecycle of a Solr Search Request - Chris "Hoss" Hostetter, Lucidworks
 
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
 
Introduction to Apache Spark
Introduction to Apache SparkIntroduction to Apache Spark
Introduction to Apache Spark
 

Andere mochten auch

Andere mochten auch (7)

Routing for an Anycast CDN
Routing for an Anycast CDNRouting for an Anycast CDN
Routing for an Anycast CDN
 
Good Peering Practices
Good Peering Practices Good Peering Practices
Good Peering Practices
 
flowspec @ APF 2013
flowspec @ APF 2013flowspec @ APF 2013
flowspec @ APF 2013
 
EVCache at Netflix
EVCache at NetflixEVCache at Netflix
EVCache at Netflix
 
GPGPU Accelerates PostgreSQL (English)
GPGPU Accelerates PostgreSQL (English)GPGPU Accelerates PostgreSQL (English)
GPGPU Accelerates PostgreSQL (English)
 
Intro to IndexedDB (Beta)
Intro to IndexedDB (Beta)Intro to IndexedDB (Beta)
Intro to IndexedDB (Beta)
 
Going real time with Socket.io
Going real time with Socket.ioGoing real time with Socket.io
Going real time with Socket.io
 

Ähnlich wie Webinar: Solr & Spark for Real Time Big Data Analytics

Dive into spark2
Dive into spark2Dive into spark2
Dive into spark2Gal Marder
 
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
 
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 SharkYahooTechConference
 
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
 
11. From Hadoop to Spark 2/2
11. From Hadoop to Spark 2/211. From Hadoop to Spark 2/2
11. From Hadoop to Spark 2/2Fabio Fumarola
 
Ingesting hdfs intosolrusingsparktrimmed
Ingesting hdfs intosolrusingsparktrimmedIngesting hdfs intosolrusingsparktrimmed
Ingesting hdfs intosolrusingsparktrimmedwhoschek
 
Solr As A SparkSQL DataSource
Solr As A SparkSQL DataSourceSolr As A SparkSQL DataSource
Solr As A SparkSQL DataSourceSpark Summit
 
Artigo 81 - spark_tutorial.pdf
Artigo 81 - spark_tutorial.pdfArtigo 81 - spark_tutorial.pdf
Artigo 81 - spark_tutorial.pdfWalmirCouto3
 
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 meetupRadu Chilom
 
Unified Big Data Processing with Apache Spark (QCON 2014)
Unified Big Data Processing with Apache Spark (QCON 2014)Unified Big Data Processing with Apache Spark (QCON 2014)
Unified Big Data Processing with Apache Spark (QCON 2014)Databricks
 
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
 
Apache Spark Workshop, Apr. 2016, Euangelos Linardos
Apache Spark Workshop, Apr. 2016, Euangelos LinardosApache Spark Workshop, Apr. 2016, Euangelos Linardos
Apache Spark Workshop, Apr. 2016, Euangelos LinardosEuangelos Linardos
 
Apache spark-melbourne-april-2015-meetup
Apache spark-melbourne-april-2015-meetupApache spark-melbourne-april-2015-meetup
Apache spark-melbourne-april-2015-meetupNed Shawa
 
Unified Big Data Processing with Apache Spark
Unified Big Data Processing with Apache SparkUnified Big Data Processing with Apache Spark
Unified Big Data Processing with Apache SparkC4Media
 
TriHUG talk on Spark and Shark
TriHUG talk on Spark and SharkTriHUG talk on Spark and Shark
TriHUG talk on Spark and Sharktrihug
 
Apache Spark Overview @ ferret
Apache Spark Overview @ ferretApache Spark Overview @ ferret
Apache Spark Overview @ ferretAndrii Gakhov
 

Ähnlich wie Webinar: Solr & Spark for Real Time Big Data Analytics (20)

Dive into spark2
Dive into spark2Dive into spark2
Dive into spark2
 
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
 
Spark core
Spark coreSpark core
Spark core
 
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
 
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
 
11. From Hadoop to Spark 2/2
11. From Hadoop to Spark 2/211. From Hadoop to Spark 2/2
11. From Hadoop to Spark 2/2
 
Scala and spark
Scala and sparkScala and spark
Scala and spark
 
Ingesting hdfs intosolrusingsparktrimmed
Ingesting hdfs intosolrusingsparktrimmedIngesting hdfs intosolrusingsparktrimmed
Ingesting hdfs intosolrusingsparktrimmed
 
Solr As A SparkSQL DataSource
Solr As A SparkSQL DataSourceSolr As A SparkSQL DataSource
Solr As A SparkSQL DataSource
 
Artigo 81 - spark_tutorial.pdf
Artigo 81 - spark_tutorial.pdfArtigo 81 - spark_tutorial.pdf
Artigo 81 - spark_tutorial.pdf
 
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
 
Unified Big Data Processing with Apache Spark (QCON 2014)
Unified Big Data Processing with Apache Spark (QCON 2014)Unified Big Data Processing with Apache Spark (QCON 2014)
Unified Big Data Processing with Apache Spark (QCON 2014)
 
Apache Spark
Apache SparkApache Spark
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
 
Apache Spark Workshop, Apr. 2016, Euangelos Linardos
Apache Spark Workshop, Apr. 2016, Euangelos LinardosApache Spark Workshop, Apr. 2016, Euangelos Linardos
Apache Spark Workshop, Apr. 2016, Euangelos Linardos
 
Apache spark-melbourne-april-2015-meetup
Apache spark-melbourne-april-2015-meetupApache spark-melbourne-april-2015-meetup
Apache spark-melbourne-april-2015-meetup
 
Unified Big Data Processing with Apache Spark
Unified Big Data Processing with Apache SparkUnified Big Data Processing with Apache Spark
Unified Big Data Processing with Apache Spark
 
TriHUG talk on Spark and Shark
TriHUG talk on Spark and SharkTriHUG talk on Spark and Shark
TriHUG talk on Spark and Shark
 
Apache Spark Overview @ ferret
Apache Spark Overview @ ferretApache Spark Overview @ ferret
Apache Spark Overview @ ferret
 

Mehr von Lucidworks

Search is the Tip of the Spear for Your B2B eCommerce Strategy
Search is the Tip of the Spear for Your B2B eCommerce StrategySearch is the Tip of the Spear for Your B2B eCommerce Strategy
Search is the Tip of the Spear for Your B2B eCommerce StrategyLucidworks
 
Drive Agent Effectiveness in Salesforce
Drive Agent Effectiveness in SalesforceDrive Agent Effectiveness in Salesforce
Drive Agent Effectiveness in SalesforceLucidworks
 
How Crate & Barrel Connects Shoppers with Relevant Products
How Crate & Barrel Connects Shoppers with Relevant ProductsHow Crate & Barrel Connects Shoppers with Relevant Products
How Crate & Barrel Connects Shoppers with Relevant ProductsLucidworks
 
Lucidworks & IMRG Webinar – Best-In-Class Retail Product Discovery
Lucidworks & IMRG Webinar – Best-In-Class Retail Product DiscoveryLucidworks & IMRG Webinar – Best-In-Class Retail Product Discovery
Lucidworks & IMRG Webinar – Best-In-Class Retail Product DiscoveryLucidworks
 
Connected Experiences Are Personalized Experiences
Connected Experiences Are Personalized ExperiencesConnected Experiences Are Personalized Experiences
Connected Experiences Are Personalized ExperiencesLucidworks
 
Intelligent Insight Driven Policing with MC+A, Toronto Police Service and Luc...
Intelligent Insight Driven Policing with MC+A, Toronto Police Service and Luc...Intelligent Insight Driven Policing with MC+A, Toronto Police Service and Luc...
Intelligent Insight Driven Policing with MC+A, Toronto Police Service and Luc...Lucidworks
 
[Webinar] Intelligent Policing. Leveraging Data to more effectively Serve Com...
[Webinar] Intelligent Policing. Leveraging Data to more effectively Serve Com...[Webinar] Intelligent Policing. Leveraging Data to more effectively Serve Com...
[Webinar] Intelligent Policing. Leveraging Data to more effectively Serve Com...Lucidworks
 
Preparing for Peak in Ecommerce | eTail Asia 2020
Preparing for Peak in Ecommerce | eTail Asia 2020Preparing for Peak in Ecommerce | eTail Asia 2020
Preparing for Peak in Ecommerce | eTail Asia 2020Lucidworks
 
Accelerate The Path To Purchase With Product Discovery at Retail Innovation C...
Accelerate The Path To Purchase With Product Discovery at Retail Innovation C...Accelerate The Path To Purchase With Product Discovery at Retail Innovation C...
Accelerate The Path To Purchase With Product Discovery at Retail Innovation C...Lucidworks
 
AI-Powered Linguistics and Search with Fusion and Rosette
AI-Powered Linguistics and Search with Fusion and RosetteAI-Powered Linguistics and Search with Fusion and Rosette
AI-Powered Linguistics and Search with Fusion and RosetteLucidworks
 
The Service Industry After COVID-19: The Soul of Service in a Virtual Moment
The Service Industry After COVID-19: The Soul of Service in a Virtual MomentThe Service Industry After COVID-19: The Soul of Service in a Virtual Moment
The Service Industry After COVID-19: The Soul of Service in a Virtual MomentLucidworks
 
Webinar: Smart answers for employee and customer support after covid 19 - Europe
Webinar: Smart answers for employee and customer support after covid 19 - EuropeWebinar: Smart answers for employee and customer support after covid 19 - Europe
Webinar: Smart answers for employee and customer support after covid 19 - EuropeLucidworks
 
Smart Answers for Employee and Customer Support After COVID-19
Smart Answers for Employee and Customer Support After COVID-19Smart Answers for Employee and Customer Support After COVID-19
Smart Answers for Employee and Customer Support After COVID-19Lucidworks
 
Applying AI & Search in Europe - featuring 451 Research
Applying AI & Search in Europe - featuring 451 ResearchApplying AI & Search in Europe - featuring 451 Research
Applying AI & Search in Europe - featuring 451 ResearchLucidworks
 
Webinar: Accelerate Data Science with Fusion 5.1
Webinar: Accelerate Data Science with Fusion 5.1Webinar: Accelerate Data Science with Fusion 5.1
Webinar: Accelerate Data Science with Fusion 5.1Lucidworks
 
Webinar: 5 Must-Have Items You Need for Your 2020 Ecommerce Strategy
Webinar: 5 Must-Have Items You Need for Your 2020 Ecommerce StrategyWebinar: 5 Must-Have Items You Need for Your 2020 Ecommerce Strategy
Webinar: 5 Must-Have Items You Need for Your 2020 Ecommerce StrategyLucidworks
 
Where Search Meets Science and Style Meets Savings: Nordstrom Rack's Journey ...
Where Search Meets Science and Style Meets Savings: Nordstrom Rack's Journey ...Where Search Meets Science and Style Meets Savings: Nordstrom Rack's Journey ...
Where Search Meets Science and Style Meets Savings: Nordstrom Rack's Journey ...Lucidworks
 
Apply Knowledge Graphs and Search for Real-World Decision Intelligence
Apply Knowledge Graphs and Search for Real-World Decision IntelligenceApply Knowledge Graphs and Search for Real-World Decision Intelligence
Apply Knowledge Graphs and Search for Real-World Decision IntelligenceLucidworks
 
Webinar: Building a Business Case for Enterprise Search
Webinar: Building a Business Case for Enterprise SearchWebinar: Building a Business Case for Enterprise Search
Webinar: Building a Business Case for Enterprise SearchLucidworks
 
Why Insight Engines Matter in 2020 and Beyond
Why Insight Engines Matter in 2020 and BeyondWhy Insight Engines Matter in 2020 and Beyond
Why Insight Engines Matter in 2020 and BeyondLucidworks
 

Mehr von Lucidworks (20)

Search is the Tip of the Spear for Your B2B eCommerce Strategy
Search is the Tip of the Spear for Your B2B eCommerce StrategySearch is the Tip of the Spear for Your B2B eCommerce Strategy
Search is the Tip of the Spear for Your B2B eCommerce Strategy
 
Drive Agent Effectiveness in Salesforce
Drive Agent Effectiveness in SalesforceDrive Agent Effectiveness in Salesforce
Drive Agent Effectiveness in Salesforce
 
How Crate & Barrel Connects Shoppers with Relevant Products
How Crate & Barrel Connects Shoppers with Relevant ProductsHow Crate & Barrel Connects Shoppers with Relevant Products
How Crate & Barrel Connects Shoppers with Relevant Products
 
Lucidworks & IMRG Webinar – Best-In-Class Retail Product Discovery
Lucidworks & IMRG Webinar – Best-In-Class Retail Product DiscoveryLucidworks & IMRG Webinar – Best-In-Class Retail Product Discovery
Lucidworks & IMRG Webinar – Best-In-Class Retail Product Discovery
 
Connected Experiences Are Personalized Experiences
Connected Experiences Are Personalized ExperiencesConnected Experiences Are Personalized Experiences
Connected Experiences Are Personalized Experiences
 
Intelligent Insight Driven Policing with MC+A, Toronto Police Service and Luc...
Intelligent Insight Driven Policing with MC+A, Toronto Police Service and Luc...Intelligent Insight Driven Policing with MC+A, Toronto Police Service and Luc...
Intelligent Insight Driven Policing with MC+A, Toronto Police Service and Luc...
 
[Webinar] Intelligent Policing. Leveraging Data to more effectively Serve Com...
[Webinar] Intelligent Policing. Leveraging Data to more effectively Serve Com...[Webinar] Intelligent Policing. Leveraging Data to more effectively Serve Com...
[Webinar] Intelligent Policing. Leveraging Data to more effectively Serve Com...
 
Preparing for Peak in Ecommerce | eTail Asia 2020
Preparing for Peak in Ecommerce | eTail Asia 2020Preparing for Peak in Ecommerce | eTail Asia 2020
Preparing for Peak in Ecommerce | eTail Asia 2020
 
Accelerate The Path To Purchase With Product Discovery at Retail Innovation C...
Accelerate The Path To Purchase With Product Discovery at Retail Innovation C...Accelerate The Path To Purchase With Product Discovery at Retail Innovation C...
Accelerate The Path To Purchase With Product Discovery at Retail Innovation C...
 
AI-Powered Linguistics and Search with Fusion and Rosette
AI-Powered Linguistics and Search with Fusion and RosetteAI-Powered Linguistics and Search with Fusion and Rosette
AI-Powered Linguistics and Search with Fusion and Rosette
 
The Service Industry After COVID-19: The Soul of Service in a Virtual Moment
The Service Industry After COVID-19: The Soul of Service in a Virtual MomentThe Service Industry After COVID-19: The Soul of Service in a Virtual Moment
The Service Industry After COVID-19: The Soul of Service in a Virtual Moment
 
Webinar: Smart answers for employee and customer support after covid 19 - Europe
Webinar: Smart answers for employee and customer support after covid 19 - EuropeWebinar: Smart answers for employee and customer support after covid 19 - Europe
Webinar: Smart answers for employee and customer support after covid 19 - Europe
 
Smart Answers for Employee and Customer Support After COVID-19
Smart Answers for Employee and Customer Support After COVID-19Smart Answers for Employee and Customer Support After COVID-19
Smart Answers for Employee and Customer Support After COVID-19
 
Applying AI & Search in Europe - featuring 451 Research
Applying AI & Search in Europe - featuring 451 ResearchApplying AI & Search in Europe - featuring 451 Research
Applying AI & Search in Europe - featuring 451 Research
 
Webinar: Accelerate Data Science with Fusion 5.1
Webinar: Accelerate Data Science with Fusion 5.1Webinar: Accelerate Data Science with Fusion 5.1
Webinar: Accelerate Data Science with Fusion 5.1
 
Webinar: 5 Must-Have Items You Need for Your 2020 Ecommerce Strategy
Webinar: 5 Must-Have Items You Need for Your 2020 Ecommerce StrategyWebinar: 5 Must-Have Items You Need for Your 2020 Ecommerce Strategy
Webinar: 5 Must-Have Items You Need for Your 2020 Ecommerce Strategy
 
Where Search Meets Science and Style Meets Savings: Nordstrom Rack's Journey ...
Where Search Meets Science and Style Meets Savings: Nordstrom Rack's Journey ...Where Search Meets Science and Style Meets Savings: Nordstrom Rack's Journey ...
Where Search Meets Science and Style Meets Savings: Nordstrom Rack's Journey ...
 
Apply Knowledge Graphs and Search for Real-World Decision Intelligence
Apply Knowledge Graphs and Search for Real-World Decision IntelligenceApply Knowledge Graphs and Search for Real-World Decision Intelligence
Apply Knowledge Graphs and Search for Real-World Decision Intelligence
 
Webinar: Building a Business Case for Enterprise Search
Webinar: Building a Business Case for Enterprise SearchWebinar: Building a Business Case for Enterprise Search
Webinar: Building a Business Case for Enterprise Search
 
Why Insight Engines Matter in 2020 and Beyond
Why Insight Engines Matter in 2020 and BeyondWhy Insight Engines Matter in 2020 and Beyond
Why Insight Engines Matter in 2020 and Beyond
 

Kürzlich hochgeladen

Unlocking the Future of AI Agents with Large Language Models
Unlocking the Future of AI Agents with Large Language ModelsUnlocking the Future of AI Agents with Large Language Models
Unlocking the Future of AI Agents with Large Language Modelsaagamshah0812
 
Hand gesture recognition PROJECT PPT.pptx
Hand gesture recognition PROJECT PPT.pptxHand gesture recognition PROJECT PPT.pptx
Hand gesture recognition PROJECT PPT.pptxbodapatigopi8531
 
why an Opensea Clone Script might be your perfect match.pdf
why an Opensea Clone Script might be your perfect match.pdfwhy an Opensea Clone Script might be your perfect match.pdf
why an Opensea Clone Script might be your perfect match.pdfjoe51371421
 
Optimizing AI for immediate response in Smart CCTV
Optimizing AI for immediate response in Smart CCTVOptimizing AI for immediate response in Smart CCTV
Optimizing AI for immediate response in Smart CCTVshikhaohhpro
 
Software Quality Assurance Interview Questions
Software Quality Assurance Interview QuestionsSoftware Quality Assurance Interview Questions
Software Quality Assurance Interview QuestionsArshad QA
 
HR Software Buyers Guide in 2024 - HRSoftware.com
HR Software Buyers Guide in 2024 - HRSoftware.comHR Software Buyers Guide in 2024 - HRSoftware.com
HR Software Buyers Guide in 2024 - HRSoftware.comFatema Valibhai
 
call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️
call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️
call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️Delhi Call girls
 
Test Automation Strategy for Frontend and Backend
Test Automation Strategy for Frontend and BackendTest Automation Strategy for Frontend and Backend
Test Automation Strategy for Frontend and BackendArshad QA
 
Steps To Getting Up And Running Quickly With MyTimeClock Employee Scheduling ...
Steps To Getting Up And Running Quickly With MyTimeClock Employee Scheduling ...Steps To Getting Up And Running Quickly With MyTimeClock Employee Scheduling ...
Steps To Getting Up And Running Quickly With MyTimeClock Employee Scheduling ...MyIntelliSource, Inc.
 
Reassessing the Bedrock of Clinical Function Models: An Examination of Large ...
Reassessing the Bedrock of Clinical Function Models: An Examination of Large ...Reassessing the Bedrock of Clinical Function Models: An Examination of Large ...
Reassessing the Bedrock of Clinical Function Models: An Examination of Large ...harshavardhanraghave
 
Unveiling the Tech Salsa of LAMs with Janus in Real-Time Applications
Unveiling the Tech Salsa of LAMs with Janus in Real-Time ApplicationsUnveiling the Tech Salsa of LAMs with Janus in Real-Time Applications
Unveiling the Tech Salsa of LAMs with Janus in Real-Time ApplicationsAlberto González Trastoy
 
Salesforce Certified Field Service Consultant
Salesforce Certified Field Service ConsultantSalesforce Certified Field Service Consultant
Salesforce Certified Field Service ConsultantAxelRicardoTrocheRiq
 
Shapes for Sharing between Graph Data Spaces - and Epistemic Querying of RDF-...
Shapes for Sharing between Graph Data Spaces - and Epistemic Querying of RDF-...Shapes for Sharing between Graph Data Spaces - and Epistemic Querying of RDF-...
Shapes for Sharing between Graph Data Spaces - and Epistemic Querying of RDF-...Steffen Staab
 
W01_panagenda_Navigating-the-Future-with-The-Hitchhikers-Guide-to-Notes-and-D...
W01_panagenda_Navigating-the-Future-with-The-Hitchhikers-Guide-to-Notes-and-D...W01_panagenda_Navigating-the-Future-with-The-Hitchhikers-Guide-to-Notes-and-D...
W01_panagenda_Navigating-the-Future-with-The-Hitchhikers-Guide-to-Notes-and-D...panagenda
 
SyndBuddy AI 2k Review 2024: Revolutionizing Content Syndication with AI
SyndBuddy AI 2k Review 2024: Revolutionizing Content Syndication with AISyndBuddy AI 2k Review 2024: Revolutionizing Content Syndication with AI
SyndBuddy AI 2k Review 2024: Revolutionizing Content Syndication with AIABDERRAOUF MEHENNI
 
Learn the Fundamentals of XCUITest Framework_ A Beginner's Guide.pdf
Learn the Fundamentals of XCUITest Framework_ A Beginner's Guide.pdfLearn the Fundamentals of XCUITest Framework_ A Beginner's Guide.pdf
Learn the Fundamentals of XCUITest Framework_ A Beginner's Guide.pdfkalichargn70th171
 
Tech Tuesday-Harness the Power of Effective Resource Planning with OnePlan’s ...
Tech Tuesday-Harness the Power of Effective Resource Planning with OnePlan’s ...Tech Tuesday-Harness the Power of Effective Resource Planning with OnePlan’s ...
Tech Tuesday-Harness the Power of Effective Resource Planning with OnePlan’s ...OnePlan Solutions
 
How To Use Server-Side Rendering with Nuxt.js
How To Use Server-Side Rendering with Nuxt.jsHow To Use Server-Side Rendering with Nuxt.js
How To Use Server-Side Rendering with Nuxt.jsAndolasoft Inc
 

Kürzlich hochgeladen (20)

Unlocking the Future of AI Agents with Large Language Models
Unlocking the Future of AI Agents with Large Language ModelsUnlocking the Future of AI Agents with Large Language Models
Unlocking the Future of AI Agents with Large Language Models
 
Hand gesture recognition PROJECT PPT.pptx
Hand gesture recognition PROJECT PPT.pptxHand gesture recognition PROJECT PPT.pptx
Hand gesture recognition PROJECT PPT.pptx
 
Microsoft AI Transformation Partner Playbook.pdf
Microsoft AI Transformation Partner Playbook.pdfMicrosoft AI Transformation Partner Playbook.pdf
Microsoft AI Transformation Partner Playbook.pdf
 
why an Opensea Clone Script might be your perfect match.pdf
why an Opensea Clone Script might be your perfect match.pdfwhy an Opensea Clone Script might be your perfect match.pdf
why an Opensea Clone Script might be your perfect match.pdf
 
Optimizing AI for immediate response in Smart CCTV
Optimizing AI for immediate response in Smart CCTVOptimizing AI for immediate response in Smart CCTV
Optimizing AI for immediate response in Smart CCTV
 
Software Quality Assurance Interview Questions
Software Quality Assurance Interview QuestionsSoftware Quality Assurance Interview Questions
Software Quality Assurance Interview Questions
 
HR Software Buyers Guide in 2024 - HRSoftware.com
HR Software Buyers Guide in 2024 - HRSoftware.comHR Software Buyers Guide in 2024 - HRSoftware.com
HR Software Buyers Guide in 2024 - HRSoftware.com
 
call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️
call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️
call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️
 
Test Automation Strategy for Frontend and Backend
Test Automation Strategy for Frontend and BackendTest Automation Strategy for Frontend and Backend
Test Automation Strategy for Frontend and Backend
 
Steps To Getting Up And Running Quickly With MyTimeClock Employee Scheduling ...
Steps To Getting Up And Running Quickly With MyTimeClock Employee Scheduling ...Steps To Getting Up And Running Quickly With MyTimeClock Employee Scheduling ...
Steps To Getting Up And Running Quickly With MyTimeClock Employee Scheduling ...
 
Reassessing the Bedrock of Clinical Function Models: An Examination of Large ...
Reassessing the Bedrock of Clinical Function Models: An Examination of Large ...Reassessing the Bedrock of Clinical Function Models: An Examination of Large ...
Reassessing the Bedrock of Clinical Function Models: An Examination of Large ...
 
Call Girls In Mukherjee Nagar 📱 9999965857 🤩 Delhi 🫦 HOT AND SEXY VVIP 🍎 SE...
Call Girls In Mukherjee Nagar 📱  9999965857  🤩 Delhi 🫦 HOT AND SEXY VVIP 🍎 SE...Call Girls In Mukherjee Nagar 📱  9999965857  🤩 Delhi 🫦 HOT AND SEXY VVIP 🍎 SE...
Call Girls In Mukherjee Nagar 📱 9999965857 🤩 Delhi 🫦 HOT AND SEXY VVIP 🍎 SE...
 
Unveiling the Tech Salsa of LAMs with Janus in Real-Time Applications
Unveiling the Tech Salsa of LAMs with Janus in Real-Time ApplicationsUnveiling the Tech Salsa of LAMs with Janus in Real-Time Applications
Unveiling the Tech Salsa of LAMs with Janus in Real-Time Applications
 
Salesforce Certified Field Service Consultant
Salesforce Certified Field Service ConsultantSalesforce Certified Field Service Consultant
Salesforce Certified Field Service Consultant
 
Shapes for Sharing between Graph Data Spaces - and Epistemic Querying of RDF-...
Shapes for Sharing between Graph Data Spaces - and Epistemic Querying of RDF-...Shapes for Sharing between Graph Data Spaces - and Epistemic Querying of RDF-...
Shapes for Sharing between Graph Data Spaces - and Epistemic Querying of RDF-...
 
W01_panagenda_Navigating-the-Future-with-The-Hitchhikers-Guide-to-Notes-and-D...
W01_panagenda_Navigating-the-Future-with-The-Hitchhikers-Guide-to-Notes-and-D...W01_panagenda_Navigating-the-Future-with-The-Hitchhikers-Guide-to-Notes-and-D...
W01_panagenda_Navigating-the-Future-with-The-Hitchhikers-Guide-to-Notes-and-D...
 
SyndBuddy AI 2k Review 2024: Revolutionizing Content Syndication with AI
SyndBuddy AI 2k Review 2024: Revolutionizing Content Syndication with AISyndBuddy AI 2k Review 2024: Revolutionizing Content Syndication with AI
SyndBuddy AI 2k Review 2024: Revolutionizing Content Syndication with AI
 
Learn the Fundamentals of XCUITest Framework_ A Beginner's Guide.pdf
Learn the Fundamentals of XCUITest Framework_ A Beginner's Guide.pdfLearn the Fundamentals of XCUITest Framework_ A Beginner's Guide.pdf
Learn the Fundamentals of XCUITest Framework_ A Beginner's Guide.pdf
 
Tech Tuesday-Harness the Power of Effective Resource Planning with OnePlan’s ...
Tech Tuesday-Harness the Power of Effective Resource Planning with OnePlan’s ...Tech Tuesday-Harness the Power of Effective Resource Planning with OnePlan’s ...
Tech Tuesday-Harness the Power of Effective Resource Planning with OnePlan’s ...
 
How To Use Server-Side Rendering with Nuxt.js
How To Use Server-Side Rendering with Nuxt.jsHow To Use Server-Side Rendering with Nuxt.js
How To Use Server-Side Rendering with Nuxt.js
 

Webinar: Solr & Spark for Real Time Big Data Analytics

  • 1.
  • 3. Solr & Spark •  Spark Overview •  Resilient Distributed Dataset (RDD) •  Solr as a SparkSQL DataSource •  Interacting with Solr from the Spark shell •  Indexing from Spark Streaming •  Lucidworks Fusion and Spark •  Other goodies (document matching, term vectors, etc.)
  • 4. •  Solr user since 2010, committer since April 2014, work for Lucidworks, PMC member ~ May 2015 •  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 … working with Spark for about 1 year •  Other contributions include Solr on YARN, Solr Scale Toolkit, and Solr/Storm integration project on github About Me …
  • 5. About Solr •  Vibrant, thriving open source community •  Solr 5.3 just released! ü  Authentication and authorization framework; plugins for Kerberos and Ranger ü  ~2x indexing performance w/ replication http://lucidworks.com/blog/indexing-performance-solr-5-2-now-twice-fast/ ü  Field cardinality estimation using HyperLogLog ü  Rule-based replica placement strategy •  Deploy to YARN cluster using Slider
  • 6. 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
  • 7. Spark: Memory-centric Distributed Computation 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 Tachyon languages Scala Java Python R shared memory
  • 8. 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
  • 9. 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
  • 10. 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://...")  
  • 11. 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
  • 12. Solr as a Spark SQL Data Source •  DataFrame is a DSL for distributed data manipulation •  Data source provides a DataFrame •  Uniform way of working with data from multiple sources •  Hive, JDBC, Solr, Cassandra, etc. •  Seamless integration with other Spark technologies: SparkR, Python, MLlib … Map<String,  String>  options  =  new  HashMap<String,  String>();   options.put("zkhost",  zkHost);   options.put("collection”,  "tweets");     DataFrame  df  =  sqlContext.read().format("solr").options(options).load();   count  =  df.filter(df.col("type_s").equalTo(“echo")).count();  
  • 13. Spark SQL Query Solr, then expose results as a SQL table Map<String,  String>  options  =  new  HashMap<String,  String>();   options.put("zkhost",  zkHost);   options.put("collection”,  "tweets");     DataFrame  df  =  sqlContext.read().format("solr").options(options).load();     df.registerTempTable("tweets");     sqlContext.sql("SELECT  count(*)  FROM  tweets  WHERE  type_s='echo'");  
  • 14. 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) •  Stream docs from Solr (vs. building lists on the server-side) •  More parallelism using a range filter on a numeric field (_version_) e.g. 10 shards x 10 splits per shard == 100 concurrent Spark tasks
  • 15. 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”  -­‐>  “socialdata”,            “zkHost”  -­‐>  “…”)   SolrRDD: Reading data from Solr into Spark
  • 16. Query Solr from the Spark Shell Interactive data mining with the full power of Solr queries ADD_JARS=$PROJECT_HOME/target/spark-­‐solr-­‐1.0-­‐SNAPSHOT.jar  bin/spark-­‐shell     val  solrDF  =  sqlContext.load("solr",  Map(      "zkHost"  -­‐>  "localhost:9983",      "collection"  -­‐>  "gettingstarted")).filter("provider_s='twitter'")     solrDF.registerTempTable("tweets")     sqlContext.sql("SELECT  COUNT(type_s)  FROM  tweets  WHERE  type_s='echo'").show()     tweets.filter("type_s='post'").groupBy("author_s").count().    orderBy(desc("count")).limit(10).show()  
  • 17. 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
  • 18. 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
  • 19. 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);  
  • 20. 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;                  }              }          );      }  
  • 21. com.lucidworks.spark.ShardPartitioner •  Custom partitioning scheme for RDD using Solr’s DocRouter •  Stream docs directly to each shard leader using metadata from ZooKeeper, document 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);              }        }   });  
  • 23. Fusion & Spark •  Users leave evidence of their needs & experience as they use your app •  Fusion closes the feedback loop to improve results based on user “signals” •  Scale out distributed aggregation jobs across Spark cluster •  Recommendations at scale
  • 24. Extra goodies: 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());    
  • 25. 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
  • 26. 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
  • 27. Getting Started git clone https://github.com/LucidWorks/spark-solr/ cd spark-solr mvn clean package -DskipTests See: https://lucidworks.com/blog/solr-spark-sql-datasource/
  • 28. Wrap-up and Q & A Download Fusion: http://lucidworks.com/fusion/download/ Feel free to reach out to me with questions: tim.potter@lucidworks.com / @thelabdude