SlideShare una empresa de Scribd logo
1 de 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

Más contenido relacionado

La actualidad más candente

Top Three Big Data Governance Issues and How Apache ATLAS resolves it for the...
Top Three Big Data Governance Issues and How Apache ATLAS resolves it for the...Top Three Big Data Governance Issues and How Apache ATLAS resolves it for the...
Top Three Big Data Governance Issues and How Apache ATLAS resolves it for the...DataWorks Summit/Hadoop Summit
 
Druid and Hive Together : Use Cases and Best Practices
Druid and Hive Together : Use Cases and Best PracticesDruid and Hive Together : Use Cases and Best Practices
Druid and Hive Together : Use Cases and Best PracticesDataWorks Summit
 
Hadoop Architecture and HDFS
Hadoop Architecture and HDFSHadoop Architecture and HDFS
Hadoop Architecture and HDFSEdureka!
 
The "Big Data" Ecosystem at LinkedIn
The "Big Data" Ecosystem at LinkedInThe "Big Data" Ecosystem at LinkedIn
The "Big Data" Ecosystem at LinkedInSam Shah
 
Hadoop Infrastructure @Uber Past, Present and Future
Hadoop Infrastructure @Uber Past, Present and FutureHadoop Infrastructure @Uber Past, Present and Future
Hadoop Infrastructure @Uber Past, Present and FutureDataWorks Summit
 
Real-time Twitter Sentiment Analysis and Image Recognition with Apache NiFi
Real-time Twitter Sentiment Analysis and Image Recognition with Apache NiFiReal-time Twitter Sentiment Analysis and Image Recognition with Apache NiFi
Real-time Twitter Sentiment Analysis and Image Recognition with Apache NiFiTimothy Spann
 
Netflix Data Engineering @ Uber Engineering Meetup
Netflix Data Engineering @ Uber Engineering MeetupNetflix Data Engineering @ Uber Engineering Meetup
Netflix Data Engineering @ Uber Engineering MeetupBlake Irvine
 
Using Alluxio as a Fault-tolerant Pluggable Optimization Component of JD.com'...
Using Alluxio as a Fault-tolerant Pluggable Optimization Component of JD.com'...Using Alluxio as a Fault-tolerant Pluggable Optimization Component of JD.com'...
Using Alluxio as a Fault-tolerant Pluggable Optimization Component of JD.com'...Alluxio, Inc.
 
A Big Data Timeline
A Big Data TimelineA Big Data Timeline
A Big Data TimelineBig Cloud
 
Hadoop Training | Hadoop Training For Beginners | Hadoop Architecture | Hadoo...
Hadoop Training | Hadoop Training For Beginners | Hadoop Architecture | Hadoo...Hadoop Training | Hadoop Training For Beginners | Hadoop Architecture | Hadoo...
Hadoop Training | Hadoop Training For Beginners | Hadoop Architecture | Hadoo...Simplilearn
 
Transactional SQL in Apache Hive
Transactional SQL in Apache HiveTransactional SQL in Apache Hive
Transactional SQL in Apache HiveDataWorks Summit
 
Scaling Apache Pulsar to 10 PB/day
Scaling Apache Pulsar to 10 PB/dayScaling Apache Pulsar to 10 PB/day
Scaling Apache Pulsar to 10 PB/dayKarthik Ramasamy
 
Hadoop Tutorial | What is Hadoop | Hadoop Project on Reddit | Edureka
Hadoop Tutorial | What is Hadoop | Hadoop Project on Reddit | EdurekaHadoop Tutorial | What is Hadoop | Hadoop Project on Reddit | Edureka
Hadoop Tutorial | What is Hadoop | Hadoop Project on Reddit | EdurekaEdureka!
 

La actualidad más candente (20)

Hadoop and HBase @eBay
Hadoop and HBase @eBayHadoop and HBase @eBay
Hadoop and HBase @eBay
 
Real time analytics
Real time analyticsReal time analytics
Real time analytics
 
