SlideShare a Scribd company logo
1 of 34
Experimentation Platform
on Hadoop
Tony Ng, Director, Data Services
Padma Gopal, Manager, Experimentation
Agenda
 Experimentation 101
 Reporting Work flow
 Why Hadoop?
 Framework Architecture
 Challenges & Learnings
 Q & A
Experimentation 101
• What is A/B Testing?
• Why is it important?
• Intuition vs. Reality
• eBay Wins
What is A/B Testing?
• A/B Testing is comparing two versions of a page or process to see which one
performs better
• Variations could be: UI Components, Content, Algorithms etc.
• Measures: Financial metrics, Click rate, Conversion rate etc.
Control - Current design Treatment - Variations of current design
EP – Hadoop Summit 2015 4
How is A/B Testing is done?
EP – Hadoop Summit 2015 5
Why is it important?
• Intuition vs. Reality
–Intuition especially on novel ideas should be backed up by data.
–Demographics and preferences vary
• Data Driven; not based on opinion
• Reduce risk
EP – Hadoop Summit 2015 6
Increased prominence of BIN button compared to Watch, leads to
faster checkouts.
EP – Hadoop Summit 2015 7
Merch placements perform much better when title and price
information is provided upfront.
EP – Hadoop Summit 2015 8
New sign-in design effectively pushed more new users to use
guest checkout
9EP – Hadoop Summit 2015
10
What do we support?
EP – Hadoop Summit 2015
Experimentation Reporting
• How does EP work?
• Work Flow
• DW Challenges
Experiment Lifecycle
EP – Hadoop Summit 2015 12
EP – Hadoop Summit 2015 13
User Behavior &
Transactional Data
Experiment
Metadata
Detail Intermediate Summaries
4 Billion Rows
4 TB
User1 Homepage
User1 Search for IPhone6
User1 View Item1
User2 Search for Coach bag
User2 View Item2
User2 Bid
Treatment 2 User1 Homepage
Treatment 1 User1 Search for IPhone6
Treatment 2 User1 Search for IPhone6
Treatment 1 User1 View Item 1
Treatment 2 User1 View Item 1
Treatment 1 User2 Search for Coach bag
Treatment 2 User2 Search for Coach bag
Treatment 1 100+ Metrics
Treatment 1 20 X Dimensions
Treatment 1 10 Metric Insights
Treatment 2 100+ Metrics
Treatment 2 20 X Dimensions
Treatment 2 10 Data Insights
EP – Hadoop Summit 2015 14
Transactional Metrics
Activity Metrics
Acquisition Metrics
AD Metrics
Email Metrics
Seller Metrics
Engagement metrics
Absolute - Actual number/counts
Normalized - Weighted mean (by GUID/UID)
Lift - Difference between treatment and control
Standard Deviation - Weighted standard deviation
Confidence Interval – Range within which treatment
effect is likely to lie
P-values – Statistically significance
Outlier capped – Trim tail values
Post Stratified – Adjustment method to reduce
variance
DATA INSIGHTS
Daily
Weekly
Cumulative
Browser
OS
Device
Site/Country
Category
Segment
Geo
Hadoop Migration
• Why Hadoop
• Tech Stack
• Architecture Overview
EP – Hadoop Summit 2015 16
Why Hadoop?
• Design & Development flexibility
• Store large amounts of data without the schemas constraints
• System to support complex data transformation logic
• Code base reduction
• Configurability
• Code not tied to environment & easier to share
• Support for complex structures
Scheduler/Client
EP – Hadoop Summit 2015 17
Physical Architecture
Hadoop Cluster
Job
Workflow
RDBMS
ETL
Bridge
Agent
BI
&
PresentationmySQL DW
User
Behavior
Data
1
2
43
5
Hive Scoobi Spark (poc)
AVRO ORC
EP – Hadoop Summit 2015 18
Tech Stack - Scoobi
•Scoobi
– Written in Scala, a functional programming language
– Supports Object Oriented Designs
– Abstraction of MR Framework code to lower
– Portability of typical dataset operations like map, flatMap, filter, groupBy, sort, orderBy, partition
– DList (Distributed Lists): Jobs are submitted as a series of “steps” representing granular MR jobs.
– Enables developers to write a more concise code compared to Java MR code.
EP – Hadoop Summit 2015 19
Word Count in Java M/R.
import java.io.IOException;
import java.util.*;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.conf.*;
import org.apache.hadoop.io.*;
import org.apache.hadoop.mapreduce.*;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
public class WordCount {
public static class Map extends Mapper<LongWritable, Text, Text,
IntWritable> {
private final static IntWritable one = new IntWritable(1);
private Text word = new Text();
public void map(LongWritable key, Text value, Context context)
throws IOException, InterruptedException {
String line = value.toString();
StringTokenizer tokenizer = new StringTokenizer(line);
while (tokenizer.hasMoreTokens()) {
word.set(tokenizer.nextToken());
context.write(word, one);
}
}
}
public static class Reduce extends Reducer<Text, IntWritable, Text,
IntWritable> {
public void reduce(Text key, Iterable<IntWritable> values, Context
context)
throws IOException, InterruptedException {
int sum = 0;
for (IntWritable val : values) {
sum += val.get();
}
context.write(key, new IntWritable(sum));
}
}
public static void main(String[] args) throws Exception {
Configuration conf = new Configuration();
Job job = new Job(conf, "wordcount");
job.setOutputKeyClass(Text.class);
job.setOutputValueClass(IntWritable.class);
job.setMapperClass(Map.class);
job.setReducerClass(Reduce.class);
job.setInputFormatClass(TextInputFormat.class);
job.setOutputFormatClass(TextOutputFormat.class);
FileInputFormat.addInputPath(job, new Path(args[0]));
FileOutputFormat.setOutputPath(job,new Path(args[1]));
job.waitForCompletion(true);
}
}
EP – Hadoop Summit 2015 20
Word Count in Scoobi
import Scoobi._, Reduction._
val lines = fromTextFile("hdfs://in/...")
val counts = lines.mapFlatten(_.split(" "))
.map(word => (word, 1))
.groupByKey
.combine(Sum.int)
counts.toTextFile("hdfs://out/...",
overwrite=true).persist(ScoobiConfiguration())
EP – Hadoop Summit 2015 21
Tech Stack - File Format
• Avro
– Supports rich and complex data structures such as Maps, Unions
– Self-Describing data files enabling portability (Schema co-exists with data)
– Supports schema dynamicity using Generic Records
– Supports backward compatibility for data files w.r.t schema changes
• ORC (Optimized Row Columnar)
– A single file as the output of each task, which reduces the NameNode's load
– Metadata stored using Protocol Buffers, which allows addition and removal of fields
– Better performance of queries (bound the amount of memory needed for reading or writing)
– Light-weight indexes stored within the file
EP – Hadoop Summit 2015 22
Tech Stack - Hive
• Efficient Joins for large datasets.
• UDF for use cases like median and percentile calculations.
• Hive Optimizer Joins:
- Smaller is loaded into memory as a hash table and the larger is scanned
- Map joins are automatically picked up by the optimizer.
• Ad-hoc Analysis, Data Reconciliation use-cases and Testing.
EP – Hadoop Summit 2015 23
Fun Facts of EP Processing
• We read more than 200 TB of data for processing daily.
• We run 350 M/R jobs daily.
• We perform more than 30 joins using M/R & Hive, including the ones with heavy data skew.
• We use 40 TB of YARN memory at peak time on a 170 TB Hadoop cluster.
• We can run 150+ concurrent experiments daily.
• Report generation takes around 18 hours.
24
Logical Architecture
EP – Hadoop Summit 2015
EP Reporting Services
Detail Intermediate 1 Intermediate 2 Summary
Configuration
Filters Data Providers Processors
Calculators Metric Providers
Output
ColumnsMetricsDimensions
Framework
Components
Reporting
Context
Cache
Util/Helpers
Command
Line
Input/Output
Conduit
Ancillary
Services
Alerts
Shell
Scripts
Processed
Data Store
Tools
Logging &
Monitoring
CHALLENGES &
LEARNINGS
• Joins
• Job Optimization
• Data Skew
25EP – Hadoop Summit 2015
EP – Hadoop Summit 2015 26
Key Challenges
•Performance
– Job runtimes are subject to SLA & heavily tied to
resources
•Data Skew (Long tail data distribution)
– May cause unrecoverable runtime failures
– Poor performance
•Joins, Combiner
•Job Resiliency
– Auto remediation
– Alerts and Monitoring
EP – Hadoop Summit 2015 27
Solution to Key Challenge - Performance
– Tuned the Hadoop job parameters – a few of them are listed below
• -Dmapreduce.input.fileinputformat.split.minsize and -Dmapreduce.input.fileinputformat.split.maxsize
– Job run times were reduced in the range of 9% to 35%
• -Dscoobi.mapreduce.reducers.bytesperreducer
– Adjusting this parameter helped optimize the number of reducers to use. Job run times were
reduced to the extent of 50% in some cases
• -Dscoobi.concurrentjobs
– Setting this parameter to true enables multiple steps of a scoobi job to run concurrently
• -Dmapreduce.reduce.memory.mb
– Tuning this parameter helped relieving memory pressure
EP – Hadoop Summit 2015 28
Solution to Key Challenge - Performance
– Implement Data cache for objects
• Achieved cache hit ratio of over 99% per job
• Runtime performance improved in the range of 18% to 39% depending on the job
– Redesign/Refactor Jobs and Job Schedules
• Extracted logic from existing jobs into their own jobs
• Job workflow optimization for better parallelism
– Dedicated Hadoop queue with more than 50 TB of YARN memory.
• Shared Hadoop cluster resulted in long waiting times, dedicated queue solved the problem of
resource crunch.
Joins
– Data skew in one or both datasets
 Scoobi block join divides the skewed data into blocks and joins the data one block at a time.
