SlideShare a Scribd company logo
Druid in Production
Dori Waldman - Big Data Lead
Guy Shemer - Big Data Expert
Alon Edelman - Big Data consultant
● Druid
○ Demo
○ What is druid and Why you need it
○ Other solutions ...
● Production pains and how it works :
○ cardinality
○ cache
○ dimension types (list/regular/map)
○ segment size
○ partition
○ monitoring and analyze hotspot
○ Query examples
○ lookups
● Interface
○ Pivot, Facet , superset ...
○ Druid-Sql (JDBC)
○ Rest
Agenda
Demo
Why ?
● fast (real-time) analytics on large time series data
○ MapReduce / Spark - are not design for real time queries.
○ MPP expensive / slow
● Just send raw data to druid → mention which attributes are the
dimensions, metrics and how to aggregate the metrics → Druid
will create cube (datasource)
○ Relational does not scale , we need fast queries on large
data.
○ Key-value tables require table per predefined query , and we
need dynamic queries (cube)
http://static.druid.io/docs/druid.pdf
● We want to answer questions like:
○ #edits on the page Justin Bieber from males in San Francisco?
○ average #characters , added by people from Calgary over the last month?
○ arbitrary combination of dimensions to return with subsecond latencies.
Row value can be dimension (~where in sql) or metric (measure)
● Dimensions are fields that can be filtered on or grouped by.
● Metrics are fields that can be aggregated. They are often stored as numbers but
can also be stored as HyperLogLog sketches (approximate)..
For example
If Click is a dimension we can select this dimension and see how the data is splitted
according to the selected value (might be better to convert as categories 0-20)
If Click is a metric it will be a counter result like for how many clicks we have in
Israel
Dimension / Metric
Country ApplicationId Clicks
Israel 2 18
Israel 3 22
USA 80 19
Other options
● Open source solution:
○ Pinot (https://github.com/linkedin/pinot)
○ clickHouse (https://clickhouse.yandex/)
○ Presto (https://prestodb.io/)
https://medium.com/@leventov/comparison-of-the-open-source-olap-systems-for-big-data-clickhouse-druid-and-pinot-8e042a5ed1c7
Druid
Components
Components
● RealTime nodes - ingest and query event streams, events
are immediately available to query, saved in cache and
persist to global storage (s3/hdfs) “deepStorage”
● Historical nodes - load and serve the immutable blocks of
data (segments) from deep storage, those are the main
workers
● Broker nodes - query routers to historical and real-time
nodes, communicate with ZK to understand where relevant
segments are located
● Coordinator nodes - tell historical nodes to load new data,
drop outdated data, replicate data, and Balance by move data
Components
● Overlord node - Manages task
distribution to middle managers.
responsible for accepting tasks,
coordinating task distribution, creating
locks around tasks, and returning statuses
to callers
● Middle manager node - Executes
submitted tasks by forward slices of tasks
to peons.
In case druid runs in local mode, this part
is redundant since overlord will also take
this responsibility
● Peon - single task in a single JVM.
Several peons may run on same node
Components
(Stream)
● Tranquility -
○ Ingest from kafka/http/samza/flink …
○ Will be out of life
○ Connects to the ZooKeeper of the kafka cluster
○ Can connect to several clusters and read from several topics for the
same Druid data source.
○ Can't handle events after window closes
● Kafka-Indexing-Service
○ ingest from kafka only (kafka 10+)
○ Connects directly to Kafka’s brokers
○ Is able to connect to one cluster and one topic for one druid data source
○ Indexer manage its tasks better, use checkpoint (~exactly once)
○ Can update events for old segments (no window)
○ can be spot instance (for other nodes its less recommended)
Druid
Extensions
Batch
Ingestion
Batch
Ingestion
Druid support :
JSON
CSV
AVRO
PARQUET
ORC
Support:
● Values
● multiValue (array)
- each item in the
list will be explode
to row
● maps (new
feature)
Batch
Ingestion
Indexing Task Types
● index_hadoop (with EMR)
○ Hadoop-based batch ingestion. using Hadoop/EMR cluster to
perform data processing and ingestion
● Index (No EMR)
○ For small amount of data , task execute within the indexing
service without external hadoop resources
Batch
Ingestion
Input source for batch indexing
● local
○ For POC
● Static (S3/ HDFS etc..)
○ Ingesting from your raw data
○ Support also Parquet
○ Can be mapped dynamically
to specific date
● Druid’s Deep Storage
○ Use segments from one
datasource from deep storage
and transform them to another
datasource, clean dimensions,
change granularity etc..
"inputSpec" : {
"type" : "static",
"paths" : "/MyDirectory/example/wikipedia_data.json"
}
"inputSpec": {
"type": "static",
"paths": "s3n://prod/raw/2018-01-01/00/,
s3n://staging/raw/2018-01-01/00/",
"filePattern": ".gz"
}
"inputSpec": {
"type": "dataSource",
"ingestionSpec": {
"dataSource" : "Hourly",
"intervals" :
["2017-11-06T00:00:00.000Z/2017-11-07T00:00:00.000Z"]
}
Lookups
Lookups
● Purpose : replace dimensions values , for example replace “1”
with “New York City”
● in case the mapping is 1:1 an optimization (“injective”:true)
should be used, it will replace the value on the query result
and not on the query input
● Lookups has no history (if value of 1 was “new york” and it was
changed to “new your city” the old value will not appear in the
query result.
● Very small lookups (count of keys on the order of a few dozen to
a few hundred) can be passed at query time as a "map" lookup
● Usually you will use global cached lookups from DB / file / kafka
Queries
Query:
TopN
● TopN
○ grouped by single dimension, sort
(order) according to the metric
(~ “group by” one dimension + order )
○ TopNs are approximate in that each node
will rank their top K results and only return
those top K results to the broker
○ To get exact result use groupBy query
and sort the results (better to avoid)
Query:
TopN
● TopN Hell- in the Pivot
Pivot use nested TopN’s (filter and topN per row)
Try to reduce number of unnecessary topN queries
Query:
GroupBy
GroupBy
○ Grouped by multiple dimensions.
○ Unlike TopN, can use ‘having’ conditions over aggregated data.
Druid vision is to
replace timeseries and
topN with groupBy
advance query
Query:
TimeSeries
● Timeseries
○ grouped by time dimension only (“no dimensions)
○ Timeseries query will generally be faster than groupBy as it
taking advantage of the fact that segments are already sorted on
time and does not need to use a hash table for merging.
Query:
SQL
● Druid SQL
○ Translates SQL into native Druid queries on the query broker
■ using JSON over HTTP by posting to the endpoint /druid/v2/sql/
■ SQL queries using the Avatica JDBC driver.
Query:
TimeBoundary
/ MetaData
● Time boundary
○ Return the earliest and latest
data points of a data set
● Segment metadata
○ Per-segment information:
■ dimensions cardinality,
■ min/max value in
dimension
■ number of rows
● DataSource metadata
○ ...
Other
Queries...
● Select / Scan / Search
○ select - supports pagination, all data is loaded to memory
○ scan - return result in streaming mode
○ search - returns dimension values which match a search criteria.
The biggest difference between select and scan is that, scan query
doesn't retain all rows in memory before rows can be returned to
client.
Query
Performance
● Query with metrics
Metric calculation is done in real time per metric meaning doing sum
of impression and later sum of impressions and sum of clicks will
double the metric calculation time (think about list dimension...)
Druid in Fyber
29PAGE //
Druid
Usage
Hour
Day
● Index 5T row daily from 3 different resources (s3 / kafka)
● 40 dimensions, 10 metrics
● Datasource (table) should be updated every 3 hours
● Query latency ~10 second for query on one dimension , 3
month range
○ Some dimensions are list …
○ Some dimensions use lookups
Requirements
Work in scale
● We started with 14 dimensions (no lists) → for 8 month druid
answer all requirements
● We added 20 more dimensions (with list) → druid query time
was slow ...
● Hardware :
○ 30 nodes(i3.8xlarge), each node manage historical and
middleManager service
○ 2 nodes (m4.2xlarge) , each node manage coordinator and
overload services
○ 11 nodes (c4.2xlarge), each node manage tranquility service
○ 2 nodes (i3.8xlarge), each node manage broker service
■ (1 broker : 10 historical)
○ Memcached : 3 nodes (cache.r3.8xlarge), version: 1.4.34
Hardware
Data cleanup
● Cleanup reduce cardinality (replace it with dummy value)
● Its all about reducing number of rows in the datasource
○ Druid saves the data in columnar storage but in order to get
better performance the cleanup process reduces #rows
(although the query is on 3 columns it needs to read all items in
the column)
Data cleanup
● The dimensions correlation is important.
○ lets say we have one dimension which is city with 2000 unique
cities
■ Adding gender dimension will double #rows (assume in our
row data we have both male/female per city)
■ Adding country (although we have 200 unique countries) will
not impact the same (cartesian product) as there is a relation
between city and county of 1:M.
● better to reduce non related dimensions like country and age
Data cleanup
○ Use timeseries query with “count” aggregation (~ count(*) in
druid Sql) to measure your cleanup benefit
○ you can also use estimator with cardinality aggregation
○ if you want to estimate without doing cleanup you can use
virtualColumns (filter out specific values) with byRow cardinality
estimator
segments
● Shard size should be balanced between disk optimization
500M-1.5G and cpu optimization (core per segment during
query), take list in this calculation …
Shard minimum size should be 100M
● POC - convert list to bitwise vector
Partition
● Partition type
○ By default, druid partitions the data according to timestamp In
addition you need to specify hashed/ single dimension partition
■ partition may result with unbalanced segments
■ The default of hashed partition using all dimensions
■ Hashed partitioning is recommended in most cases, as it will improve indexing
performance and create more uniformly sized data segments relative to
single-dimension partitioning.
■ single-dimension partition may be preferred in context of multi tenancy use cases.
■ Might want to avoid default hashed in case of long tail
"partitionsSpec": {
"type": "hashed",
"targetPartitionSize": "4500000"
}
"partitionsSpec": {
"type": "dimension",
"targetPartitionSize": "10000000",
"partitionDimension": publisherId"
}
"partitionsSpec": {
"type": "hashed",
"numShards": "12",
"partitionDimensions": ["publisherId"]
}
Cache
● Cache :
○ hybrid
■ L1-caffein (local),
■ L2-memcached (global)
when segment move between machines , caffein cache will be
invalidate for those segments
○ warm Cache for popular queries (~300ms)
○ Cache is saved per segment and date , cache key contains the
dimensions ,metrics , filters
○ TopN threshold are part of the key : 0-1000, 1001, 1002 …
○ Cache in the historical nodes not broker in order to merge less
data in the broker side
Cache
● Cache :
○ Lookup has pollPeriod meaning if its set to 1 day then cache will
be invalid (no eviction) every day even if lookup was not updated
(tscolumn), since Imply 2.4.6 this issue should be fixed by setting
injective=true in the lookup configuration meaning lookup is not
part of the cache key anymore its a post-aggregation action in the
brokers.
■ increase lookup pooling period + hard set injective=true in the
query is workaround till 2.4.6
○ rebuild segment (~new) cause the cache to be invalidate
Cache
Monitoring
All segments (507) are scanned not from the cache
Broker logs
● Production issue :
○ Cluster was slow
■ doing rebalance all the time
■ nodes disappear , no crash in the nodes
■ we found that during this time GC took long time , and in the log
we saw ZK disconnect-connect
○ We increased ZK connection timeout
○ Solution was to decrease historical memory (reduce GC time)
Monitoring
/ Debug
Fix hotspot by increase
#segment to move till
data is balanced
Statsd emitter does not
send all metrics , use
another (clarity / kafka)
Druid
Pattern
● Two data sources
○ small (less rows and dimensions)
○ Large (all data) , query with filter only
Extra
● Load rules are used to manage which data is available to druid,
for example we can set it to save only last month data and drop
old data every day
● Priority - druid support query by priority
● Avoid Javascript extension (post aggregation function)
THANK YOU
Dori.waldman@fyber.com
https://www.linkedin.com/in/doriwaldman/
https://www.slideshare.net/doriwaldman
Alon@nextstage.co.il
Guy.shemer@fyber.com
https://www.linkedin.com/in/guysh/

More Related Content

What's hot

Physical Plans in Spark SQL
Physical Plans in Spark SQLPhysical Plans in Spark SQL
Physical Plans in Spark SQL
Databricks
 
A Deep Dive into Query Execution Engine of Spark SQL
A Deep Dive into Query Execution Engine of Spark SQLA Deep Dive into Query Execution Engine of Spark SQL
A Deep Dive into Query Execution Engine of Spark SQL
Databricks
 
Apache Iceberg - A Table Format for Hige Analytic Datasets
Apache Iceberg - A Table Format for Hige Analytic DatasetsApache Iceberg - A Table Format for Hige Analytic Datasets
Apache Iceberg - A Table Format for Hige Analytic Datasets
Alluxio, Inc.
 
Hive + Tez: A Performance Deep Dive
Hive + Tez: A Performance Deep DiveHive + Tez: A Performance Deep Dive
Hive + Tez: A Performance Deep Dive
DataWorks Summit
 
Tame the small files problem and optimize data layout for streaming ingestion...
Tame the small files problem and optimize data layout for streaming ingestion...Tame the small files problem and optimize data layout for streaming ingestion...
Tame the small files problem and optimize data layout for streaming ingestion...
Flink Forward
 
A Thorough Comparison of Delta Lake, Iceberg and Hudi
A Thorough Comparison of Delta Lake, Iceberg and HudiA Thorough Comparison of Delta Lake, Iceberg and Hudi
A Thorough Comparison of Delta Lake, Iceberg and Hudi
Databricks
 
Using Apache Arrow, Calcite, and Parquet to Build a Relational Cache
Using Apache Arrow, Calcite, and Parquet to Build a Relational CacheUsing Apache Arrow, Calcite, and Parquet to Build a Relational Cache
Using Apache Arrow, Calcite, and Parquet to Build a Relational Cache
Dremio Corporation
 
Dr. Elephant for Monitoring and Tuning Apache Spark Jobs on Hadoop with Carl ...
Dr. Elephant for Monitoring and Tuning Apache Spark Jobs on Hadoop with Carl ...Dr. Elephant for Monitoring and Tuning Apache Spark Jobs on Hadoop with Carl ...
Dr. Elephant for Monitoring and Tuning Apache Spark Jobs on Hadoop with Carl ...
Databricks
 
Streaming SQL with Apache Calcite
Streaming SQL with Apache CalciteStreaming SQL with Apache Calcite
Streaming SQL with Apache Calcite
Julian Hyde
 
Evening out the uneven: dealing with skew in Flink
Evening out the uneven: dealing with skew in FlinkEvening out the uneven: dealing with skew in Flink
Evening out the uneven: dealing with skew in Flink
Flink Forward
 
Change Data Feed in Delta
Change Data Feed in DeltaChange Data Feed in Delta
Change Data Feed in Delta
Databricks
 
What is new in Apache Hive 3.0?
What is new in Apache Hive 3.0?What is new in Apache Hive 3.0?
What is new in Apache Hive 3.0?
DataWorks Summit
 
Apache Tez - A New Chapter in Hadoop Data Processing
Apache Tez - A New Chapter in Hadoop Data ProcessingApache Tez - A New Chapter in Hadoop Data Processing
Apache Tez - A New Chapter in Hadoop Data Processing
DataWorks Summit
 
Deep dive into stateful stream processing in structured streaming by Tathaga...
Deep dive into stateful stream processing in structured streaming  by Tathaga...Deep dive into stateful stream processing in structured streaming  by Tathaga...
Deep dive into stateful stream processing in structured streaming by Tathaga...
Databricks
 
Deep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache SparkDeep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache Spark
Databricks
 
Building a Streaming Microservice Architecture: with Apache Spark Structured ...
Building a Streaming Microservice Architecture: with Apache Spark Structured ...Building a Streaming Microservice Architecture: with Apache Spark Structured ...
Building a Streaming Microservice Architecture: with Apache Spark Structured ...
Databricks
 
Flink history, roadmap and vision
Flink history, roadmap and visionFlink history, roadmap and vision
Flink history, roadmap and vision
Stephan Ewen
 
Building a Virtual Data Lake with Apache Arrow
Building a Virtual Data Lake with Apache ArrowBuilding a Virtual Data Lake with Apache Arrow
Building a Virtual Data Lake with Apache Arrow
Dremio Corporation
 
Processing Large Data with Apache Spark -- HasGeek
Processing Large Data with Apache Spark -- HasGeekProcessing Large Data with Apache Spark -- HasGeek
Processing Large Data with Apache Spark -- HasGeek
Venkata Naga Ravi
 
Apache flink
Apache flinkApache flink
Apache flink
pranay kumar
 

What's hot (20)

Physical Plans in Spark SQL
Physical Plans in Spark SQLPhysical Plans in Spark SQL
Physical Plans in Spark SQL
 
A Deep Dive into Query Execution Engine of Spark SQL
A Deep Dive into Query Execution Engine of Spark SQLA Deep Dive into Query Execution Engine of Spark SQL
A Deep Dive into Query Execution Engine of Spark SQL
 
Apache Iceberg - A Table Format for Hige Analytic Datasets
Apache Iceberg - A Table Format for Hige Analytic DatasetsApache Iceberg - A Table Format for Hige Analytic Datasets
Apache Iceberg - A Table Format for Hige Analytic Datasets
 
Hive + Tez: A Performance Deep Dive
Hive + Tez: A Performance Deep DiveHive + Tez: A Performance Deep Dive
Hive + Tez: A Performance Deep Dive
 
Tame the small files problem and optimize data layout for streaming ingestion...
Tame the small files problem and optimize data layout for streaming ingestion...Tame the small files problem and optimize data layout for streaming ingestion...
Tame the small files problem and optimize data layout for streaming ingestion...
 
A Thorough Comparison of Delta Lake, Iceberg and Hudi
A Thorough Comparison of Delta Lake, Iceberg and HudiA Thorough Comparison of Delta Lake, Iceberg and Hudi
A Thorough Comparison of Delta Lake, Iceberg and Hudi
 
Using Apache Arrow, Calcite, and Parquet to Build a Relational Cache
Using Apache Arrow, Calcite, and Parquet to Build a Relational CacheUsing Apache Arrow, Calcite, and Parquet to Build a Relational Cache
Using Apache Arrow, Calcite, and Parquet to Build a Relational Cache
 
Dr. Elephant for Monitoring and Tuning Apache Spark Jobs on Hadoop with Carl ...
Dr. Elephant for Monitoring and Tuning Apache Spark Jobs on Hadoop with Carl ...Dr. Elephant for Monitoring and Tuning Apache Spark Jobs on Hadoop with Carl ...
Dr. Elephant for Monitoring and Tuning Apache Spark Jobs on Hadoop with Carl ...
 
Streaming SQL with Apache Calcite
Streaming SQL with Apache CalciteStreaming SQL with Apache Calcite
Streaming SQL with Apache Calcite
 
Evening out the uneven: dealing with skew in Flink
Evening out the uneven: dealing with skew in FlinkEvening out the uneven: dealing with skew in Flink
Evening out the uneven: dealing with skew in Flink
 
Change Data Feed in Delta
Change Data Feed in DeltaChange Data Feed in Delta
Change Data Feed in Delta
 
What is new in Apache Hive 3.0?
What is new in Apache Hive 3.0?What is new in Apache Hive 3.0?
What is new in Apache Hive 3.0?
 
Apache Tez - A New Chapter in Hadoop Data Processing
Apache Tez - A New Chapter in Hadoop Data ProcessingApache Tez - A New Chapter in Hadoop Data Processing
Apache Tez - A New Chapter in Hadoop Data Processing
 
Deep dive into stateful stream processing in structured streaming by Tathaga...
Deep dive into stateful stream processing in structured streaming  by Tathaga...Deep dive into stateful stream processing in structured streaming  by Tathaga...
Deep dive into stateful stream processing in structured streaming by Tathaga...
 
Deep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache SparkDeep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache Spark
 
Building a Streaming Microservice Architecture: with Apache Spark Structured ...
Building a Streaming Microservice Architecture: with Apache Spark Structured ...Building a Streaming Microservice Architecture: with Apache Spark Structured ...
Building a Streaming Microservice Architecture: with Apache Spark Structured ...
 
Flink history, roadmap and vision
Flink history, roadmap and visionFlink history, roadmap and vision
Flink history, roadmap and vision
 
Building a Virtual Data Lake with Apache Arrow
Building a Virtual Data Lake with Apache ArrowBuilding a Virtual Data Lake with Apache Arrow
Building a Virtual Data Lake with Apache Arrow
 
Processing Large Data with Apache Spark -- HasGeek
Processing Large Data with Apache Spark -- HasGeekProcessing Large Data with Apache Spark -- HasGeek
Processing Large Data with Apache Spark -- HasGeek
 
Apache flink
Apache flinkApache flink
Apache flink
 

Similar to Druid

Real-time analytics with Druid at Appsflyer
Real-time analytics with Druid at AppsflyerReal-time analytics with Druid at Appsflyer
Real-time analytics with Druid at Appsflyer
Michael Spector
 
Ledingkart Meetup #2: Scaling Search @Lendingkart
Ledingkart Meetup #2: Scaling Search @LendingkartLedingkart Meetup #2: Scaling Search @Lendingkart
Ledingkart Meetup #2: Scaling Search @Lendingkart
Mukesh Singh
 
Big Data processing with Apache Spark
Big Data processing with Apache SparkBig Data processing with Apache Spark
Big Data processing with Apache Spark
Lucian Neghina
 
MongoDB World 2019: Packing Up Your Data and Moving to MongoDB Atlas
MongoDB World 2019: Packing Up Your Data and Moving to MongoDB AtlasMongoDB World 2019: Packing Up Your Data and Moving to MongoDB Atlas
MongoDB World 2019: Packing Up Your Data and Moving to MongoDB Atlas
MongoDB
 
Optimizing Geospatial Operations with Server-side Programming in HBase and Ac...
Optimizing Geospatial Operations with Server-side Programming in HBase and Ac...Optimizing Geospatial Operations with Server-side Programming in HBase and Ac...
Optimizing Geospatial Operations with Server-side Programming in HBase and Ac...
DataWorks Summit
 
OSMC 2018 | Learnings, patterns and Uber’s metrics platform M3, open sourced ...
OSMC 2018 | Learnings, patterns and Uber’s metrics platform M3, open sourced ...OSMC 2018 | Learnings, patterns and Uber’s metrics platform M3, open sourced ...
OSMC 2018 | Learnings, patterns and Uber’s metrics platform M3, open sourced ...
NETWAYS
 
Dirty Data? Clean it up! - Rocky Mountain DataCon 2016
Dirty Data? Clean it up! - Rocky Mountain DataCon 2016Dirty Data? Clean it up! - Rocky Mountain DataCon 2016
Dirty Data? Clean it up! - Rocky Mountain DataCon 2016
Dan Lynn
 
Our journey with druid - from initial research to full production scale
Our journey with druid - from initial research to full production scaleOur journey with druid - from initial research to full production scale
Our journey with druid - from initial research to full production scale
Itai Yaffe
 
Lessons learned from designing a QA Automation for analytics databases (big d...
Lessons learned from designing a QA Automation for analytics databases (big d...Lessons learned from designing a QA Automation for analytics databases (big d...
Lessons learned from designing a QA Automation for analytics databases (big d...
Omid Vahdaty
 
Big data processing systems research
Big data processing systems researchBig data processing systems research
Big data processing systems research
Vasia Kalavri
 
Big Data in 200 km/h | AWS Big Data Demystified #1.3
Big Data in 200 km/h | AWS Big Data Demystified #1.3  Big Data in 200 km/h | AWS Big Data Demystified #1.3
Big Data in 200 km/h | AWS Big Data Demystified #1.3
Omid Vahdaty
 
Data pipelines from zero to solid
Data pipelines from zero to solidData pipelines from zero to solid
Data pipelines from zero to solid
Lars Albertsson
 
Dirty data? Clean it up! - Datapalooza Denver 2016
Dirty data? Clean it up! - Datapalooza Denver 2016Dirty data? Clean it up! - Datapalooza Denver 2016
Dirty data? Clean it up! - Datapalooza Denver 2016
Dan Lynn
 
BlaBlaCar Elastic Search Feedback
BlaBlaCar Elastic Search FeedbackBlaBlaCar Elastic Search Feedback
BlaBlaCar Elastic Search Feedback
sinfomicien
 
Big data should be simple
Big data should be simpleBig data should be simple
Big data should be simple
Dori Waldman
 
Druid meetup @walkme
Druid meetup @walkmeDruid meetup @walkme
Druid meetup @walkme
Dori Waldman
 
AWS Big Data Demystified #1.2 | Big Data architecture lessons learned
AWS Big Data Demystified #1.2 | Big Data architecture lessons learned AWS Big Data Demystified #1.2 | Big Data architecture lessons learned
AWS Big Data Demystified #1.2 | Big Data architecture lessons learned
Omid Vahdaty
 
NoSQL Solutions - a comparative study
NoSQL Solutions - a comparative studyNoSQL Solutions - a comparative study
NoSQL Solutions - a comparative study
Guillaume Lefranc
 
Big data real time architectures
Big data real time architecturesBig data real time architectures
Big data real time architectures
Daniel Marcous
 
Counting Unique Users in Real-Time: Here's a Challenge for You!
Counting Unique Users in Real-Time: Here's a Challenge for You!Counting Unique Users in Real-Time: Here's a Challenge for You!
Counting Unique Users in Real-Time: Here's a Challenge for You!
DataWorks Summit
 

Similar to Druid (20)

Real-time analytics with Druid at Appsflyer
Real-time analytics with Druid at AppsflyerReal-time analytics with Druid at Appsflyer
Real-time analytics with Druid at Appsflyer
 
Ledingkart Meetup #2: Scaling Search @Lendingkart
Ledingkart Meetup #2: Scaling Search @LendingkartLedingkart Meetup #2: Scaling Search @Lendingkart
Ledingkart Meetup #2: Scaling Search @Lendingkart
 
Big Data processing with Apache Spark
Big Data processing with Apache SparkBig Data processing with Apache Spark
Big Data processing with Apache Spark
 
MongoDB World 2019: Packing Up Your Data and Moving to MongoDB Atlas
MongoDB World 2019: Packing Up Your Data and Moving to MongoDB AtlasMongoDB World 2019: Packing Up Your Data and Moving to MongoDB Atlas
MongoDB World 2019: Packing Up Your Data and Moving to MongoDB Atlas
 
Optimizing Geospatial Operations with Server-side Programming in HBase and Ac...
Optimizing Geospatial Operations with Server-side Programming in HBase and Ac...Optimizing Geospatial Operations with Server-side Programming in HBase and Ac...
Optimizing Geospatial Operations with Server-side Programming in HBase and Ac...
 
OSMC 2018 | Learnings, patterns and Uber’s metrics platform M3, open sourced ...
OSMC 2018 | Learnings, patterns and Uber’s metrics platform M3, open sourced ...OSMC 2018 | Learnings, patterns and Uber’s metrics platform M3, open sourced ...
OSMC 2018 | Learnings, patterns and Uber’s metrics platform M3, open sourced ...
 
Dirty Data? Clean it up! - Rocky Mountain DataCon 2016
Dirty Data? Clean it up! - Rocky Mountain DataCon 2016Dirty Data? Clean it up! - Rocky Mountain DataCon 2016
Dirty Data? Clean it up! - Rocky Mountain DataCon 2016
 
Our journey with druid - from initial research to full production scale
Our journey with druid - from initial research to full production scaleOur journey with druid - from initial research to full production scale
Our journey with druid - from initial research to full production scale
 
Lessons learned from designing a QA Automation for analytics databases (big d...
Lessons learned from designing a QA Automation for analytics databases (big d...Lessons learned from designing a QA Automation for analytics databases (big d...
Lessons learned from designing a QA Automation for analytics databases (big d...
 
Big data processing systems research
Big data processing systems researchBig data processing systems research
Big data processing systems research
 
Big Data in 200 km/h | AWS Big Data Demystified #1.3
Big Data in 200 km/h | AWS Big Data Demystified #1.3  Big Data in 200 km/h | AWS Big Data Demystified #1.3
Big Data in 200 km/h | AWS Big Data Demystified #1.3
 
Data pipelines from zero to solid
Data pipelines from zero to solidData pipelines from zero to solid
Data pipelines from zero to solid
 
Dirty data? Clean it up! - Datapalooza Denver 2016
Dirty data? Clean it up! - Datapalooza Denver 2016Dirty data? Clean it up! - Datapalooza Denver 2016
Dirty data? Clean it up! - Datapalooza Denver 2016
 
BlaBlaCar Elastic Search Feedback
BlaBlaCar Elastic Search FeedbackBlaBlaCar Elastic Search Feedback
BlaBlaCar Elastic Search Feedback
 
Big data should be simple
Big data should be simpleBig data should be simple
Big data should be simple
 
Druid meetup @walkme
Druid meetup @walkmeDruid meetup @walkme
Druid meetup @walkme
 
AWS Big Data Demystified #1.2 | Big Data architecture lessons learned
AWS Big Data Demystified #1.2 | Big Data architecture lessons learned AWS Big Data Demystified #1.2 | Big Data architecture lessons learned
AWS Big Data Demystified #1.2 | Big Data architecture lessons learned
 
NoSQL Solutions - a comparative study
NoSQL Solutions - a comparative studyNoSQL Solutions - a comparative study
NoSQL Solutions - a comparative study
 
Big data real time architectures
Big data real time architecturesBig data real time architectures
Big data real time architectures
 
Counting Unique Users in Real-Time: Here's a Challenge for You!
Counting Unique Users in Real-Time: Here's a Challenge for You!Counting Unique Users in Real-Time: Here's a Challenge for You!
Counting Unique Users in Real-Time: Here's a Challenge for You!
 

More from Dori Waldman

openai.pptx
openai.pptxopenai.pptx
openai.pptx
Dori Waldman
 
iceberg introduction.pptx
iceberg introduction.pptxiceberg introduction.pptx
iceberg introduction.pptx
Dori Waldman
 
spark stream - kafka - the right way
spark stream - kafka - the right way spark stream - kafka - the right way
spark stream - kafka - the right way
Dori Waldman
 
Machine Learning and Deep Learning 4 dummies
Machine Learning and Deep Learning 4 dummies Machine Learning and Deep Learning 4 dummies
Machine Learning and Deep Learning 4 dummies
Dori Waldman
 
Memcached
MemcachedMemcached
Memcached
Dori Waldman
 
whats new in java 8
whats new in java 8 whats new in java 8
whats new in java 8
Dori Waldman
 
Spark streaming with kafka
Spark streaming with kafkaSpark streaming with kafka
Spark streaming with kafka
Dori Waldman
 
Spark stream - Kafka
Spark stream - Kafka Spark stream - Kafka
Spark stream - Kafka
Dori Waldman
 
Dori waldman android _course_2
Dori waldman android _course_2Dori waldman android _course_2
Dori waldman android _course_2
Dori Waldman
 
Dori waldman android _course
Dori waldman android _courseDori waldman android _course
Dori waldman android _course
Dori Waldman
 

More from Dori Waldman (10)

openai.pptx
openai.pptxopenai.pptx
openai.pptx
 
iceberg introduction.pptx
iceberg introduction.pptxiceberg introduction.pptx
iceberg introduction.pptx
 
spark stream - kafka - the right way
spark stream - kafka - the right way spark stream - kafka - the right way
spark stream - kafka - the right way
 
Machine Learning and Deep Learning 4 dummies
Machine Learning and Deep Learning 4 dummies Machine Learning and Deep Learning 4 dummies
Machine Learning and Deep Learning 4 dummies
 
Memcached
MemcachedMemcached
Memcached
 
whats new in java 8
whats new in java 8 whats new in java 8
whats new in java 8
 
Spark streaming with kafka
Spark streaming with kafkaSpark streaming with kafka
Spark streaming with kafka
 
Spark stream - Kafka
Spark stream - Kafka Spark stream - Kafka
Spark stream - Kafka
 
Dori waldman android _course_2
Dori waldman android _course_2Dori waldman android _course_2
Dori waldman android _course_2
 
Dori waldman android _course
Dori waldman android _courseDori waldman android _course
Dori waldman android _course
 

Recently uploaded

Data Storytelling Final Project for MBA 635
Data Storytelling Final Project for MBA 635Data Storytelling Final Project for MBA 635
Data Storytelling Final Project for MBA 635
HeidiLivengood
 
Solution Manual for First Course in Abstract Algebra A, 8th Edition by John B...
Solution Manual for First Course in Abstract Algebra A, 8th Edition by John B...Solution Manual for First Course in Abstract Algebra A, 8th Edition by John B...
Solution Manual for First Course in Abstract Algebra A, 8th Edition by John B...
rightmanforbloodline
 
SOFTWARE ENGINEERING-UNIT-1SOFTWARE ENGINEERING
SOFTWARE ENGINEERING-UNIT-1SOFTWARE ENGINEERINGSOFTWARE ENGINEERING-UNIT-1SOFTWARE ENGINEERING
SOFTWARE ENGINEERING-UNIT-1SOFTWARE ENGINEERING
PrabhuB33
 
Vrinda store data analysis project using Excel
Vrinda store data analysis project using ExcelVrinda store data analysis project using Excel
Vrinda store data analysis project using Excel
SantuJana12
 
Cal Girls The Lalit Jaipur 8445551418 Khusi Top Class Girls Call Jaipur Avail...
Cal Girls The Lalit Jaipur 8445551418 Khusi Top Class Girls Call Jaipur Avail...Cal Girls The Lalit Jaipur 8445551418 Khusi Top Class Girls Call Jaipur Avail...
Cal Girls The Lalit Jaipur 8445551418 Khusi Top Class Girls Call Jaipur Avail...
deepikakumaridk25
 
Aws MLOps Interview Questions with answers
Aws MLOps Interview Questions  with answersAws MLOps Interview Questions  with answers
Aws MLOps Interview Questions with answers
Sathiakumar Chandr
 
Training on CSPro and step by steps.pptx
Training on CSPro and step by steps.pptxTraining on CSPro and step by steps.pptx
Training on CSPro and step by steps.pptx
lenjisoHussein
 
DESIGN AND DEVELOPMENT OF AUTO OXYGEN CONCENTRATOR WITH SOS ALERT FOR HIKING ...
DESIGN AND DEVELOPMENT OF AUTO OXYGEN CONCENTRATOR WITH SOS ALERT FOR HIKING ...DESIGN AND DEVELOPMENT OF AUTO OXYGEN CONCENTRATOR WITH SOS ALERT FOR HIKING ...
DESIGN AND DEVELOPMENT OF AUTO OXYGEN CONCENTRATOR WITH SOS ALERT FOR HIKING ...
JeevanKp7
 
Histology of Muscle types histology o.ppt
Histology of Muscle types histology o.pptHistology of Muscle types histology o.ppt
Histology of Muscle types histology o.ppt
SamanArshad11
 
Selcuk Topal Arbitrum Scientific Report.pdf
Selcuk Topal Arbitrum Scientific Report.pdfSelcuk Topal Arbitrum Scientific Report.pdf
Selcuk Topal Arbitrum Scientific Report.pdf
SelcukTOPAL2
 
Towards an Analysis-Ready, Cloud-Optimised service for FAIR fusion data
Towards an Analysis-Ready, Cloud-Optimised service for FAIR fusion dataTowards an Analysis-Ready, Cloud-Optimised service for FAIR fusion data
Towards an Analysis-Ready, Cloud-Optimised service for FAIR fusion data
Samuel Jackson
 
From Signals to Solutions: Effective Strategies for CDR Analysis in Fraud Det...
From Signals to Solutions: Effective Strategies for CDR Analysis in Fraud Det...From Signals to Solutions: Effective Strategies for CDR Analysis in Fraud Det...
From Signals to Solutions: Effective Strategies for CDR Analysis in Fraud Det...
Milind Agarwal
 
Technology used in Ott data analysis project
Technology used in Ott data analysis  projectTechnology used in Ott data analysis  project
Technology used in Ott data analysis project
49AkshitYadav
 
How AI is Revolutionizing Data Collection.pdf
How AI is Revolutionizing Data Collection.pdfHow AI is Revolutionizing Data Collection.pdf
How AI is Revolutionizing Data Collection.pdf
PromptCloud
 
Data Analytics for Decision Making By District 11 Solutions
Data Analytics for Decision Making By District 11 SolutionsData Analytics for Decision Making By District 11 Solutions
Data Analytics for Decision Making By District 11 Solutions
District 11 Solutions
 
393947940-The-Dell-EMC-PowerMax-Family-Overview.pdf
393947940-The-Dell-EMC-PowerMax-Family-Overview.pdf393947940-The-Dell-EMC-PowerMax-Family-Overview.pdf
393947940-The-Dell-EMC-PowerMax-Family-Overview.pdf
Ladislau5
 
Full Disclosure Board Policy.docx BRGY LICUMA
Full  Disclosure Board Policy.docx BRGY LICUMAFull  Disclosure Board Policy.docx BRGY LICUMA
Full Disclosure Board Policy.docx BRGY LICUMA
brgylicumaormoccity
 
Harnessing Wild and Untamed (Publicly Available) Data for the Cost efficient ...
Harnessing Wild and Untamed (Publicly Available) Data for the Cost efficient ...Harnessing Wild and Untamed (Publicly Available) Data for the Cost efficient ...
Harnessing Wild and Untamed (Publicly Available) Data for the Cost efficient ...
weiwchu
 
The Rise of Python in Finance,Automating Trading Strategies: _.pdf
The Rise of Python in Finance,Automating Trading Strategies: _.pdfThe Rise of Python in Finance,Automating Trading Strategies: _.pdf
The Rise of Python in Finance,Automating Trading Strategies: _.pdf
Riya Sen
 
Getting Started with Interactive Brokers API and Python.pdf
Getting Started with Interactive Brokers API and Python.pdfGetting Started with Interactive Brokers API and Python.pdf
Getting Started with Interactive Brokers API and Python.pdf
Riya Sen
 

Recently uploaded (20)

Data Storytelling Final Project for MBA 635
Data Storytelling Final Project for MBA 635Data Storytelling Final Project for MBA 635
Data Storytelling Final Project for MBA 635
 
Solution Manual for First Course in Abstract Algebra A, 8th Edition by John B...
Solution Manual for First Course in Abstract Algebra A, 8th Edition by John B...Solution Manual for First Course in Abstract Algebra A, 8th Edition by John B...
Solution Manual for First Course in Abstract Algebra A, 8th Edition by John B...
 
SOFTWARE ENGINEERING-UNIT-1SOFTWARE ENGINEERING
SOFTWARE ENGINEERING-UNIT-1SOFTWARE ENGINEERINGSOFTWARE ENGINEERING-UNIT-1SOFTWARE ENGINEERING
SOFTWARE ENGINEERING-UNIT-1SOFTWARE ENGINEERING
 
Vrinda store data analysis project using Excel
Vrinda store data analysis project using ExcelVrinda store data analysis project using Excel
Vrinda store data analysis project using Excel
 
Cal Girls The Lalit Jaipur 8445551418 Khusi Top Class Girls Call Jaipur Avail...
Cal Girls The Lalit Jaipur 8445551418 Khusi Top Class Girls Call Jaipur Avail...Cal Girls The Lalit Jaipur 8445551418 Khusi Top Class Girls Call Jaipur Avail...
Cal Girls The Lalit Jaipur 8445551418 Khusi Top Class Girls Call Jaipur Avail...
 
Aws MLOps Interview Questions with answers
Aws MLOps Interview Questions  with answersAws MLOps Interview Questions  with answers
Aws MLOps Interview Questions with answers
 
Training on CSPro and step by steps.pptx
Training on CSPro and step by steps.pptxTraining on CSPro and step by steps.pptx
Training on CSPro and step by steps.pptx
 
DESIGN AND DEVELOPMENT OF AUTO OXYGEN CONCENTRATOR WITH SOS ALERT FOR HIKING ...
DESIGN AND DEVELOPMENT OF AUTO OXYGEN CONCENTRATOR WITH SOS ALERT FOR HIKING ...DESIGN AND DEVELOPMENT OF AUTO OXYGEN CONCENTRATOR WITH SOS ALERT FOR HIKING ...
DESIGN AND DEVELOPMENT OF AUTO OXYGEN CONCENTRATOR WITH SOS ALERT FOR HIKING ...
 
Histology of Muscle types histology o.ppt
Histology of Muscle types histology o.pptHistology of Muscle types histology o.ppt
Histology of Muscle types histology o.ppt
 
Selcuk Topal Arbitrum Scientific Report.pdf
Selcuk Topal Arbitrum Scientific Report.pdfSelcuk Topal Arbitrum Scientific Report.pdf
Selcuk Topal Arbitrum Scientific Report.pdf
 
Towards an Analysis-Ready, Cloud-Optimised service for FAIR fusion data
Towards an Analysis-Ready, Cloud-Optimised service for FAIR fusion dataTowards an Analysis-Ready, Cloud-Optimised service for FAIR fusion data
Towards an Analysis-Ready, Cloud-Optimised service for FAIR fusion data
 
From Signals to Solutions: Effective Strategies for CDR Analysis in Fraud Det...
From Signals to Solutions: Effective Strategies for CDR Analysis in Fraud Det...From Signals to Solutions: Effective Strategies for CDR Analysis in Fraud Det...
From Signals to Solutions: Effective Strategies for CDR Analysis in Fraud Det...
 
Technology used in Ott data analysis project
Technology used in Ott data analysis  projectTechnology used in Ott data analysis  project
Technology used in Ott data analysis project
 
How AI is Revolutionizing Data Collection.pdf
How AI is Revolutionizing Data Collection.pdfHow AI is Revolutionizing Data Collection.pdf
How AI is Revolutionizing Data Collection.pdf
 
Data Analytics for Decision Making By District 11 Solutions
Data Analytics for Decision Making By District 11 SolutionsData Analytics for Decision Making By District 11 Solutions
Data Analytics for Decision Making By District 11 Solutions
 
393947940-The-Dell-EMC-PowerMax-Family-Overview.pdf
393947940-The-Dell-EMC-PowerMax-Family-Overview.pdf393947940-The-Dell-EMC-PowerMax-Family-Overview.pdf
393947940-The-Dell-EMC-PowerMax-Family-Overview.pdf
 
Full Disclosure Board Policy.docx BRGY LICUMA
Full  Disclosure Board Policy.docx BRGY LICUMAFull  Disclosure Board Policy.docx BRGY LICUMA
Full Disclosure Board Policy.docx BRGY LICUMA
 
Harnessing Wild and Untamed (Publicly Available) Data for the Cost efficient ...
Harnessing Wild and Untamed (Publicly Available) Data for the Cost efficient ...Harnessing Wild and Untamed (Publicly Available) Data for the Cost efficient ...
Harnessing Wild and Untamed (Publicly Available) Data for the Cost efficient ...
 
The Rise of Python in Finance,Automating Trading Strategies: _.pdf
The Rise of Python in Finance,Automating Trading Strategies: _.pdfThe Rise of Python in Finance,Automating Trading Strategies: _.pdf
The Rise of Python in Finance,Automating Trading Strategies: _.pdf
 
Getting Started with Interactive Brokers API and Python.pdf
Getting Started with Interactive Brokers API and Python.pdfGetting Started with Interactive Brokers API and Python.pdf
Getting Started with Interactive Brokers API and Python.pdf
 

Druid

  • 1. Druid in Production Dori Waldman - Big Data Lead Guy Shemer - Big Data Expert Alon Edelman - Big Data consultant
  • 2. ● Druid ○ Demo ○ What is druid and Why you need it ○ Other solutions ... ● Production pains and how it works : ○ cardinality ○ cache ○ dimension types (list/regular/map) ○ segment size ○ partition ○ monitoring and analyze hotspot ○ Query examples ○ lookups ● Interface ○ Pivot, Facet , superset ... ○ Druid-Sql (JDBC) ○ Rest Agenda
  • 4. Why ? ● fast (real-time) analytics on large time series data ○ MapReduce / Spark - are not design for real time queries. ○ MPP expensive / slow ● Just send raw data to druid → mention which attributes are the dimensions, metrics and how to aggregate the metrics → Druid will create cube (datasource) ○ Relational does not scale , we need fast queries on large data. ○ Key-value tables require table per predefined query , and we need dynamic queries (cube) http://static.druid.io/docs/druid.pdf ● We want to answer questions like: ○ #edits on the page Justin Bieber from males in San Francisco? ○ average #characters , added by people from Calgary over the last month? ○ arbitrary combination of dimensions to return with subsecond latencies.
  • 5. Row value can be dimension (~where in sql) or metric (measure) ● Dimensions are fields that can be filtered on or grouped by. ● Metrics are fields that can be aggregated. They are often stored as numbers but can also be stored as HyperLogLog sketches (approximate).. For example If Click is a dimension we can select this dimension and see how the data is splitted according to the selected value (might be better to convert as categories 0-20) If Click is a metric it will be a counter result like for how many clicks we have in Israel Dimension / Metric Country ApplicationId Clicks Israel 2 18 Israel 3 22 USA 80 19
  • 6. Other options ● Open source solution: ○ Pinot (https://github.com/linkedin/pinot) ○ clickHouse (https://clickhouse.yandex/) ○ Presto (https://prestodb.io/) https://medium.com/@leventov/comparison-of-the-open-source-olap-systems-for-big-data-clickhouse-druid-and-pinot-8e042a5ed1c7
  • 8. Components ● RealTime nodes - ingest and query event streams, events are immediately available to query, saved in cache and persist to global storage (s3/hdfs) “deepStorage” ● Historical nodes - load and serve the immutable blocks of data (segments) from deep storage, those are the main workers ● Broker nodes - query routers to historical and real-time nodes, communicate with ZK to understand where relevant segments are located ● Coordinator nodes - tell historical nodes to load new data, drop outdated data, replicate data, and Balance by move data
  • 9. Components ● Overlord node - Manages task distribution to middle managers. responsible for accepting tasks, coordinating task distribution, creating locks around tasks, and returning statuses to callers ● Middle manager node - Executes submitted tasks by forward slices of tasks to peons. In case druid runs in local mode, this part is redundant since overlord will also take this responsibility ● Peon - single task in a single JVM. Several peons may run on same node
  • 10. Components (Stream) ● Tranquility - ○ Ingest from kafka/http/samza/flink … ○ Will be out of life ○ Connects to the ZooKeeper of the kafka cluster ○ Can connect to several clusters and read from several topics for the same Druid data source. ○ Can't handle events after window closes ● Kafka-Indexing-Service ○ ingest from kafka only (kafka 10+) ○ Connects directly to Kafka’s brokers ○ Is able to connect to one cluster and one topic for one druid data source ○ Indexer manage its tasks better, use checkpoint (~exactly once) ○ Can update events for old segments (no window) ○ can be spot instance (for other nodes its less recommended)
  • 14. Batch Ingestion Druid support : JSON CSV AVRO PARQUET ORC Support: ● Values ● multiValue (array) - each item in the list will be explode to row ● maps (new feature)
  • 15. Batch Ingestion Indexing Task Types ● index_hadoop (with EMR) ○ Hadoop-based batch ingestion. using Hadoop/EMR cluster to perform data processing and ingestion ● Index (No EMR) ○ For small amount of data , task execute within the indexing service without external hadoop resources
  • 16. Batch Ingestion Input source for batch indexing ● local ○ For POC ● Static (S3/ HDFS etc..) ○ Ingesting from your raw data ○ Support also Parquet ○ Can be mapped dynamically to specific date ● Druid’s Deep Storage ○ Use segments from one datasource from deep storage and transform them to another datasource, clean dimensions, change granularity etc.. "inputSpec" : { "type" : "static", "paths" : "/MyDirectory/example/wikipedia_data.json" } "inputSpec": { "type": "static", "paths": "s3n://prod/raw/2018-01-01/00/, s3n://staging/raw/2018-01-01/00/", "filePattern": ".gz" } "inputSpec": { "type": "dataSource", "ingestionSpec": { "dataSource" : "Hourly", "intervals" : ["2017-11-06T00:00:00.000Z/2017-11-07T00:00:00.000Z"] }
  • 18. Lookups ● Purpose : replace dimensions values , for example replace “1” with “New York City” ● in case the mapping is 1:1 an optimization (“injective”:true) should be used, it will replace the value on the query result and not on the query input ● Lookups has no history (if value of 1 was “new york” and it was changed to “new your city” the old value will not appear in the query result. ● Very small lookups (count of keys on the order of a few dozen to a few hundred) can be passed at query time as a "map" lookup ● Usually you will use global cached lookups from DB / file / kafka
  • 20. Query: TopN ● TopN ○ grouped by single dimension, sort (order) according to the metric (~ “group by” one dimension + order ) ○ TopNs are approximate in that each node will rank their top K results and only return those top K results to the broker ○ To get exact result use groupBy query and sort the results (better to avoid)
  • 21. Query: TopN ● TopN Hell- in the Pivot Pivot use nested TopN’s (filter and topN per row) Try to reduce number of unnecessary topN queries
  • 22. Query: GroupBy GroupBy ○ Grouped by multiple dimensions. ○ Unlike TopN, can use ‘having’ conditions over aggregated data. Druid vision is to replace timeseries and topN with groupBy advance query
  • 23. Query: TimeSeries ● Timeseries ○ grouped by time dimension only (“no dimensions) ○ Timeseries query will generally be faster than groupBy as it taking advantage of the fact that segments are already sorted on time and does not need to use a hash table for merging.
  • 24. Query: SQL ● Druid SQL ○ Translates SQL into native Druid queries on the query broker ■ using JSON over HTTP by posting to the endpoint /druid/v2/sql/ ■ SQL queries using the Avatica JDBC driver.
  • 25. Query: TimeBoundary / MetaData ● Time boundary ○ Return the earliest and latest data points of a data set ● Segment metadata ○ Per-segment information: ■ dimensions cardinality, ■ min/max value in dimension ■ number of rows ● DataSource metadata ○ ...
  • 26. Other Queries... ● Select / Scan / Search ○ select - supports pagination, all data is loaded to memory ○ scan - return result in streaming mode ○ search - returns dimension values which match a search criteria. The biggest difference between select and scan is that, scan query doesn't retain all rows in memory before rows can be returned to client.
  • 27. Query Performance ● Query with metrics Metric calculation is done in real time per metric meaning doing sum of impression and later sum of impressions and sum of clicks will double the metric calculation time (think about list dimension...)
  • 30. ● Index 5T row daily from 3 different resources (s3 / kafka) ● 40 dimensions, 10 metrics ● Datasource (table) should be updated every 3 hours ● Query latency ~10 second for query on one dimension , 3 month range ○ Some dimensions are list … ○ Some dimensions use lookups Requirements
  • 31. Work in scale ● We started with 14 dimensions (no lists) → for 8 month druid answer all requirements ● We added 20 more dimensions (with list) → druid query time was slow ...
  • 32. ● Hardware : ○ 30 nodes(i3.8xlarge), each node manage historical and middleManager service ○ 2 nodes (m4.2xlarge) , each node manage coordinator and overload services ○ 11 nodes (c4.2xlarge), each node manage tranquility service ○ 2 nodes (i3.8xlarge), each node manage broker service ■ (1 broker : 10 historical) ○ Memcached : 3 nodes (cache.r3.8xlarge), version: 1.4.34 Hardware
  • 33. Data cleanup ● Cleanup reduce cardinality (replace it with dummy value) ● Its all about reducing number of rows in the datasource ○ Druid saves the data in columnar storage but in order to get better performance the cleanup process reduces #rows (although the query is on 3 columns it needs to read all items in the column)
  • 34. Data cleanup ● The dimensions correlation is important. ○ lets say we have one dimension which is city with 2000 unique cities ■ Adding gender dimension will double #rows (assume in our row data we have both male/female per city) ■ Adding country (although we have 200 unique countries) will not impact the same (cartesian product) as there is a relation between city and county of 1:M. ● better to reduce non related dimensions like country and age
  • 35. Data cleanup ○ Use timeseries query with “count” aggregation (~ count(*) in druid Sql) to measure your cleanup benefit ○ you can also use estimator with cardinality aggregation ○ if you want to estimate without doing cleanup you can use virtualColumns (filter out specific values) with byRow cardinality estimator
  • 36. segments ● Shard size should be balanced between disk optimization 500M-1.5G and cpu optimization (core per segment during query), take list in this calculation … Shard minimum size should be 100M ● POC - convert list to bitwise vector
  • 37. Partition ● Partition type ○ By default, druid partitions the data according to timestamp In addition you need to specify hashed/ single dimension partition ■ partition may result with unbalanced segments ■ The default of hashed partition using all dimensions ■ Hashed partitioning is recommended in most cases, as it will improve indexing performance and create more uniformly sized data segments relative to single-dimension partitioning. ■ single-dimension partition may be preferred in context of multi tenancy use cases. ■ Might want to avoid default hashed in case of long tail "partitionsSpec": { "type": "hashed", "targetPartitionSize": "4500000" } "partitionsSpec": { "type": "dimension", "targetPartitionSize": "10000000", "partitionDimension": publisherId" } "partitionsSpec": { "type": "hashed", "numShards": "12", "partitionDimensions": ["publisherId"] }
  • 38. Cache ● Cache : ○ hybrid ■ L1-caffein (local), ■ L2-memcached (global) when segment move between machines , caffein cache will be invalidate for those segments ○ warm Cache for popular queries (~300ms) ○ Cache is saved per segment and date , cache key contains the dimensions ,metrics , filters ○ TopN threshold are part of the key : 0-1000, 1001, 1002 … ○ Cache in the historical nodes not broker in order to merge less data in the broker side
  • 39. Cache ● Cache : ○ Lookup has pollPeriod meaning if its set to 1 day then cache will be invalid (no eviction) every day even if lookup was not updated (tscolumn), since Imply 2.4.6 this issue should be fixed by setting injective=true in the lookup configuration meaning lookup is not part of the cache key anymore its a post-aggregation action in the brokers. ■ increase lookup pooling period + hard set injective=true in the query is workaround till 2.4.6 ○ rebuild segment (~new) cause the cache to be invalidate
  • 40. Cache Monitoring All segments (507) are scanned not from the cache Broker logs
  • 41. ● Production issue : ○ Cluster was slow ■ doing rebalance all the time ■ nodes disappear , no crash in the nodes ■ we found that during this time GC took long time , and in the log we saw ZK disconnect-connect ○ We increased ZK connection timeout ○ Solution was to decrease historical memory (reduce GC time)
  • 42. Monitoring / Debug Fix hotspot by increase #segment to move till data is balanced Statsd emitter does not send all metrics , use another (clarity / kafka)
  • 43. Druid Pattern ● Two data sources ○ small (less rows and dimensions) ○ Large (all data) , query with filter only
  • 44. Extra ● Load rules are used to manage which data is available to druid, for example we can set it to save only last month data and drop old data every day ● Priority - druid support query by priority ● Avoid Javascript extension (post aggregation function)