Top Three Big Data Governance Issues and How Apache ATLAS resolves it for the...
Top Three Big Data Governance Issues and How Apache ATLAS resolves it for the...Top Three Big Data Governance Issues and How Apache ATLAS resolves it for the...
Top Three Big Data Governance Issues and How Apache ATLAS resolves it for the...
 
Druid and Hive Together : Use Cases and Best Practices
Druid and Hive Together : Use Cases and Best PracticesDruid and Hive Together : Use Cases and Best Practices
Druid and Hive Together : Use Cases and Best Practices
 
Hadoop Architecture and HDFS
Hadoop Architecture and HDFSHadoop Architecture and HDFS
Hadoop Architecture and HDFS
 
Apache Nifi Crash Course
Apache Nifi Crash CourseApache Nifi Crash Course
Apache Nifi Crash Course
 
File Format Benchmark - Avro, JSON, ORC & Parquet
File Format Benchmark - Avro, JSON, ORC & ParquetFile Format Benchmark - Avro, JSON, ORC & Parquet
File Format Benchmark - Avro, JSON, ORC & Parquet
 
The "Big Data" Ecosystem at LinkedIn
The "Big Data" Ecosystem at LinkedInThe "Big Data" Ecosystem at LinkedIn
The "Big Data" Ecosystem at LinkedIn
 
Hadoop Infrastructure @Uber Past, Present and Future
Hadoop Infrastructure @Uber Past, Present and FutureHadoop Infrastructure @Uber Past, Present and Future
Hadoop Infrastructure @Uber Past, Present and Future
 
Real-time Twitter Sentiment Analysis and Image Recognition with Apache NiFi
Real-time Twitter Sentiment Analysis and Image Recognition with Apache NiFiReal-time Twitter Sentiment Analysis and Image Recognition with Apache NiFi
Real-time Twitter Sentiment Analysis and Image Recognition with Apache NiFi
 
Netflix Data Engineering @ Uber Engineering Meetup
Netflix Data Engineering @ Uber Engineering MeetupNetflix Data Engineering @ Uber Engineering Meetup
Netflix Data Engineering @ Uber Engineering Meetup
 
Using Alluxio as a Fault-tolerant Pluggable Optimization Component of JD.com'...
Using Alluxio as a Fault-tolerant Pluggable Optimization Component of JD.com'...Using Alluxio as a Fault-tolerant Pluggable Optimization Component of JD.com'...
Using Alluxio as a Fault-tolerant Pluggable Optimization Component of JD.com'...
 
A Big Data Timeline
A Big Data TimelineA Big Data Timeline
A Big Data Timeline
 
Apache Flume
Apache FlumeApache Flume
Apache Flume
 
Hadoop Training | Hadoop Training For Beginners | Hadoop Architecture | Hadoo...
Hadoop Training | Hadoop Training For Beginners | Hadoop Architecture | Hadoo...Hadoop Training | Hadoop Training For Beginners | Hadoop Architecture | Hadoo...
Hadoop Training | Hadoop Training For Beginners | Hadoop Architecture | Hadoo...
 
Transactional SQL in Apache Hive
Transactional SQL in Apache HiveTransactional SQL in Apache Hive
Transactional SQL in Apache Hive
 
Scaling Apache Pulsar to 10 PB/day
Scaling Apache Pulsar to 10 PB/dayScaling Apache Pulsar to 10 PB/day
Scaling Apache Pulsar to 10 PB/day
 
Zuul @ Netflix SpringOne Platform
Zuul @ Netflix SpringOne PlatformZuul @ Netflix SpringOne Platform
Zuul @ Netflix SpringOne Platform
 
Hadoop Tutorial | What is Hadoop | Hadoop Project on Reddit | Edureka
Hadoop Tutorial | What is Hadoop | Hadoop Project on Reddit | EdurekaHadoop Tutorial | What is Hadoop | Hadoop Project on Reddit | Edureka
Hadoop Tutorial | What is Hadoop | Hadoop Project on Reddit | Edureka
 
An Introduction to Druid
An Introduction to DruidAn Introduction to Druid
An Introduction to Druid
 

Destacado

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
 