– Multiple joins in a process
 Rewrote a process, which needed join with 11 datasets whose size varied from 49 TB to a few mega
byte, in hive, as this process was taking 6+ hours in Scoobi and reduced the time to 3 hours in hive.
– Other join solutions
 Also looked into Hive’s bucket join, but the cost to sort and bucket the datasets was more than regular
join.
EP – Hadoop Summit 2015 29
EP – Hadoop Summit 2015 30
Combiner
To relieve Reducer memory pressure and prevent OOM
Solution – Emit part-values of the complete operation for the same key using Combiners
– Calculating Mean
• Mean = ( X1 + X2 + X3 …. Xn )/ (1 + 1 + 1 + 1 … n)
• formula is composed of 2 parts and mapper emits 2 part values combining records for the
same key.
• Reducer receives way fewer records after combining and applies the two parts from each
mapper into the actual mean formula.
• Concept can be applied to other complex formula such as Variance, as long as the formula
can be reduced to parts that are commutative and associative.
Job Resiliency
– Auto-remediation
• Auto-restart in case of job failure due to intermittent cluster issues
- Monitoring & Alerting for Hadoop jobs
• Continuous monitoring and email alert generated when a long-running job or failure detected
- Monitoring & Alerting for Data quality
• Daily monitoring of data trend set up for key metrics and email Alert on any anomaly or violations detected
- Recon scripts
• Checks and alerts setup for intermediate data
- Daily data backup
• Daily data back up with distcp to a secondary cluster and ability to restore
EP – Hadoop Summit 2015 31
Next - Evaluate Spark
Current Problems
- Data processing through Map Reduce is slow for a complex DAG as data is persisted to disk
at each step. It is Multiple stages in pipeline are chained together making the overall process
very complex.
- Massive Joins against very large datasets are slow.
- Expressing every complicated business logic into Hadoop Map Reduce is a problem.
Alternatives
- Apache Spark has wide adoption, expressive, industry backing and thriving community
support.
- Apache spark has 10x to 100x speed improvements in comparison to traditional M/R jobs.
EP – Hadoop Summit 2015 32
Summary
• Hadoop is ideal for large data processing and provides a
highly scalable storage platform.
• Hadoop eco-system is still evolving and have to face the
issues around the software which is still
underdevelopment.
• Moving to Hadoop helped to free up huge capacity in DW
for deep dive analysis.
• Huge cost reduction for business like us with exploding
data sets.
EP – Hadoop Summit 2015 33
Q & A

More Related Content

What's hot

Performance analysis of MongoDB and HBase
Performance analysis of MongoDB and HBasePerformance analysis of MongoDB and HBase
Performance analysis of MongoDB and HBaseSindhujanDhayalan
 
Data Science Applications | Data Science For Beginners | Data Science Trainin...
Data Science Applications | Data Science For Beginners | Data Science Trainin...Data Science Applications | Data Science For Beginners | Data Science Trainin...
Data Science Applications | Data Science For Beginners | Data Science Trainin...Edureka!
 
Big Data: An Overview
Big Data: An OverviewBig Data: An Overview
Big Data: An OverviewC. Scyphers
 
Productionizing Machine Learning Pipelines with Databricks and Azure ML
Productionizing Machine Learning Pipelines with Databricks and Azure MLProductionizing Machine Learning Pipelines with Databricks and Azure ML
Productionizing Machine Learning Pipelines with Databricks and Azure MLDatabricks
 
Introducing Databricks Delta
Introducing Databricks DeltaIntroducing Databricks Delta
Introducing Databricks DeltaDatabricks
 
Web 3.0 presentation
Web 3.0 presentationWeb 3.0 presentation
Web 3.0 presentationmcdowellmike
 
Databricks Platform.pptx
Databricks Platform.pptxDatabricks Platform.pptx
Databricks Platform.pptxAlex Ivy
 
Databricks for Dummies
Databricks for DummiesDatabricks for Dummies
Databricks for DummiesRodney Joyce
 
Moving to Databricks & Delta
Moving to Databricks & DeltaMoving to Databricks & Delta
Moving to Databricks & DeltaDatabricks
 
Dataiku Data Science Studio (datasheet)
Dataiku Data Science Studio (datasheet)Dataiku Data Science Studio (datasheet)
Dataiku Data Science Studio (datasheet)John Cann
 
What Is Hadoop | Hadoop Tutorial For Beginners | Edureka
What Is Hadoop | Hadoop Tutorial For Beginners | EdurekaWhat Is Hadoop | Hadoop Tutorial For Beginners | Edureka
What Is Hadoop | Hadoop Tutorial For Beginners | EdurekaEdureka!
 
Big Data Architecture
Big Data ArchitectureBig Data Architecture
Big Data ArchitectureGuido Schmutz
 
Virtual Reality and Healthcare - The Past, the Present, and the Future
Virtual Reality and Healthcare -   The Past, the Present, and the FutureVirtual Reality and Healthcare -   The Past, the Present, and the Future
Virtual Reality and Healthcare - The Past, the Present, and the FutureStanford University
 
AI and Healthcare 2023.pdf
AI and Healthcare 2023.pdfAI and Healthcare 2023.pdf
AI and Healthcare 2023.pdfKR_Barker
 

What's hot (20)

Performance analysis of MongoDB and HBase
Performance analysis of MongoDB and HBasePerformance analysis of MongoDB and HBase
Performance analysis of MongoDB and HBase
 
Data Science Applications | Data Science For Beginners | Data Science Trainin...
Data Science Applications | Data Science For Beginners | Data Science Trainin...Data Science Applications | Data Science For Beginners | Data Science Trainin...
Data Science Applications | Data Science For Beginners | Data Science Trainin...
 
Big data analytics
Big data analyticsBig data analytics
Big data analytics
 
Big Data: An Overview
Big Data: An OverviewBig Data: An Overview
Big Data: An Overview
 
Productionizing Machine Learning Pipelines with Databricks and Azure ML
Productionizing Machine Learning Pipelines with Databricks and Azure MLProductionizing Machine Learning Pipelines with Databricks and Azure ML
Productionizing Machine Learning Pipelines with Databricks and Azure ML
 
Big Data ppt
Big Data pptBig Data ppt
Big Data ppt
 
Introducing Databricks Delta
Introducing Databricks DeltaIntroducing Databricks Delta
Introducing Databricks Delta
 
Web 3.0 presentation
Web 3.0 presentationWeb 3.0 presentation
Web 3.0 presentation
 
Databricks Platform.pptx
Databricks Platform.pptxDatabricks Platform.pptx
Databricks Platform.pptx
 
Databricks for Dummies
Databricks for DummiesDatabricks for Dummies
Databricks for Dummies
 
Fog computing in IoT
Fog computing in IoTFog computing in IoT
Fog computing in IoT
 
Moving to Databricks & Delta
Moving to Databricks & DeltaMoving to Databricks & Delta
Moving to Databricks & Delta
 
Web 3.0
Web 3.0Web 3.0
Web 3.0
 
The Computing Continuum.pdf
The Computing Continuum.pdfThe Computing Continuum.pdf
The Computing Continuum.pdf
 
Dataiku Data Science Studio (datasheet)
Dataiku Data Science Studio (datasheet)Dataiku Data Science Studio (datasheet)
Dataiku Data Science Studio (datasheet)
 
Hadoop and HBase @eBay
Hadoop and HBase @eBayHadoop and HBase @eBay
Hadoop and HBase @eBay
 
What Is Hadoop | Hadoop Tutorial For Beginners | Edureka
What Is Hadoop | Hadoop Tutorial For Beginners | EdurekaWhat Is Hadoop | Hadoop Tutorial For Beginners | Edureka
What Is Hadoop | Hadoop Tutorial For Beginners | Edureka
 
Big Data Architecture
Big Data ArchitectureBig Data Architecture
Big Data Architecture
 
Virtual Reality and Healthcare - The Past, the Present, and the Future
Virtual Reality and Healthcare -   The Past, the Present, and the FutureVirtual Reality and Healthcare -   The Past, the Present, and the Future
Virtual Reality and Healthcare - The Past, the Present, and the Future
 
AI and Healthcare 2023.pdf
AI and Healthcare 2023.pdfAI and Healthcare 2023.pdf
AI and Healthcare 2023.pdf
 

Viewers also liked

Pulsar: Real-time Analytics at Scale with Kafka, Kylin and Druid
Pulsar: Real-time Analytics at Scale with Kafka, Kylin and DruidPulsar: Real-time Analytics at Scale with Kafka, Kylin and Druid
Pulsar: Real-time Analytics at Scale with Kafka, Kylin and DruidTony Ng
 
Apache Kylin: OLAP Engine on Hadoop - Tech Deep Dive
Apache Kylin: OLAP Engine on Hadoop - Tech Deep DiveApache Kylin: OLAP Engine on Hadoop - Tech Deep Dive
Apache Kylin: OLAP Engine on Hadoop - Tech Deep DiveXu Jiang
 
Building an experimentation framework
Building an experimentation frameworkBuilding an experimentation framework
Building an experimentation frameworkzsqr
 
Case Study: Realtime Analytics with Druid
Case Study: Realtime Analytics with DruidCase Study: Realtime Analytics with Druid
Case Study: Realtime Analytics with DruidSalil Kalia
 
ElasticSearch: Distributed Multitenant NoSQL Datastore and Search Engine
ElasticSearch: Distributed Multitenant NoSQL Datastore and Search EngineElasticSearch: Distributed Multitenant NoSQL Datastore and Search Engine
ElasticSearch: Distributed Multitenant NoSQL Datastore and Search EngineDaniel N
 
Eventually Elasticsearch: Eventual Consistency in the Real World
Eventually Elasticsearch: Eventual Consistency in the Real WorldEventually Elasticsearch: Eventual Consistency in the Real World
Eventually Elasticsearch: Eventual Consistency in the Real WorldBeyondTrees
 
Pinot: Realtime Distributed OLAP datastore
Pinot: Realtime Distributed OLAP datastorePinot: Realtime Distributed OLAP datastore
Pinot: Realtime Distributed OLAP datastoreKishore Gopalakrishna
 
Big Data Paris 2015 - Cassandra chez Chronopost
Big Data Paris 2015 - Cassandra chez ChronopostBig Data Paris 2015 - Cassandra chez Chronopost
Big Data Paris 2015 - Cassandra chez ChronopostAlexander DEJANOVSKI
 
Kylin OLAP Engine Tour
Kylin OLAP Engine TourKylin OLAP Engine Tour
Kylin OLAP Engine TourLuke Han
 
Architecture Big Data open source S.M.A.C.K
Architecture Big Data open source S.M.A.C.KArchitecture Big Data open source S.M.A.C.K
Architecture Big Data open source S.M.A.C.KJulien Anguenot
 
Experimentation Platform at Netflix
Experimentation Platform at NetflixExperimentation Platform at Netflix
Experimentation Platform at NetflixSteve Urban
 
IS OLAP DEAD IN THE AGE OF BIG DATA?
IS OLAP DEAD IN THE AGE OF BIG DATA?IS OLAP DEAD IN THE AGE OF BIG DATA?
IS OLAP DEAD IN THE AGE OF BIG DATA?DataWorks Summit
 
Aggregated queries with Druid on terrabytes and petabytes of data
Aggregated queries with Druid on terrabytes and petabytes of dataAggregated queries with Druid on terrabytes and petabytes of data
Aggregated queries with Druid on terrabytes and petabytes of dataRostislav Pashuto
 
Apache Kylin – Cubes on Hadoop
Apache Kylin – Cubes on HadoopApache Kylin – Cubes on Hadoop
Apache Kylin – Cubes on HadoopDataWorks Summit
 
Devoxx 2016 - Dropwizard : Création de services REST production-ready
Devoxx 2016 - Dropwizard : Création de services REST production-readyDevoxx 2016 - Dropwizard : Création de services REST production-ready
Devoxx 2016 - Dropwizard : Création de services REST production-readyAlexander DEJANOVSKI
 
Streaming Analytics with Spark, Kafka, Cassandra and Akka by Helena Edelson
Streaming Analytics with Spark, Kafka, Cassandra and Akka by Helena EdelsonStreaming Analytics with Spark, Kafka, Cassandra and Akka by Helena Edelson
Streaming Analytics with Spark, Kafka, Cassandra and Akka by Helena EdelsonSpark Summit
 
Design cube in Apache Kylin
Design cube in Apache KylinDesign cube in Apache Kylin
Design cube in Apache KylinYang Li
 
Requêtes multi-critères avec Cassandra
Requêtes multi-critères avec CassandraRequêtes multi-critères avec Cassandra
Requêtes multi-critères avec CassandraJulien Dubois
 

Viewers also liked (20)

Pulsar: Real-time Analytics at Scale with Kafka, Kylin and Druid
Pulsar: Real-time Analytics at Scale with Kafka, Kylin and DruidPulsar: Real-time Analytics at Scale with Kafka, Kylin and Druid
Pulsar: Real-time Analytics at Scale with Kafka, Kylin and Druid
 
Apache Kylin: OLAP Engine on Hadoop - Tech Deep Dive
Apache Kylin: OLAP Engine on Hadoop - Tech Deep DiveApache Kylin: OLAP Engine on Hadoop - Tech Deep Dive
Apache Kylin: OLAP Engine on Hadoop - Tech Deep Dive
 