Destacado (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 a 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 a 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
 

Último

Professional Resume Template for Software Developers
Professional Resume Template for Software DevelopersProfessional Resume Template for Software Developers
Professional Resume Template for Software DevelopersVinodh Ram
 
5 Signs You Need a Fashion PLM Software.pdf
5 Signs You Need a Fashion PLM Software.pdf5 Signs You Need a Fashion PLM Software.pdf
5 Signs You Need a Fashion PLM Software.pdfWave PLM
 
Tech Tuesday-Harness the Power of Effective Resource Planning with OnePlan’s ...
Tech Tuesday-Harness the Power of Effective Resource Planning with OnePlan’s ...Tech Tuesday-Harness the Power of Effective Resource Planning with OnePlan’s ...
Tech Tuesday-Harness the Power of Effective Resource Planning with OnePlan’s ...OnePlan Solutions
 
How To Troubleshoot Collaboration Apps for the Modern Connected Worker
How To Troubleshoot Collaboration Apps for the Modern Connected WorkerHow To Troubleshoot Collaboration Apps for the Modern Connected Worker
How To Troubleshoot Collaboration Apps for the Modern Connected WorkerThousandEyes
 
Short Story: Unveiling the Reasoning Abilities of Large Language Models by Ke...
Short Story: Unveiling the Reasoning Abilities of Large Language Models by Ke...Short Story: Unveiling the Reasoning Abilities of Large Language Models by Ke...
Short Story: Unveiling the Reasoning Abilities of Large Language Models by Ke...kellynguyen01
 
Test Automation Strategy for Frontend and Backend
Test Automation Strategy for Frontend and BackendTest Automation Strategy for Frontend and Backend
Test Automation Strategy for Frontend and BackendArshad QA
 
CALL ON ➥8923113531 🔝Call Girls Kakori Lucknow best sexual service Online ☂️
CALL ON ➥8923113531 🔝Call Girls Kakori Lucknow best sexual service Online  ☂️CALL ON ➥8923113531 🔝Call Girls Kakori Lucknow best sexual service Online  ☂️
CALL ON ➥8923113531 🔝Call Girls Kakori Lucknow best sexual service Online ☂️anilsa9823
 
Advancing Engineering with AI through the Next Generation of Strategic Projec...
Advancing Engineering with AI through the Next Generation of Strategic Projec...Advancing Engineering with AI through the Next Generation of Strategic Projec...
Advancing Engineering with AI through the Next Generation of Strategic Projec...OnePlan Solutions
 
Try MyIntelliAccount Cloud Accounting Software As A Service Solution Risk Fre...
Try MyIntelliAccount Cloud Accounting Software As A Service Solution Risk Fre...Try MyIntelliAccount Cloud Accounting Software As A Service Solution Risk Fre...
Try MyIntelliAccount Cloud Accounting Software As A Service Solution Risk Fre...MyIntelliSource, Inc.
 
The Real-World Challenges of Medical Device Cybersecurity- Mitigating Vulnera...
The Real-World Challenges of Medical Device Cybersecurity- Mitigating Vulnera...The Real-World Challenges of Medical Device Cybersecurity- Mitigating Vulnera...
The Real-World Challenges of Medical Device Cybersecurity- Mitigating Vulnera...ICS
 
Steps To Getting Up And Running Quickly With MyTimeClock Employee Scheduling ...
Steps To Getting Up And Running Quickly With MyTimeClock Employee Scheduling ...Steps To Getting Up And Running Quickly With MyTimeClock Employee Scheduling ...
Steps To Getting Up And Running Quickly With MyTimeClock Employee Scheduling ...MyIntelliSource, Inc.
 
How To Use Server-Side Rendering with Nuxt.js
How To Use Server-Side Rendering with Nuxt.jsHow To Use Server-Side Rendering with Nuxt.js
How To Use Server-Side Rendering with Nuxt.jsAndolasoft Inc
 
(Genuine) Escort Service Lucknow | Starting ₹,5K To @25k with A/C 🧑🏽‍❤️‍🧑🏻 89...
(Genuine) Escort Service Lucknow | Starting ₹,5K To @25k with A/C 🧑🏽‍❤️‍🧑🏻 89...(Genuine) Escort Service Lucknow | Starting ₹,5K To @25k with A/C 🧑🏽‍❤️‍🧑🏻 89...
(Genuine) Escort Service Lucknow | Starting ₹,5K To @25k with A/C 🧑🏽‍❤️‍🧑🏻 89...gurkirankumar98700
 
Reassessing the Bedrock of Clinical Function Models: An Examination of Large ...
Reassessing the Bedrock of Clinical Function Models: An Examination of Large ...Reassessing the Bedrock of Clinical Function Models: An Examination of Large ...
Reassessing the Bedrock of Clinical Function Models: An Examination of Large ...harshavardhanraghave
 
W01_panagenda_Navigating-the-Future-with-The-Hitchhikers-Guide-to-Notes-and-D...
W01_panagenda_Navigating-the-Future-with-The-Hitchhikers-Guide-to-Notes-and-D...W01_panagenda_Navigating-the-Future-with-The-Hitchhikers-Guide-to-Notes-and-D...
W01_panagenda_Navigating-the-Future-with-The-Hitchhikers-Guide-to-Notes-and-D...panagenda
 
Salesforce Certified Field Service Consultant
Salesforce Certified Field Service ConsultantSalesforce Certified Field Service Consultant
Salesforce Certified Field Service ConsultantAxelRicardoTrocheRiq
 

Último (20)

Professional Resume Template for Software Developers
Professional Resume Template for Software DevelopersProfessional Resume Template for Software Developers
Professional Resume Template for Software Developers
 
5 Signs You Need a Fashion PLM Software.pdf
5 Signs You Need a Fashion PLM Software.pdf5 Signs You Need a Fashion PLM Software.pdf
5 Signs You Need a Fashion PLM Software.pdf
 
Tech Tuesday-Harness the Power of Effective Resource Planning with OnePlan’s ...
Tech Tuesday-Harness the Power of Effective Resource Planning with OnePlan’s ...Tech Tuesday-Harness the Power of Effective Resource Planning with OnePlan’s ...
Tech Tuesday-Harness the Power of Effective Resource Planning with OnePlan’s ...
 
Exploring iOS App Development: Simplifying the Process
Exploring iOS App Development: Simplifying the ProcessExploring iOS App Development: Simplifying the Process
Exploring iOS App Development: Simplifying the Process
 
How To Troubleshoot Collaboration Apps for the Modern Connected Worker
How To Troubleshoot Collaboration Apps for the Modern Connected WorkerHow To Troubleshoot Collaboration Apps for the Modern Connected Worker
How To Troubleshoot Collaboration Apps for the Modern Connected Worker
 
Short Story: Unveiling the Reasoning Abilities of Large Language Models by Ke...
Short Story: Unveiling the Reasoning Abilities of Large Language Models by Ke...Short Story: Unveiling the Reasoning Abilities of Large Language Models by Ke...
Short Story: Unveiling the Reasoning Abilities of Large Language Models by Ke...
 
Test Automation Strategy for Frontend and Backend
Test Automation Strategy for Frontend and BackendTest Automation Strategy for Frontend and Backend
Test Automation Strategy for Frontend and Backend
 
Call Girls In Mukherjee Nagar 📱 9999965857 🤩 Delhi 🫦 HOT AND SEXY VVIP 🍎 SE...
Call Girls In Mukherjee Nagar 📱  9999965857  🤩 Delhi 🫦 HOT AND SEXY VVIP 🍎 SE...Call Girls In Mukherjee Nagar 📱  9999965857  🤩 Delhi 🫦 HOT AND SEXY VVIP 🍎 SE...
Call Girls In Mukherjee Nagar 📱 9999965857 🤩 Delhi 🫦 HOT AND SEXY VVIP 🍎 SE...
 
CALL ON ➥8923113531 🔝Call Girls Kakori Lucknow best sexual service Online ☂️
CALL ON ➥8923113531 🔝Call Girls Kakori Lucknow best sexual service Online  ☂️CALL ON ➥8923113531 🔝Call Girls Kakori Lucknow best sexual service Online  ☂️
CALL ON ➥8923113531 🔝Call Girls Kakori Lucknow best sexual service Online ☂️
 
Advancing Engineering with AI through the Next Generation of Strategic Projec...
Advancing Engineering with AI through the Next Generation of Strategic Projec...Advancing Engineering with AI through the Next Generation of Strategic Projec...
Advancing Engineering with AI through the Next Generation of Strategic Projec...
 
Microsoft AI Transformation Partner Playbook.pdf
Microsoft AI Transformation Partner Playbook.pdfMicrosoft AI Transformation Partner Playbook.pdf
Microsoft AI Transformation Partner Playbook.pdf
 
Try MyIntelliAccount Cloud Accounting Software As A Service Solution Risk Fre...
Try MyIntelliAccount Cloud Accounting Software As A Service Solution Risk Fre...Try MyIntelliAccount Cloud Accounting Software As A Service Solution Risk Fre...
Try MyIntelliAccount Cloud Accounting Software As A Service Solution Risk Fre...
 
The Real-World Challenges of Medical Device Cybersecurity- Mitigating Vulnera...
The Real-World Challenges of Medical Device Cybersecurity- Mitigating Vulnera...The Real-World Challenges of Medical Device Cybersecurity- Mitigating Vulnera...
The Real-World Challenges of Medical Device Cybersecurity- Mitigating Vulnera...
 
Steps To Getting Up And Running Quickly With MyTimeClock Employee Scheduling ...
Steps To Getting Up And Running Quickly With MyTimeClock Employee Scheduling ...Steps To Getting Up And Running Quickly With MyTimeClock Employee Scheduling ...
Steps To Getting Up And Running Quickly With MyTimeClock Employee Scheduling ...
 
How To Use Server-Side Rendering with Nuxt.js
How To Use Server-Side Rendering with Nuxt.jsHow To Use Server-Side Rendering with Nuxt.js
How To Use Server-Side Rendering with Nuxt.js
 
(Genuine) Escort Service Lucknow | Starting ₹,5K To @25k with A/C 🧑🏽‍❤️‍🧑🏻 89...
(Genuine) Escort Service Lucknow | Starting ₹,5K To @25k with A/C 🧑🏽‍❤️‍🧑🏻 89...(Genuine) Escort Service Lucknow | Starting ₹,5K To @25k with A/C 🧑🏽‍❤️‍🧑🏻 89...
(Genuine) Escort Service Lucknow | Starting ₹,5K To @25k with A/C 🧑🏽‍❤️‍🧑🏻 89...
 
Reassessing the Bedrock of Clinical Function Models: An Examination of Large ...
Reassessing the Bedrock of Clinical Function Models: An Examination of Large ...Reassessing the Bedrock of Clinical Function Models: An Examination of Large ...
Reassessing the Bedrock of Clinical Function Models: An Examination of Large ...
 
W01_panagenda_Navigating-the-Future-with-The-Hitchhikers-Guide-to-Notes-and-D...
W01_panagenda_Navigating-the-Future-with-The-Hitchhikers-Guide-to-Notes-and-D...W01_panagenda_Navigating-the-Future-with-The-Hitchhikers-Guide-to-Notes-and-D...
W01_panagenda_Navigating-the-Future-with-The-Hitchhikers-Guide-to-Notes-and-D...
 
Salesforce Certified Field Service Consultant
Salesforce Certified Field Service ConsultantSalesforce Certified Field Service Consultant
Salesforce Certified Field Service Consultant
 
Vip Call Girls Noida ➡️ Delhi ➡️ 9999965857 No Advance 24HRS Live
Vip Call Girls Noida ➡️ Delhi ➡️ 9999965857 No Advance 24HRS LiveVip Call Girls Noida ➡️ Delhi ➡️ 9999965857 No Advance 24HRS Live
Vip Call Girls Noida ➡️ Delhi ➡️ 9999965857 No Advance 24HRS Live
 

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

Notas del editor

  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.