Building an experimentation framework
Building an experimentation frameworkBuilding an experimentation framework
Building an experimentation framework
 
Case Study: Realtime Analytics with Druid
Case Study: Realtime Analytics with DruidCase Study: Realtime Analytics with Druid
Case Study: Realtime Analytics with Druid
 
The Evolution of Apache Kylin
The Evolution of Apache KylinThe Evolution of Apache Kylin
The Evolution of Apache Kylin
 
Scalable Real-time analytics using Druid
Scalable Real-time analytics using DruidScalable Real-time analytics using Druid
Scalable Real-time analytics using Druid
 
ElasticSearch: Distributed Multitenant NoSQL Datastore and Search Engine
ElasticSearch: Distributed Multitenant NoSQL Datastore and Search EngineElasticSearch: Distributed Multitenant NoSQL Datastore and Search Engine
ElasticSearch: Distributed Multitenant NoSQL Datastore and Search Engine
 
Eventually Elasticsearch: Eventual Consistency in the Real World
Eventually Elasticsearch: Eventual Consistency in the Real WorldEventually Elasticsearch: Eventual Consistency in the Real World
Eventually Elasticsearch: Eventual Consistency in the Real World
 
Pinot: Realtime Distributed OLAP datastore
Pinot: Realtime Distributed OLAP datastorePinot: Realtime Distributed OLAP datastore
Pinot: Realtime Distributed OLAP datastore
 
Big Data Paris 2015 - Cassandra chez Chronopost
Big Data Paris 2015 - Cassandra chez ChronopostBig Data Paris 2015 - Cassandra chez Chronopost
Big Data Paris 2015 - Cassandra chez Chronopost
 
Kylin OLAP Engine Tour
Kylin OLAP Engine TourKylin OLAP Engine Tour
Kylin OLAP Engine Tour
 
Architecture Big Data open source S.M.A.C.K
Architecture Big Data open source S.M.A.C.KArchitecture Big Data open source S.M.A.C.K
Architecture Big Data open source S.M.A.C.K
 
Experimentation Platform at Netflix
Experimentation Platform at NetflixExperimentation Platform at Netflix
Experimentation Platform at Netflix
 
IS OLAP DEAD IN THE AGE OF BIG DATA?
IS OLAP DEAD IN THE AGE OF BIG DATA?IS OLAP DEAD IN THE AGE OF BIG DATA?
IS OLAP DEAD IN THE AGE OF BIG DATA?
 
Aggregated queries with Druid on terrabytes and petabytes of data
Aggregated queries with Druid on terrabytes and petabytes of dataAggregated queries with Druid on terrabytes and petabytes of data
Aggregated queries with Druid on terrabytes and petabytes of data
 
Apache Kylin – Cubes on Hadoop
Apache Kylin – Cubes on HadoopApache Kylin – Cubes on Hadoop
Apache Kylin – Cubes on Hadoop
 
Devoxx 2016 - Dropwizard : Création de services REST production-ready
Devoxx 2016 - Dropwizard : Création de services REST production-readyDevoxx 2016 - Dropwizard : Création de services REST production-ready
Devoxx 2016 - Dropwizard : Création de services REST production-ready
 
Streaming Analytics with Spark, Kafka, Cassandra and Akka by Helena Edelson
Streaming Analytics with Spark, Kafka, Cassandra and Akka by Helena EdelsonStreaming Analytics with Spark, Kafka, Cassandra and Akka by Helena Edelson
Streaming Analytics with Spark, Kafka, Cassandra and Akka by Helena Edelson
 
Design cube in Apache Kylin
Design cube in Apache KylinDesign cube in Apache Kylin
Design cube in Apache Kylin
 
Requêtes multi-critères avec Cassandra
Requêtes multi-critères avec CassandraRequêtes multi-critères avec Cassandra
Requêtes multi-critères avec Cassandra
 

Similar to eBay Experimentation Platform on Hadoop

Gobblin' Big Data With Ease @ QConSF 2014
Gobblin' Big Data With Ease @ QConSF 2014Gobblin' Big Data With Ease @ QConSF 2014
Gobblin' Big Data With Ease @ QConSF 2014Lin Qiao
 
Hadoop and the Data Warehouse: Point/Counter Point
Hadoop and the Data Warehouse: Point/Counter PointHadoop and the Data Warehouse: Point/Counter Point
Hadoop and the Data Warehouse: Point/Counter PointInside Analysis
 
Architecting the Future of Big Data and Search
Architecting the Future of Big Data and SearchArchitecting the Future of Big Data and Search
Architecting the Future of Big Data and SearchHortonworks
 
Summer Shorts: Big Data Integration
Summer Shorts: Big Data IntegrationSummer Shorts: Big Data Integration
Summer Shorts: Big Data Integrationibi
 
Skillwise Big Data part 2
Skillwise Big Data part 2Skillwise Big Data part 2
Skillwise Big Data part 2Skillwise Group
 
Hadoop and SQL: Delivery Analytics Across the Organization
Hadoop and SQL:  Delivery Analytics Across the OrganizationHadoop and SQL:  Delivery Analytics Across the Organization
Hadoop and SQL: Delivery Analytics Across the OrganizationSeeling Cheung
 
Accelerating Big Data Analytics
Accelerating Big Data AnalyticsAccelerating Big Data Analytics
Accelerating Big Data AnalyticsAttunity
 
Using the Power of Big SQL 3.0 to Build a Big Data-Ready Hybrid Warehouse
Using the Power of Big SQL 3.0 to Build a Big Data-Ready Hybrid WarehouseUsing the Power of Big SQL 3.0 to Build a Big Data-Ready Hybrid Warehouse
Using the Power of Big SQL 3.0 to Build a Big Data-Ready Hybrid WarehouseRizaldy Ignacio
 
Justin Sheppard & Ankur Gupta from Sears Holdings Corporation - Single point ...
Justin Sheppard & Ankur Gupta from Sears Holdings Corporation - Single point ...Justin Sheppard & Ankur Gupta from Sears Holdings Corporation - Single point ...
Justin Sheppard & Ankur Gupta from Sears Holdings Corporation - Single point ...Global Business Events
 
Bring Your SAP and Enterprise Data to Hadoop, Kafka, and the Cloud
Bring Your SAP and Enterprise Data to Hadoop, Kafka, and the CloudBring Your SAP and Enterprise Data to Hadoop, Kafka, and the Cloud
Bring Your SAP and Enterprise Data to Hadoop, Kafka, and the CloudDataWorks Summit
 
Move to Hadoop, Go Faster and Save Millions - Mainframe Legacy Modernization
Move to Hadoop, Go Faster and Save Millions - Mainframe Legacy ModernizationMove to Hadoop, Go Faster and Save Millions - Mainframe Legacy Modernization
Move to Hadoop, Go Faster and Save Millions - Mainframe Legacy ModernizationDataWorks Summit
 
Game Changed – How Hadoop is Reinventing Enterprise Thinking
Game Changed – How Hadoop is Reinventing Enterprise ThinkingGame Changed – How Hadoop is Reinventing Enterprise Thinking
Game Changed – How Hadoop is Reinventing Enterprise ThinkingInside Analysis
 
Faster, Cheaper, Easier... and Successful Best Practices for Big Data Integra...
Faster, Cheaper, Easier... and Successful Best Practices for Big Data Integra...Faster, Cheaper, Easier... and Successful Best Practices for Big Data Integra...
Faster, Cheaper, Easier... and Successful Best Practices for Big Data Integra...DataWorks Summit
 
Better Total Value of Ownership (TVO) for Complex Analytic Workflows with the...
Better Total Value of Ownership (TVO) for Complex Analytic Workflows with the...Better Total Value of Ownership (TVO) for Complex Analytic Workflows with the...
Better Total Value of Ownership (TVO) for Complex Analytic Workflows with the...ModusOptimum
 
Optimizing your Modern Data Architecture - with Attunity, RCG Global Services...
Optimizing your Modern Data Architecture - with Attunity, RCG Global Services...Optimizing your Modern Data Architecture - with Attunity, RCG Global Services...
Optimizing your Modern Data Architecture - with Attunity, RCG Global Services...Hortonworks
 
Hadoop Summit San Jose 2014: Costing Your Big Data Operations
Hadoop Summit San Jose 2014: Costing Your Big Data Operations Hadoop Summit San Jose 2014: Costing Your Big Data Operations
Hadoop Summit San Jose 2014: Costing Your Big Data Operations Sumeet Singh
 

Similar to eBay Experimentation Platform on Hadoop (20)

Gobblin' Big Data With Ease @ QConSF 2014
Gobblin' Big Data With Ease @ QConSF 2014Gobblin' Big Data With Ease @ QConSF 2014
Gobblin' Big Data With Ease @ QConSF 2014
 
Hadoop and the Data Warehouse: Point/Counter Point
Hadoop and the Data Warehouse: Point/Counter PointHadoop and the Data Warehouse: Point/Counter Point
Hadoop and the Data Warehouse: Point/Counter Point
 
Architecting the Future of Big Data and Search
Architecting the Future of Big Data and SearchArchitecting the Future of Big Data and Search
Architecting the Future of Big Data and Search
 
Summer Shorts: Big Data Integration
Summer Shorts: Big Data IntegrationSummer Shorts: Big Data Integration
Summer Shorts: Big Data Integration
 
Skillwise Big Data part 2
Skillwise Big Data part 2Skillwise Big Data part 2
Skillwise Big Data part 2
 
Hadoop and SQL: Delivery Analytics Across the Organization
Hadoop and SQL:  Delivery Analytics Across the OrganizationHadoop and SQL:  Delivery Analytics Across the Organization
Hadoop and SQL: Delivery Analytics Across the Organization
 
Skilwise Big data
Skilwise Big dataSkilwise Big data
Skilwise Big data
 
Accelerating Big Data Analytics
Accelerating Big Data AnalyticsAccelerating Big Data Analytics
Accelerating Big Data Analytics
 
Prashanth Kumar_Hadoop_NEW
Prashanth Kumar_Hadoop_NEWPrashanth Kumar_Hadoop_NEW
Prashanth Kumar_Hadoop_NEW
 
Using the Power of Big SQL 3.0 to Build a Big Data-Ready Hybrid Warehouse
Using the Power of Big SQL 3.0 to Build a Big Data-Ready Hybrid WarehouseUsing the Power of Big SQL 3.0 to Build a Big Data-Ready Hybrid Warehouse
Using the Power of Big SQL 3.0 to Build a Big Data-Ready Hybrid Warehouse
 
Justin Sheppard & Ankur Gupta from Sears Holdings Corporation - Single point ...
Justin Sheppard & Ankur Gupta from Sears Holdings Corporation - Single point ...Justin Sheppard & Ankur Gupta from Sears Holdings Corporation - Single point ...
Justin Sheppard & Ankur Gupta from Sears Holdings Corporation - Single point ...
 
Bring Your SAP and Enterprise Data to Hadoop, Kafka, and the Cloud
Bring Your SAP and Enterprise Data to Hadoop, Kafka, and the CloudBring Your SAP and Enterprise Data to Hadoop, Kafka, and the Cloud
Bring Your SAP and Enterprise Data to Hadoop, Kafka, and the Cloud
 
Move to Hadoop, Go Faster and Save Millions - Mainframe Legacy Modernization
Move to Hadoop, Go Faster and Save Millions - Mainframe Legacy ModernizationMove to Hadoop, Go Faster and Save Millions - Mainframe Legacy Modernization
Move to Hadoop, Go Faster and Save Millions - Mainframe Legacy Modernization
 
Retail & CPG
Retail & CPGRetail & CPG
Retail & CPG
 
Game Changed – How Hadoop is Reinventing Enterprise Thinking
Game Changed – How Hadoop is Reinventing Enterprise ThinkingGame Changed – How Hadoop is Reinventing Enterprise Thinking
Game Changed – How Hadoop is Reinventing Enterprise Thinking
 
Hadoop and Your Enterprise Data Warehouse
Hadoop and Your Enterprise Data WarehouseHadoop and Your Enterprise Data Warehouse
Hadoop and Your Enterprise Data Warehouse
 
Faster, Cheaper, Easier... and Successful Best Practices for Big Data Integra...
Faster, Cheaper, Easier... and Successful Best Practices for Big Data Integra...Faster, Cheaper, Easier... and Successful Best Practices for Big Data Integra...
Faster, Cheaper, Easier... and Successful Best Practices for Big Data Integra...
 
Better Total Value of Ownership (TVO) for Complex Analytic Workflows with the...
Better Total Value of Ownership (TVO) for Complex Analytic Workflows with the...Better Total Value of Ownership (TVO) for Complex Analytic Workflows with the...
Better Total Value of Ownership (TVO) for Complex Analytic Workflows with the...
 
Optimizing your Modern Data Architecture - with Attunity, RCG Global Services...
Optimizing your Modern Data Architecture - with Attunity, RCG Global Services...Optimizing your Modern Data Architecture - with Attunity, RCG Global Services...
Optimizing your Modern Data Architecture - with Attunity, RCG Global Services...
 
Hadoop Summit San Jose 2014: Costing Your Big Data Operations
Hadoop Summit San Jose 2014: Costing Your Big Data Operations Hadoop Summit San Jose 2014: Costing Your Big Data Operations
Hadoop Summit San Jose 2014: Costing Your Big Data Operations
 

Recently uploaded

Implementing Zero Trust strategy with Azure
Implementing Zero Trust strategy with AzureImplementing Zero Trust strategy with Azure
Implementing Zero Trust strategy with AzureDinusha Kumarasiri
 
Taming Distributed Systems: Key Insights from Wix's Large-Scale Experience - ...
Taming Distributed Systems: Key Insights from Wix's Large-Scale Experience - ...Taming Distributed Systems: Key Insights from Wix's Large-Scale Experience - ...
Taming Distributed Systems: Key Insights from Wix's Large-Scale Experience - ...Natan Silnitsky
 
Ahmed Motair CV April 2024 (Senior SW Developer)
Ahmed Motair CV April 2024 (Senior SW Developer)Ahmed Motair CV April 2024 (Senior SW Developer)
Ahmed Motair CV April 2024 (Senior SW Developer)Ahmed Mater
 
GOING AOT WITH GRAALVM – DEVOXX GREECE.pdf
GOING AOT WITH GRAALVM – DEVOXX GREECE.pdfGOING AOT WITH GRAALVM – DEVOXX GREECE.pdf
GOING AOT WITH GRAALVM – DEVOXX GREECE.pdfAlina Yurenko
 
Cyber security and its impact on E commerce
Cyber security and its impact on E commerceCyber security and its impact on E commerce
Cyber security and its impact on E commercemanigoyal112
 
A healthy diet for your Java application Devoxx France.pdf
A healthy diet for your Java application Devoxx France.pdfA healthy diet for your Java application Devoxx France.pdf
A healthy diet for your Java application Devoxx France.pdfMarharyta Nedzelska
 
Cloud Data Center Network Construction - IEEE
Cloud Data Center Network Construction - IEEECloud Data Center Network Construction - IEEE
Cloud Data Center Network Construction - IEEEVICTOR MAESTRE RAMIREZ
 
How to Track Employee Performance A Comprehensive Guide.pdf
How to Track Employee Performance A Comprehensive Guide.pdfHow to Track Employee Performance A Comprehensive Guide.pdf
How to Track Employee Performance A Comprehensive Guide.pdfLivetecs LLC
 
Intelligent Home Wi-Fi Solutions | ThinkPalm
Intelligent Home Wi-Fi Solutions | ThinkPalmIntelligent Home Wi-Fi Solutions | ThinkPalm
Intelligent Home Wi-Fi Solutions | ThinkPalmSujith Sukumaran
 
Global Identity Enrolment and Verification Pro Solution - Cizo Technology Ser...
Global Identity Enrolment and Verification Pro Solution - Cizo Technology Ser...Global Identity Enrolment and Verification Pro Solution - Cizo Technology Ser...
Global Identity Enrolment and Verification Pro Solution - Cizo Technology Ser...Cizo Technology Services
 
Alfresco TTL#157 - Troubleshooting Made Easy: Deciphering Alfresco mTLS Confi...
Alfresco TTL#157 - Troubleshooting Made Easy: Deciphering Alfresco mTLS Confi...Alfresco TTL#157 - Troubleshooting Made Easy: Deciphering Alfresco mTLS Confi...
Alfresco TTL#157 - Troubleshooting Made Easy: Deciphering Alfresco mTLS Confi...Angel Borroy López
 
Introduction Computer Science - Software Design.pdf
Introduction Computer Science - Software Design.pdfIntroduction Computer Science - Software Design.pdf
Introduction Computer Science - Software Design.pdfFerryKemperman
 
Folding Cheat Sheet #4 - fourth in a series
Folding Cheat Sheet #4 - fourth in a seriesFolding Cheat Sheet #4 - fourth in a series
Folding Cheat Sheet #4 - fourth in a seriesPhilip Schwarz
 
Recruitment Management Software Benefits (Infographic)
Recruitment Management Software Benefits (Infographic)Recruitment Management Software Benefits (Infographic)
Recruitment Management Software Benefits (Infographic)Hr365.us smith
 
Tech Tuesday - Mastering Time Management Unlock the Power of OnePlan's Timesh...
Tech Tuesday - Mastering Time Management Unlock the Power of OnePlan's Timesh...Tech Tuesday - Mastering Time Management Unlock the Power of OnePlan's Timesh...
Tech Tuesday - Mastering Time Management Unlock the Power of OnePlan's Timesh...OnePlan Solutions
 
What is Fashion PLM and Why Do You Need It
What is Fashion PLM and Why Do You Need ItWhat is Fashion PLM and Why Do You Need It
What is Fashion PLM and Why Do You Need ItWave PLM
 
Balasore Best It Company|| Top 10 IT Company || Balasore Software company Odisha
Balasore Best It Company|| Top 10 IT Company || Balasore Software company OdishaBalasore Best It Company|| Top 10 IT Company || Balasore Software company Odisha
Balasore Best It Company|| Top 10 IT Company || Balasore Software company Odishasmiwainfosol
 
Xen Safety Embedded OSS Summit April 2024 v4.pdf
Xen Safety Embedded OSS Summit April 2024 v4.pdfXen Safety Embedded OSS Summit April 2024 v4.pdf
Xen Safety Embedded OSS Summit April 2024 v4.pdfStefano Stabellini
 
SuccessFactors 1H 2024 Release - Sneak-Peek by Deloitte Germany
SuccessFactors 1H 2024 Release - Sneak-Peek by Deloitte GermanySuccessFactors 1H 2024 Release - Sneak-Peek by Deloitte Germany
SuccessFactors 1H 2024 Release - Sneak-Peek by Deloitte GermanyChristoph Pohl
 

Recently uploaded (20)

Implementing Zero Trust strategy with Azure
Implementing Zero Trust strategy with AzureImplementing Zero Trust strategy with Azure
Implementing Zero Trust strategy with Azure
 
Taming Distributed Systems: Key Insights from Wix's Large-Scale Experience - ...
Taming Distributed Systems: Key Insights from Wix's Large-Scale Experience - ...Taming Distributed Systems: Key Insights from Wix's Large-Scale Experience - ...
Taming Distributed Systems: Key Insights from Wix's Large-Scale Experience - ...
 
Ahmed Motair CV April 2024 (Senior SW Developer)
Ahmed Motair CV April 2024 (Senior SW Developer)Ahmed Motair CV April 2024 (Senior SW Developer)
Ahmed Motair CV April 2024 (Senior SW Developer)
 
GOING AOT WITH GRAALVM – DEVOXX GREECE.pdf
GOING AOT WITH GRAALVM – DEVOXX GREECE.pdfGOING AOT WITH GRAALVM – DEVOXX GREECE.pdf
GOING AOT WITH GRAALVM – DEVOXX GREECE.pdf
 
Cyber security and its impact on E commerce
Cyber security and its impact on E commerceCyber security and its impact on E commerce
Cyber security and its impact on E commerce
 
A healthy diet for your Java application Devoxx France.pdf
A healthy diet for your Java application Devoxx France.pdfA healthy diet for your Java application Devoxx France.pdf
A healthy diet for your Java application Devoxx France.pdf
 
Cloud Data Center Network Construction - IEEE
Cloud Data Center Network Construction - IEEECloud Data Center Network Construction - IEEE
Cloud Data Center Network Construction - IEEE
 
How to Track Employee Performance A Comprehensive Guide.pdf
How to Track Employee Performance A Comprehensive Guide.pdfHow to Track Employee Performance A Comprehensive Guide.pdf
How to Track Employee Performance A Comprehensive Guide.pdf
 
Intelligent Home Wi-Fi Solutions | ThinkPalm
Intelligent Home Wi-Fi Solutions | ThinkPalmIntelligent Home Wi-Fi Solutions | ThinkPalm
Intelligent Home Wi-Fi Solutions | ThinkPalm
 
Global Identity Enrolment and Verification Pro Solution - Cizo Technology Ser...
Global Identity Enrolment and Verification Pro Solution - Cizo Technology Ser...Global Identity Enrolment and Verification Pro Solution - Cizo Technology Ser...
Global Identity Enrolment and Verification Pro Solution - Cizo Technology Ser...
 
Alfresco TTL#157 - Troubleshooting Made Easy: Deciphering Alfresco mTLS Confi...
Alfresco TTL#157 - Troubleshooting Made Easy: Deciphering Alfresco mTLS Confi...Alfresco TTL#157 - Troubleshooting Made Easy: Deciphering Alfresco mTLS Confi...
Alfresco TTL#157 - Troubleshooting Made Easy: Deciphering Alfresco mTLS Confi...
 
Introduction Computer Science - Software Design.pdf
Introduction Computer Science - Software Design.pdfIntroduction Computer Science - Software Design.pdf
Introduction Computer Science - Software Design.pdf
 
Folding Cheat Sheet #4 - fourth in a series
Folding Cheat Sheet #4 - fourth in a seriesFolding Cheat Sheet #4 - fourth in a series
Folding Cheat Sheet #4 - fourth in a series
 
Recruitment Management Software Benefits (Infographic)
Recruitment Management Software Benefits (Infographic)Recruitment Management Software Benefits (Infographic)
Recruitment Management Software Benefits (Infographic)
 
Tech Tuesday - Mastering Time Management Unlock the Power of OnePlan's Timesh...
Tech Tuesday - Mastering Time Management Unlock the Power of OnePlan's Timesh...Tech Tuesday - Mastering Time Management Unlock the Power of OnePlan's Timesh...
Tech Tuesday - Mastering Time Management Unlock the Power of OnePlan's Timesh...
 
What is Fashion PLM and Why Do You Need It
What is Fashion PLM and Why Do You Need ItWhat is Fashion PLM and Why Do You Need It
What is Fashion PLM and Why Do You Need It
 
Balasore Best It Company|| Top 10 IT Company || Balasore Software company Odisha
Balasore Best It Company|| Top 10 IT Company || Balasore Software company OdishaBalasore Best It Company|| Top 10 IT Company || Balasore Software company Odisha
Balasore Best It Company|| Top 10 IT Company || Balasore Software company Odisha
 
Xen Safety Embedded OSS Summit April 2024 v4.pdf
Xen Safety Embedded OSS Summit April 2024 v4.pdfXen Safety Embedded OSS Summit April 2024 v4.pdf
Xen Safety Embedded OSS Summit April 2024 v4.pdf
 
SuccessFactors 1H 2024 Release - Sneak-Peek by Deloitte Germany
SuccessFactors 1H 2024 Release - Sneak-Peek by Deloitte GermanySuccessFactors 1H 2024 Release - Sneak-Peek by Deloitte Germany
SuccessFactors 1H 2024 Release - Sneak-Peek by Deloitte Germany
 
2.pdf Ejercicios de programación competitiva
2.pdf Ejercicios de programación competitiva2.pdf Ejercicios de programación competitiva
2.pdf Ejercicios de programación competitiva
 

eBay Experimentation Platform on Hadoop

  • 1. Experimentation Platform on Hadoop Tony Ng, Director, Data Services Padma Gopal, Manager, Experimentation
  • 2. Agenda  Experimentation 101  Reporting Work flow  Why Hadoop?  Framework Architecture  Challenges & Learnings  Q & A
  • 3. Experimentation 101 • What is A/B Testing? • Why is it important? • Intuition vs. Reality • eBay Wins
  • 4. What is A/B Testing? • A/B Testing is comparing two versions of a page or process to see which one performs better • Variations could be: UI Components, Content, Algorithms etc. • Measures: Financial metrics, Click rate, Conversion rate etc. Control - Current design Treatment - Variations of current design EP – Hadoop Summit 2015 4
  • 5. How is A/B Testing is done? EP – Hadoop Summit 2015 5
  • 6. Why is it important? • Intuition vs. Reality –Intuition especially on novel ideas should be backed up by data. –Demographics and preferences vary • Data Driven; not based on opinion • Reduce risk EP – Hadoop Summit 2015 6
  • 7. Increased prominence of BIN button compared to Watch, leads to faster checkouts. EP – Hadoop Summit 2015 7
  • 8. Merch placements perform much better when title and price information is provided upfront. EP – Hadoop Summit 2015 8
  • 9. New sign-in design effectively pushed more new users to use guest checkout 9EP – Hadoop Summit 2015
  • 10. 10 What do we support? EP – Hadoop Summit 2015
  • 11. Experimentation Reporting • How does EP work? • Work Flow • DW Challenges
  • 12. Experiment Lifecycle EP – Hadoop Summit 2015 12
  • 13. EP – Hadoop Summit 2015 13 User Behavior & Transactional Data Experiment Metadata Detail Intermediate Summaries 4 Billion Rows 4 TB User1 Homepage User1 Search for IPhone6 User1 View Item1 User2 Search for Coach bag User2 View Item2 User2 Bid Treatment 2 User1 Homepage Treatment 1 User1 Search for IPhone6 Treatment 2 User1 Search for IPhone6 Treatment 1 User1 View Item 1 Treatment 2 User1 View Item 1 Treatment 1 User2 Search for Coach bag Treatment 2 User2 Search for Coach bag Treatment 1 100+ Metrics Treatment 1 20 X Dimensions Treatment 1 10 Metric Insights Treatment 2 100+ Metrics Treatment 2 20 X Dimensions Treatment 2 10 Data Insights
  • 14. EP – Hadoop Summit 2015 14 Transactional Metrics Activity Metrics Acquisition Metrics AD Metrics Email Metrics Seller Metrics Engagement metrics Absolute - Actual number/counts Normalized - Weighted mean (by GUID/UID) Lift - Difference between treatment and control Standard Deviation - Weighted standard deviation Confidence Interval – Range within which treatment effect is likely to lie P-values – Statistically significance Outlier capped – Trim tail values Post Stratified – Adjustment method to reduce variance DATA INSIGHTS Daily Weekly Cumulative Browser OS Device Site/Country Category Segment Geo
  • 15. Hadoop Migration • Why Hadoop • Tech Stack • Architecture Overview
  • 16. EP – Hadoop Summit 2015 16 Why Hadoop? • Design & Development flexibility • Store large amounts of data without the schemas constraints • System to support complex data transformation logic • Code base reduction • Configurability • Code not tied to environment & easier to share • Support for complex structures
  • 17. Scheduler/Client EP – Hadoop Summit 2015 17 Physical Architecture Hadoop Cluster Job Workflow RDBMS ETL Bridge Agent BI & PresentationmySQL DW User Behavior Data 1 2 43 5 Hive Scoobi Spark (poc) AVRO ORC
  • 18. EP – Hadoop Summit 2015 18 Tech Stack - Scoobi •Scoobi – Written in Scala, a functional programming language – Supports Object Oriented Designs – Abstraction of MR Framework code to lower – Portability of typical dataset operations like map, flatMap, filter, groupBy, sort, orderBy, partition – DList (Distributed Lists): Jobs are submitted as a series of “steps” representing granular MR jobs. – Enables developers to write a more concise code compared to Java MR code.
  • 19. EP – Hadoop Summit 2015 19 Word Count in Java M/R. import java.io.IOException; import java.util.*; import org.apache.hadoop.fs.Path; import org.apache.hadoop.conf.*; import org.apache.hadoop.io.*; import org.apache.hadoop.mapreduce.*; import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat; public class WordCount { public static class Map extends Mapper<LongWritable, Text, Text, IntWritable> { private final static IntWritable one = new IntWritable(1); private Text word = new Text(); public void map(LongWritable key, Text value, Context context) throws IOException, InterruptedException { String line = value.toString(); StringTokenizer tokenizer = new StringTokenizer(line); while (tokenizer.hasMoreTokens()) { word.set(tokenizer.nextToken()); context.write(word, one); } } } public static class Reduce extends Reducer<Text, IntWritable, Text, IntWritable> { public void reduce(Text key, Iterable<IntWritable> values, Context context) throws IOException, InterruptedException { int sum = 0; for (IntWritable val : values) { sum += val.get(); } context.write(key, new IntWritable(sum)); } } public static void main(String[] args) throws Exception { Configuration conf = new Configuration(); Job job = new Job(conf, "wordcount"); job.setOutputKeyClass(Text.class); job.setOutputValueClass(IntWritable.class); job.setMapperClass(Map.class); job.setReducerClass(Reduce.class); job.setInputFormatClass(TextInputFormat.class); job.setOutputFormatClass(TextOutputFormat.class); FileInputFormat.addInputPath(job, new Path(args[0])); FileOutputFormat.setOutputPath(job,new Path(args[1])); job.waitForCompletion(true); } }
  • 20. EP – Hadoop Summit 2015 20 Word Count in Scoobi import Scoobi._, Reduction._ val lines = fromTextFile("hdfs://in/...") val counts = lines.mapFlatten(_.split(" ")) .map(word => (word, 1)) .groupByKey .combine(Sum.int) counts.toTextFile("hdfs://out/...", overwrite=true).persist(ScoobiConfiguration())
  • 21. EP – Hadoop Summit 2015 21 Tech Stack - File Format • Avro – Supports rich and complex data structures such as Maps, Unions – Self-Describing data files enabling portability (Schema co-exists with data) – Supports schema dynamicity using Generic Records – Supports backward compatibility for data files w.r.t schema changes • ORC (Optimized Row Columnar) – A single file as the output of each task, which reduces the NameNode's load – Metadata stored using Protocol Buffers, which allows addition and removal of fields – Better performance of queries (bound the amount of memory needed for reading or writing) – Light-weight indexes stored within the file
  • 22. EP – Hadoop Summit 2015 22 Tech Stack - Hive • Efficient Joins for large datasets. • UDF for use cases like median and percentile calculations. • Hive Optimizer Joins: - Smaller is loaded into memory as a hash table and the larger is scanned - Map joins are automatically picked up by the optimizer. • Ad-hoc Analysis, Data Reconciliation use-cases and Testing.
  • 23. EP – Hadoop Summit 2015 23 Fun Facts of EP Processing • We read more than 200 TB of data for processing daily. • We run 350 M/R jobs daily. • We perform more than 30 joins using M/R & Hive, including the ones with heavy data skew. • We use 40 TB of YARN memory at peak time on a 170 TB Hadoop cluster. • We can run 150+ concurrent experiments daily. • Report generation takes around 18 hours.
  • 24. 24 Logical Architecture EP – Hadoop Summit 2015 EP Reporting Services Detail Intermediate 1 Intermediate 2 Summary Configuration Filters Data Providers Processors Calculators Metric Providers Output ColumnsMetricsDimensions Framework Components Reporting Context Cache Util/Helpers Command Line Input/Output Conduit Ancillary Services Alerts Shell Scripts Processed Data Store Tools Logging & Monitoring
  • 25. CHALLENGES & LEARNINGS • Joins • Job Optimization • Data Skew 25EP – Hadoop Summit 2015
  • 26. EP – Hadoop Summit 2015 26 Key Challenges •Performance – Job runtimes are subject to SLA & heavily tied to resources •Data Skew (Long tail data distribution) – May cause unrecoverable runtime failures – Poor performance •Joins, Combiner •Job Resiliency – Auto remediation – Alerts and Monitoring
  • 27. EP – Hadoop Summit 2015 27 Solution to Key Challenge - Performance – Tuned the Hadoop job parameters – a few of them are listed below • -Dmapreduce.input.fileinputformat.split.minsize and -Dmapreduce.input.fileinputformat.split.maxsize – Job run times were reduced in the range of 9% to 35% • -Dscoobi.mapreduce.reducers.bytesperreducer – Adjusting this parameter helped optimize the number of reducers to use. Job run times were reduced to the extent of 50% in some cases • -Dscoobi.concurrentjobs – Setting this parameter to true enables multiple steps of a scoobi job to run concurrently • -Dmapreduce.reduce.memory.mb – Tuning this parameter helped relieving memory pressure
  • 28. EP – Hadoop Summit 2015 28 Solution to Key Challenge - Performance – Implement Data cache for objects • Achieved cache hit ratio of over 99% per job • Runtime performance improved in the range of 18% to 39% depending on the job – Redesign/Refactor Jobs and Job Schedules • Extracted logic from existing jobs into their own jobs • Job workflow optimization for better parallelism – Dedicated Hadoop queue with more than 50 TB of YARN memory. • Shared Hadoop cluster resulted in long waiting times, dedicated queue solved the problem of resource crunch.
  • 29. Joins – Data skew in one or both datasets  Scoobi block join divides the skewed data into blocks and joins the data one block at a time. – Multiple joins in a process  Rewrote a process, which needed join with 11 datasets whose size varied from 49 TB to a few mega byte, in hive, as this process was taking 6+ hours in Scoobi and reduced the time to 3 hours in hive. – Other join solutions  Also looked into Hive’s bucket join, but the cost to sort and bucket the datasets was more than regular join. EP – Hadoop Summit 2015 29
  • 30. EP – Hadoop Summit 2015 30 Combiner To relieve Reducer memory pressure and prevent OOM Solution – Emit part-values of the complete operation for the same key using Combiners – Calculating Mean • Mean = ( X1 + X2 + X3 …. Xn )/ (1 + 1 + 1 + 1 … n) • formula is composed of 2 parts and mapper emits 2 part values combining records for the same key. • Reducer receives way fewer records after combining and applies the two parts from each mapper into the actual mean formula. • Concept can be applied to other complex formula such as Variance, as long as the formula can be reduced to parts that are commutative and associative.
  • 31. Job Resiliency – Auto-remediation • Auto-restart in case of job failure due to intermittent cluster issues - Monitoring & Alerting for Hadoop jobs • Continuous monitoring and email alert generated when a long-running job or failure detected - Monitoring & Alerting for Data quality • Daily monitoring of data trend set up for key metrics and email Alert on any anomaly or violations detected - Recon scripts • Checks and alerts setup for intermediate data - Daily data backup • Daily data back up with distcp to a secondary cluster and ability to restore EP – Hadoop Summit 2015 31
  • 32. Next - Evaluate Spark Current Problems - Data processing through Map Reduce is slow for a complex DAG as data is persisted to disk at each step. It is Multiple stages in pipeline are chained together making the overall process very complex. - Massive Joins against very large datasets are slow. - Expressing every complicated business logic into Hadoop Map Reduce is a problem. Alternatives - Apache Spark has wide adoption, expressive, industry backing and thriving community support. - Apache spark has 10x to 100x speed improvements in comparison to traditional M/R jobs. EP – Hadoop Summit 2015 32
  • 33. Summary • Hadoop is ideal for large data processing and provides a highly scalable storage platform. • Hadoop eco-system is still evolving and have to face the issues around the software which is still underdevelopment. • Moving to Hadoop helped to free up huge capacity in DW for deep dive analysis. • Huge cost reduction for business like us with exploding data sets. EP – Hadoop Summit 2015 33
  • 34. Q & A

Editor's Notes

  1. Scoobi – Advantages compared to Java MR Written in Scala, a functional programming language, making Scoobi suitable for writing MR code Supports Object Oriented Designs (and legacy java object data models) MR Framework code is completely abstracted to lower levels leaving application developers to worry only about business logic Typical dataset operations like map, flatMap, filter, groupBy, sort, orderBy, partition are ported over in functionality to MR paradigm Large datasets are abstracted into a data type called DList (Distributed Lists). DLists represent delayed computations (a.k.a Scoobi Plan) using which jobs are submitted as a series of “steps” representing granular MR jobs. Developers do not need to create workflows for individual jobs Any MR operation can be executed on a DList enabling developers to write a more concise code compared to Java MR code. Multiple similar libraries based on Scala such as Scalding and Scrunch
  2.   Scoobi block join, where one of the datasets was heavily skewed. Join key was item_id and one of the datasets had over a million records for the same key, which was causing the job to fail. Block join divides the skewed data into blocks and joins the data one block at a time.         * Replicate the small (left) side n times including the id of the replica in the key. On the right     * side, add a random integer from 0...n-1 to the key. Join using the pseudo-key and strip out the extra     * fields.     * Useful for skewed join keys and large datasets.
  3. To relieve Reducer memory pressure and prevent OOM A combiner may be used to help by performing a map-local aggregation to prevent OOM errors on reducers due to a large number of input records. In Scoobi, a combiner takes the form of a function which may be invoked on a DList. Also, Combiner represents operations that have the Commutative and Associative properties. Further, two records must be combined in all aspects of the records’ attributes to result in a combined record. The problem of combining becomes more compounded in real-world problems where the rules of combining may not be directly applicable to attributes of records.
  4. Current Problems Data processing through Map Reduce is slow for a complex DAG as data is persisted to disk at each step. It is not designed for faster joins. Multiple stages in pipeline are chained together making the overall process very complex. Massive Joins against very large datasets. There is overwhelming need to make data more interactive/responsive and Hadoop is not built for it. Expressing every complicated business logic into Hadoop Map Reduce is a problem.