SlideShare a Scribd company logo
1 of 33
Download to read offline
Giraph: Large-scale
graph processing
infrastructure on Hadoop
Avery Ching, Facebook
Christian Kunz, Jybe
10/14/2011
@Hortonworks, Sunnyvale, CA
Agenda
 Why large-scale graph processing?
 Existing solutions
 Goals
 Design and implementation
 Bulk synchronous parallel model
 Writing an application
 Fault-tolerance
 Early performance results
 Future work / conclusion




                                      2
Why scalable graph processing?
 Web and social graphs are at immense scale and continuing to grow
  ›   In 2008, Google estimated the number of web pages at 1 trillion
  ›   At the 2011 F8, Facebook announced it has 800 million monthly active users
  ›   In September 2011, Twitter claimed to have over 100 million active monthly users
  ›   In March 2011, LinkedIn said it had over 120 million registered users

 Relevant and personalized information for users relies strongly on iterative
  graph ranking algorithms (search results, news, ads, etc.)
  ›   In web graphs, page rank and its variants




                                                  3
Example social graph applications
 Popularity rank (page rank)
  ›   Can be personalized for a user or “type” of user
  ›   Determining popular users, news, jobs, etc.

 Shortest paths
  ›   Many variants single-source, s-t shortest paths, all-to-all shortest (memory/storage
      prohibitive)
  ›   How are users, groups connected?

 Clustering, semi-clustering
  ›   Max clique, triangle closure, label propagation algorithms
  ›   Finding related people, groups, interests




                                                  4
Existing solutions
 Sequence of map-reduce jobs in Hadoop
  ›   Classic map-reduce overheads (job startup/shutdown, reloading data from
      HDFS, shuffling)
  ›   Map-reduce programming model not a good fit for graph algorithms

 Google’s Pregel
  ›   Requires its own computing infrastructure
  ›   Not available (unless you work at Google)
  ›   Master is a SPOF

 Message passing interface (MPI)
  ›   Not fault-tolerant
  ›   Too generic




                                                  5
Giraph goals
 Easy deployment on existing big data processing infrastructure
  ›   Maintaining a separate graph processing cluster is a lot of overhead for operations

 Dynamic resource management
  ›   Handle failures gracefully
  ›   Integrate new resources when available

 Graph-oriented API
  ›   Make graph processing code as simple as possible

 Open
  ›   Leverage the community




                                                6
Giraph design
Easy deployment on existing big data processing infrastructure
 Leverage Hadoop installations around the world for iterative graph processing
  ›   Big data today is processed on Hadoop with the Map-Reduce computing model
  ›   Map-Reduce with Hadoop is widely deployed outside of Yahoo! as well (i.e.
      EC2, Cloudera, etc.)

Dynamic resource management
 Bulk synchronous parallel (BSP) computing model
 Fault-tolerant/dynamic graph processing infrastructure
  ›   Automatically adjust to available resources on the Hadoop grid
  ›   No single point of failure except Hadoop namenode and jobtracker
  ›   Relies on ZooKeeper as a fault-tolerant coordination service




                                               7
Giraph design (continued)
Graph-oriented API
 Vertex-centric API to do graph processing (similar to the map() method in
  Hadoop) in a bulk synchronous parallel computing model
  ›   Inspired by Pregel

Open
 Open-sourced to the Apache Incubator
  ›   http://incubator.apache.org/giraph/




                                            8
Bulk synchronous parallel model
 Sequential computation on a single physical machine restricts the
  computational problem domain
 BSP is a proposal of a “bridging” model for parallel computation
  ›   High-level languages bridged to parallel machines
 3 main attributes
  ›   Components that process and/or provide storage
  ›   Router to deliver point-to-point messages
  ›   Synchronization of all or a subset of components through regular intervals
      (supersteps)
 Computation is done when all components are done
 Only a model, does not describe an implementation




                                          9
Why use BSP?
 A relatively simple computational model
 Parallelization of computation/messaging during a superstep
  ›   Components can only communicate by messages delivered out-of-order in the next
      superstep

 Fault-tolerant/dynamic resource utilization
  ›   Supersteps are atomic units of parallel computation
  ›   Any superstep can be restarted from a checkpoint (need not be user defined)
  ›   A new superstep provides an opportunity for rebalancing of components among
      available resources




                                               10
Maximum vertex value example
 All vertices find the maximum value in a strongly connected graph
 If 1st superstep or the set a new maximum value from incoming messages,
  send new value maximum to edges, otherwise vote to halt (gray vertices)



       5                   5                     5                    5



                           1
       1                                         5                    5
                           5


                                                 2
       2                   2                                          5
                                                 5

  Superstep 1         Superstep 2            Superstep 3       Superstep 4
                                        11
Writing a Giraph application
 Every active vertex will call compute() method once during a superstep
  ›   Analogous to map() method in Hadoop for a <key, value> tuple

 Users chooses 4 types for their implementation of Vertex (I  VertexId, V 
  VertexValue, E  EdgeValue, M  MsgValue)

                 Map Reduce                                          Giraph
  public class Mapper<                            public class Vertex<
       KEYIN,                                          I extends WritableComparable,
       VALUEIN,                                        V extends Writable,
       KEYOUT,                                         E extends Writable,
       VALUEOUT> {                                     M extends Writable> {
    void map(KEYIN key,                             void compute(
               VALUEIN value,                          Iterator<M> msgIterator);
               Context context)                   }
               throws IOException,
               InterruptedException;
  }

                                                   12
Basic Giraph API
Get/set local vertex values                                 Voting
I getVertexId();                                            void voteToHalt();
V getVertexValue();                                         boolean isHalted();
void setVertexValue(V vertexValue);
SortedMap<I, Edge<I, E>> getOutEdgeMap();
long getSuperstep();


Graph-wide mutation                                         Messaging

void addVertexRequest(MutableVertex<I, V, E, M> vertex);    void sendMsg(I id, M
void removeVertexRequest(I vertexId);                       msg);
void addEdgeRequest(I sourceVertexId, Edge<I, E> edge);     void sentMsgToAllEdges(
void removeEdgeRequest(I sourceVertexId, I                    M msg);
destVertexId);
 Local vertex mutations happen immediately
 Vertices “vote” to end the computation
  ›   Once all vertices have voted to end the computation, the application is finished
 Graph mutations are processed just prior to the next superstep
 Sent messages are available at the next superstep during compute
                                             13
Page rank example
public class SimplePageRankVertex extends Vertex<LongWritable, DoubleWritable,
  FloatWritable, DoubleWritable> {
  public void compute(Iterator<DoubleWritable> msgIterator) {
      if (getSuperstep() >= 1) {
          double sum = 0;
          while (msgIterator.hasNext()) {
              sum += msgIterator.next().get();
          }
          setVertexValue(new DoubleWritable((0.15f / getNumVertices()) + 0.85f * sum);
      }
      if (getSuperstep() < 30) {
          long edges = getOutEdgeIterator().size();
          sentMsgToAllEdges(new DoubleWritable(getVertexValue().get() / edges));
      } else {
          voteToHalt();
      }
  }
                                                 14
Hadoop
 Open-source implementation of Map-Reduce and GFS (HDFS)
 Meant to solve “big data” challenges such as distributed grep, process log
  data, sorting, etc.
 Includes resource management (JobTracker)
 Not good for message passing (every message passing step requires a Map-
  Reduce job)

   Input             Map      Intermediate        Reduce              Output
  format            tasks          files           tasks              format

  Split 0
                                                                     Output 0
  Split 1
  Split 2
  Split 3                                                            Output 1


                                         15
Giraph job from Hadoop’s perspective
 Giraph has its own InputFormat that calls the user’s VertexInputFormat
  ›   Hadoop will start up the workers requested by Giraph, not based on the InputSplit
      objects generated by the VertexInputFormat

 Giraph has its own OutputFormat that will call the user’s VertexOutputFormat
  ›   Giraph’s internal structure will write the data with the user’s VertexWriter


                  Giraph’s input             Map               Giraph’s output
                      format                tasks                  format


                       Split 0
                                                                  Output 0
                       Split 1
                       Split 2
                       Split 3                                    Output 1



                                                 16
Watching a Giraph job in Hadoop




                    17
Thread architecture
                  Map-only job in Hadoop



    Map         Map        Map             Map         Map
     0           1          2               3           4



                Thread assignment in Giraph




       Worker              Worker                     Worker
     Worker              Worker                     Worker
    Master              Worker                   ZooKeeper




                               18
Thread responsibilities
 Master
  ›   Only one active master at a time
  ›   Runs the VertexInputFormat getSplits() to get the appropriate number of InputSplit
      objects for the application and writes it to ZooKeeper
  ›   Coordinates application
      • Synchronizes supersteps, end of application

      • Handles changes within supersteps (i.e. vertex movement, change in number of workers, etc.)

 Worker
  ›   Reads vertices from InputSplit objects, splitting them into VertexRanges
  ›   Executes the compute() method for every Vertex it is assigned once per superstep
  ›   Buffers the incoming messages to every Vertex it is assigned for the next superstep

 ZooKeeper
  ›   Manages a server that Is a part of the ZooKeeper quorum (maintains global
      application state)

                                                      19
Vertex distribution




                                                                        VertexRange 0
                       V
                                             Worker                 V
                                                                                                                Worker
Master
                       V
                                               0                    V
                                                                                                                  0




                           InputSplit 0
                       V                  Workers process           V




                                                                                                                   VertexRange 0,1
                                          the InputSplit
                                          objects and divide




                                                                        VertexRange 1
                       V                                            V
                                          them further to                               Prior to every
                       V
                                          create                    V                   superstep, the
                                          VertexRange                                   master assigns
                                          objects.                                      every worker 0 or
                       V                                            V
                                                                                        more VertexRange
                                                                                        objects. They are       Master




                                                                        VertexRange 2
                       V
                                             Worker                 V
                                                                                        responsible for the




                                                                                                                    VertexRange 2,3
                       V
                                               1                    V
                                                                                        vertices that fall in
                                                                                        those VertexRange
                           InputSplit 1




                                                                                        objects.
Master uses the
                       V                                            V
VertexInputFormat
to divide the input



                                                                        VertexRange 3
                       V                                            V
data into InputSplit
objects and
serializes them to     V                                            V

ZooKeeper.                                                                                                      Worker
                       V                                            V                                             1
                                                               20
Worker phases in a superstep
                                                                     Execute
      Prepare by                                   If desired,       compute()
                            If desired,
      registering                                  exchange          on each
                            checkpoint
        worker                                        vertex         vertex and
                              vertices
        health                                       ranges          exchange
                                                                     messages


 Master selects one or more of the available workers to use for the superstep
 Users can set the checkpoint frequency
   Checkpoints are implemented by Giraph (all types implement Writable)

 Users can determine how to distribute vertex ranges on the set of available
  workers
 BSP model allows for dynamic resource usage
  ›   Every superstep is an atomic unit of computation
  ›   Resources can change between supersteps and used accordingly (shrink or grow)

                                              21
Fault tolerance
 No single point of failure from Giraph threads
  ›   With multiple master threads, if the current master dies, a new one will automatically
      take over.
  ›   If a worker thread dies, the application is rolled back to a previously checkpointed
      superstep. The next superstep will begin with the new amount of workers
  ›   If a zookeeper server dies, as long as a quorum remains, the application can
      proceed

 Hadoop single points of failure still exist
  ›   Namenode, jobtracker
  ›   Restarting manually from a checkpoint is always possible




                                                 22
Master thread fault tolerance
Before failure of active master 0          After failure of active master 0

 “Active”                                      “Active”
 Master 0                                      Master 0
                          Active                                     Active
                          Master                                     Master
 “Spare”                  State                “Active”              State
 Master 1                                      Master 1

 “Spare”                                       “Spare”
 Master 2                                      Master 2


 One active master, with spare masters taking over in the event of an active
  master failure
 All active master state is stored in ZooKeeper so that a spare master can
  immediately step in when an active master fails
 “Active” master implemented as a queue in ZooKeeper

                                          23
Worker thread fault tolerance
  Superstep i           Superstep i+1           Superstep i+2
(no checkpoint)          (checkpoint)          (no checkpoint)
                                               Worker failure!


                        Superstep i+1           Superstep i+2          Superstep i+3
                         (checkpoint)          (no checkpoint)          (checkpoint)
                                                                     Worker failure after
                                                                    checkpoint complete!


                                                                       Superstep i+3        Application
                                                                      (no checkpoint)       Complete
  A single worker failure causes the superstep to fail
  In order to disrupt the application, a worker must be registered and chosen for the
   current superstep
  Application reverts to the last committed superstep automatically
   ›   Master detects worker failure during any superstep with a ZooKeeper “health” znode

   ›   Master chooses the last committed superstep and sends a command through ZooKeeper for all
       workers to restart from that superstep
                                                      24
Optional features
 Combiners
  ›   Similar to Map-Reduce combiners
  ›   Users implement a combine() method that can reduce the amount of messages sent
      and received
  ›   Run on both the client side and server side
      • Client side saves memory and message traffic

      • Server side save memory

 Aggregators
  ›   Similar to MPI aggregation routines (i.e. max, min, sum, etc.)
  ›   Users can write their own aggregators
  ›   Commutative and associate operations that are performed globally
  ›   Examples include global communication, monitoring, and statistics




                                                  25
Early Yahoo! customers
 Web of Objects
  ›   Currently used for the movie database (10’s of millions of records, run with 400
      workers)
  ›   Popularity rank, shared connections, personalized page rank

 Web map
  ›   Next generation page-rank related algorithms will use this framework (250 billion web
      pages)
  ›   Current graph processing solution uses MPI (no fault-tolerance, customized code)




                                                26
Page rank benchmarks
 Tiberium Tan
  ›   Almost 4000 nodes, shared among numerous groups in Yahoo!
  ›   Hadoop 0.20.204 (secure Hadoop)
  ›   2x Quad Core 2.4GHz, 24 GB RAM, 1x 6TB HD

 org.apache.giraph.benchmark.PageRankBenchmark
  ›   Generates data, number of edges, number of vertices, # of supersteps
  ›   1 master/ZooKeeper
  ›   20 supersteps
  ›   No checkpoints
  ›   1 random edge per vertex




                                              27
Worker scalability (250M vertices)
                5000
                4500
                4000
                3500
Total seconds




                3000
                2500
                2000
                1500
                1000
                 500
                   0
                       0   50   100   150     200    250   300   350
                                      # of workers


                                         28
Vertex scalability (300 workers)
                2500


                2000
Total seconds




                1500


                1000


                 500


                   0
                       0   100       200            300     400         500   600
                                 # of vertices (in 100's of millions)


                                               29
Vertex/worker scalability
                2500                                                     600




                                                                               # of vertices (100's of millions)
                                                                         500
                2000
Total seconds




                                                                         400
                1500
                                                                         300
                1000
                                                                         200

                 500
                                                                         100

                   0                                                     0
                       0   50   100   150        200   250   300   350
                                      # of workers


                                            30
Apache incubation status
 9 committers in all from a variety of relevant companies
  (LinkedIn, Twitter, Hortonworks, Yahoo!, Facebook, Jybe, etc.)
 31 closed issues since incubation, 20 open
 92 posts on the user mailing list for the month of September
 Probably about a month away from an initial release (guess)
 Very interesting tweets (http://twitter.com/#!/search/giraph)
 Early project, lots of opportunity to shape the project
  ›   Looking for contributors…hint, hint.




                                             31
Future work
 Improved fault tolerance testing
  ›   Unit tests exist, but test a limited subset of failures

 More performance testing
  ›   Try other edge distributions (map closer to social graphs or web graphs)

 Importing some of the currently used graph algorithms into an included library
 Storing messages that surpass available memory on disk
  ›   Leverage the programming model, maybe even convert to a native Map-Reduce
      application

 Better vertex distribution
  ›   Hash and range based distributions
 Memory optimization
  ›   Primitive types for special cases, user chosen thread counts



                                                   32
Conclusion
 Giraph is a graph processing infrastructure that runs on existing Hadoop
  infrastructure
  ›   Already being used at Yahoo!
  ›   Lots of opportunity for new parallel graph algorithms!

 Open source
  ›   http://incubator.apache.org/giraph/
  ›   https://issues.apache.org/jira/browse/GIRAPH

 Questions/comments?
  ›   aching@apache.org




                                                33

More Related Content

What's hot

Dynamic Draph / Iterative Computation on Apache Giraph
Dynamic Draph / Iterative Computation on Apache GiraphDynamic Draph / Iterative Computation on Apache Giraph
Dynamic Draph / Iterative Computation on Apache GiraphDataWorks Summit
 
Introduction to Yarn
Introduction to YarnIntroduction to Yarn
Introduction to YarnApache Apex
 
An Introduction to MapReduce
An Introduction to MapReduceAn Introduction to MapReduce
An Introduction to MapReduceFrane Bandov
 
Fast, Scalable Graph Processing: Apache Giraph on YARN
Fast, Scalable Graph Processing: Apache Giraph on YARNFast, Scalable Graph Processing: Apache Giraph on YARN
Fast, Scalable Graph Processing: Apache Giraph on YARNDataWorks Summit
 
Pregel: A System For Large Scale Graph Processing
Pregel: A System For Large Scale Graph ProcessingPregel: A System For Large Scale Graph Processing
Pregel: A System For Large Scale Graph ProcessingRiyad Parvez
 
Matrix Factorizations at Scale: a Comparison of Scientific Data Analytics on ...
Matrix Factorizations at Scale: a Comparison of Scientific Data Analytics on ...Matrix Factorizations at Scale: a Comparison of Scientific Data Analytics on ...
Matrix Factorizations at Scale: a Comparison of Scientific Data Analytics on ...Databricks
 
MapReduce Paradigm
MapReduce ParadigmMapReduce Paradigm
MapReduce ParadigmDilip Reddy
 
Analysing of big data using map reduce
Analysing of big data using map reduceAnalysing of big data using map reduce
Analysing of big data using map reducePaladion Networks
 
PyCascading for Intuitive Flow Processing with Hadoop (gabor szabo)
PyCascading for Intuitive Flow Processing with Hadoop (gabor szabo)PyCascading for Intuitive Flow Processing with Hadoop (gabor szabo)
PyCascading for Intuitive Flow Processing with Hadoop (gabor szabo)PyData
 
Optimal Execution Of MapReduce Jobs In Cloud - Voices 2015
Optimal Execution Of MapReduce Jobs In Cloud - Voices 2015Optimal Execution Of MapReduce Jobs In Cloud - Voices 2015
Optimal Execution Of MapReduce Jobs In Cloud - Voices 2015Deanna Kosaraju
 
Pivotal Data Labs - Technology and Tools in our Data Scientist's Arsenal
Pivotal Data Labs - Technology and Tools in our Data Scientist's Arsenal Pivotal Data Labs - Technology and Tools in our Data Scientist's Arsenal
Pivotal Data Labs - Technology and Tools in our Data Scientist's Arsenal Srivatsan Ramanujam
 
Mapreduce Algorithms
Mapreduce AlgorithmsMapreduce Algorithms
Mapreduce AlgorithmsAmund Tveit
 
Thorny path to the Large-Scale Graph Processing (Highload++, 2014)
Thorny path to the Large-Scale Graph Processing (Highload++, 2014)Thorny path to the Large-Scale Graph Processing (Highload++, 2014)
Thorny path to the Large-Scale Graph Processing (Highload++, 2014)Alexey Zinoviev
 
Modeling with Hadoop kdd2011
Modeling with Hadoop kdd2011Modeling with Hadoop kdd2011
Modeling with Hadoop kdd2011Milind Bhandarkar
 
PyMADlib - A Python wrapper for MADlib : in-database, parallel, machine learn...
PyMADlib - A Python wrapper for MADlib : in-database, parallel, machine learn...PyMADlib - A Python wrapper for MADlib : in-database, parallel, machine learn...
PyMADlib - A Python wrapper for MADlib : in-database, parallel, machine learn...Srivatsan Ramanujam
 

What's hot (20)

Dynamic Draph / Iterative Computation on Apache Giraph
Dynamic Draph / Iterative Computation on Apache GiraphDynamic Draph / Iterative Computation on Apache Giraph
Dynamic Draph / Iterative Computation on Apache Giraph
 
Introduction to Yarn
Introduction to YarnIntroduction to Yarn
Introduction to Yarn
 
An Introduction to MapReduce
An Introduction to MapReduceAn Introduction to MapReduce
An Introduction to MapReduce
 
Map Reduce introduction
Map Reduce introductionMap Reduce introduction
Map Reduce introduction
 
Map Reduce
Map ReduceMap Reduce
Map Reduce
 
Fast, Scalable Graph Processing: Apache Giraph on YARN
Fast, Scalable Graph Processing: Apache Giraph on YARNFast, Scalable Graph Processing: Apache Giraph on YARN
Fast, Scalable Graph Processing: Apache Giraph on YARN
 
Pregel: A System For Large Scale Graph Processing
Pregel: A System For Large Scale Graph ProcessingPregel: A System For Large Scale Graph Processing
Pregel: A System For Large Scale Graph Processing
 
Matrix Factorizations at Scale: a Comparison of Scientific Data Analytics on ...
Matrix Factorizations at Scale: a Comparison of Scientific Data Analytics on ...Matrix Factorizations at Scale: a Comparison of Scientific Data Analytics on ...
Matrix Factorizations at Scale: a Comparison of Scientific Data Analytics on ...
 
MapReduce Paradigm
MapReduce ParadigmMapReduce Paradigm
MapReduce Paradigm
 
Analysing of big data using map reduce
Analysing of big data using map reduceAnalysing of big data using map reduce
Analysing of big data using map reduce
 
MapReduce basic
MapReduce basicMapReduce basic
MapReduce basic
 
Map Reduce
Map ReduceMap Reduce
Map Reduce
 
PyCascading for Intuitive Flow Processing with Hadoop (gabor szabo)
PyCascading for Intuitive Flow Processing with Hadoop (gabor szabo)PyCascading for Intuitive Flow Processing with Hadoop (gabor szabo)
PyCascading for Intuitive Flow Processing with Hadoop (gabor szabo)
 
Optimal Execution Of MapReduce Jobs In Cloud - Voices 2015
Optimal Execution Of MapReduce Jobs In Cloud - Voices 2015Optimal Execution Of MapReduce Jobs In Cloud - Voices 2015
Optimal Execution Of MapReduce Jobs In Cloud - Voices 2015
 
MapReduce Algorithm Design
MapReduce Algorithm DesignMapReduce Algorithm Design
MapReduce Algorithm Design
 
Pivotal Data Labs - Technology and Tools in our Data Scientist's Arsenal
Pivotal Data Labs - Technology and Tools in our Data Scientist's Arsenal Pivotal Data Labs - Technology and Tools in our Data Scientist's Arsenal
Pivotal Data Labs - Technology and Tools in our Data Scientist's Arsenal
 
Mapreduce Algorithms
Mapreduce AlgorithmsMapreduce Algorithms
Mapreduce Algorithms
 
Thorny path to the Large-Scale Graph Processing (Highload++, 2014)
Thorny path to the Large-Scale Graph Processing (Highload++, 2014)Thorny path to the Large-Scale Graph Processing (Highload++, 2014)
Thorny path to the Large-Scale Graph Processing (Highload++, 2014)
 
Modeling with Hadoop kdd2011
Modeling with Hadoop kdd2011Modeling with Hadoop kdd2011
Modeling with Hadoop kdd2011
 
PyMADlib - A Python wrapper for MADlib : in-database, parallel, machine learn...
PyMADlib - A Python wrapper for MADlib : in-database, parallel, machine learn...PyMADlib - A Python wrapper for MADlib : in-database, parallel, machine learn...
PyMADlib - A Python wrapper for MADlib : in-database, parallel, machine learn...
 

Similar to 2011.10.14 Apache Giraph - Hortonworks

Hadoop trainingin bangalore
Hadoop trainingin bangaloreHadoop trainingin bangalore
Hadoop trainingin bangaloreappaji intelhunt
 
MAP REDUCE IN DATA SCIENCE.pptx
MAP REDUCE IN DATA SCIENCE.pptxMAP REDUCE IN DATA SCIENCE.pptx
MAP REDUCE IN DATA SCIENCE.pptxHARIKRISHNANU13
 
Introduction into scalable graph analysis with Apache Giraph and Spark GraphX
Introduction into scalable graph analysis with Apache Giraph and Spark GraphXIntroduction into scalable graph analysis with Apache Giraph and Spark GraphX
Introduction into scalable graph analysis with Apache Giraph and Spark GraphXrhatr
 
Behm Shah Pagerank
Behm Shah PagerankBehm Shah Pagerank
Behm Shah Pagerankgothicane
 
Hadoop fault tolerance
Hadoop  fault toleranceHadoop  fault tolerance
Hadoop fault tolerancePallav Jha
 
PREGEL a system for large scale graph processing
PREGEL a system for large scale graph processingPREGEL a system for large scale graph processing
PREGEL a system for large scale graph processingAbolfazl Asudeh
 
Apache Flink & Graph Processing
Apache Flink & Graph ProcessingApache Flink & Graph Processing
Apache Flink & Graph ProcessingVasia Kalavri
 
Brief introduction on Hadoop,Dremel, Pig, FlumeJava and Cassandra
Brief introduction on Hadoop,Dremel, Pig, FlumeJava and CassandraBrief introduction on Hadoop,Dremel, Pig, FlumeJava and Cassandra
Brief introduction on Hadoop,Dremel, Pig, FlumeJava and CassandraSomnath Mazumdar
 
Hadoop interview questions
Hadoop interview questionsHadoop interview questions
Hadoop interview questionsKalyan Hadoop
 
Large-scale graph processing with Apache Flink @GraphDevroom FOSDEM'15
Large-scale graph processing with Apache Flink @GraphDevroom FOSDEM'15Large-scale graph processing with Apache Flink @GraphDevroom FOSDEM'15
Large-scale graph processing with Apache Flink @GraphDevroom FOSDEM'15Vasia Kalavri
 
Report Hadoop Map Reduce
Report Hadoop Map ReduceReport Hadoop Map Reduce
Report Hadoop Map ReduceUrvashi Kataria
 
Big data shim
Big data shimBig data shim
Big data shimtistrue
 
Tez: Accelerating Data Pipelines - fifthel
Tez: Accelerating Data Pipelines - fifthelTez: Accelerating Data Pipelines - fifthel
Tez: Accelerating Data Pipelines - fifthelt3rmin4t0r
 

Similar to 2011.10.14 Apache Giraph - Hortonworks (20)

MapReduce
MapReduceMapReduce
MapReduce
 
Hadoop trainingin bangalore
Hadoop trainingin bangaloreHadoop trainingin bangalore
Hadoop trainingin bangalore
 
Map Reduce
Map ReduceMap Reduce
Map Reduce
 
MAP REDUCE IN DATA SCIENCE.pptx
MAP REDUCE IN DATA SCIENCE.pptxMAP REDUCE IN DATA SCIENCE.pptx
MAP REDUCE IN DATA SCIENCE.pptx
 
Lecture 2 part 3
Lecture 2 part 3Lecture 2 part 3
Lecture 2 part 3
 
Introduction into scalable graph analysis with Apache Giraph and Spark GraphX
Introduction into scalable graph analysis with Apache Giraph and Spark GraphXIntroduction into scalable graph analysis with Apache Giraph and Spark GraphX
Introduction into scalable graph analysis with Apache Giraph and Spark GraphX
 
Behm Shah Pagerank
Behm Shah PagerankBehm Shah Pagerank
Behm Shah Pagerank
 
Map reduce prashant
Map reduce prashantMap reduce prashant
Map reduce prashant
 
Hadoop Architecture
Hadoop ArchitectureHadoop Architecture
Hadoop Architecture
 
Hadoop fault tolerance
Hadoop  fault toleranceHadoop  fault tolerance
Hadoop fault tolerance
 
PREGEL a system for large scale graph processing
PREGEL a system for large scale graph processingPREGEL a system for large scale graph processing
PREGEL a system for large scale graph processing
 
Apache Flink & Graph Processing
Apache Flink & Graph ProcessingApache Flink & Graph Processing
Apache Flink & Graph Processing
 
Brief introduction on Hadoop,Dremel, Pig, FlumeJava and Cassandra
Brief introduction on Hadoop,Dremel, Pig, FlumeJava and CassandraBrief introduction on Hadoop,Dremel, Pig, FlumeJava and Cassandra
Brief introduction on Hadoop,Dremel, Pig, FlumeJava and Cassandra
 
Hadoop interview questions
Hadoop interview questionsHadoop interview questions
Hadoop interview questions
 
Large-scale graph processing with Apache Flink @GraphDevroom FOSDEM'15
Large-scale graph processing with Apache Flink @GraphDevroom FOSDEM'15Large-scale graph processing with Apache Flink @GraphDevroom FOSDEM'15
Large-scale graph processing with Apache Flink @GraphDevroom FOSDEM'15
 
Report Hadoop Map Reduce
Report Hadoop Map ReduceReport Hadoop Map Reduce
Report Hadoop Map Reduce
 
Hadoop ecosystem
Hadoop ecosystemHadoop ecosystem
Hadoop ecosystem
 
Big data shim
Big data shimBig data shim
Big data shim
 
Tez: Accelerating Data Pipelines - fifthel
Tez: Accelerating Data Pipelines - fifthelTez: Accelerating Data Pipelines - fifthel
Tez: Accelerating Data Pipelines - fifthel
 
MapReduce basics
MapReduce basicsMapReduce basics
MapReduce basics
 

Recently uploaded

Computer 10: Lesson 10 - Online Crimes and Hazards
Computer 10: Lesson 10 - Online Crimes and HazardsComputer 10: Lesson 10 - Online Crimes and Hazards
Computer 10: Lesson 10 - Online Crimes and HazardsSeth Reyes
 
9 Steps For Building Winning Founding Team
9 Steps For Building Winning Founding Team9 Steps For Building Winning Founding Team
9 Steps For Building Winning Founding TeamAdam Moalla
 
Crea il tuo assistente AI con lo Stregatto (open source python framework)
Crea il tuo assistente AI con lo Stregatto (open source python framework)Crea il tuo assistente AI con lo Stregatto (open source python framework)
Crea il tuo assistente AI con lo Stregatto (open source python framework)Commit University
 
Comparing Sidecar-less Service Mesh from Cilium and Istio
Comparing Sidecar-less Service Mesh from Cilium and IstioComparing Sidecar-less Service Mesh from Cilium and Istio
Comparing Sidecar-less Service Mesh from Cilium and IstioChristian Posta
 
Building Your Own AI Instance (TBLC AI )
Building Your Own AI Instance (TBLC AI )Building Your Own AI Instance (TBLC AI )
Building Your Own AI Instance (TBLC AI )Brian Pichman
 
Igniting Next Level Productivity with AI-Infused Data Integration Workflows
Igniting Next Level Productivity with AI-Infused Data Integration WorkflowsIgniting Next Level Productivity with AI-Infused Data Integration Workflows
Igniting Next Level Productivity with AI-Infused Data Integration WorkflowsSafe Software
 
Secure your environment with UiPath and CyberArk technologies - Session 1
Secure your environment with UiPath and CyberArk technologies - Session 1Secure your environment with UiPath and CyberArk technologies - Session 1
Secure your environment with UiPath and CyberArk technologies - Session 1DianaGray10
 
Building AI-Driven Apps Using Semantic Kernel.pptx
Building AI-Driven Apps Using Semantic Kernel.pptxBuilding AI-Driven Apps Using Semantic Kernel.pptx
Building AI-Driven Apps Using Semantic Kernel.pptxUdaiappa Ramachandran
 
ADOPTING WEB 3 FOR YOUR BUSINESS: A STEP-BY-STEP GUIDE
ADOPTING WEB 3 FOR YOUR BUSINESS: A STEP-BY-STEP GUIDEADOPTING WEB 3 FOR YOUR BUSINESS: A STEP-BY-STEP GUIDE
ADOPTING WEB 3 FOR YOUR BUSINESS: A STEP-BY-STEP GUIDELiveplex
 
activity_diagram_combine_v4_20190827.pdfactivity_diagram_combine_v4_20190827.pdf
activity_diagram_combine_v4_20190827.pdfactivity_diagram_combine_v4_20190827.pdfactivity_diagram_combine_v4_20190827.pdfactivity_diagram_combine_v4_20190827.pdf
activity_diagram_combine_v4_20190827.pdfactivity_diagram_combine_v4_20190827.pdfJamie (Taka) Wang
 
Machine Learning Model Validation (Aijun Zhang 2024).pdf
Machine Learning Model Validation (Aijun Zhang 2024).pdfMachine Learning Model Validation (Aijun Zhang 2024).pdf
Machine Learning Model Validation (Aijun Zhang 2024).pdfAijun Zhang
 
COMPUTER 10: Lesson 7 - File Storage and Online Collaboration
COMPUTER 10: Lesson 7 - File Storage and Online CollaborationCOMPUTER 10: Lesson 7 - File Storage and Online Collaboration
COMPUTER 10: Lesson 7 - File Storage and Online Collaborationbruanjhuli
 
Designing A Time bound resource download URL
Designing A Time bound resource download URLDesigning A Time bound resource download URL
Designing A Time bound resource download URLRuncy Oommen
 
Videogame localization & technology_ how to enhance the power of translation.pdf
Videogame localization & technology_ how to enhance the power of translation.pdfVideogame localization & technology_ how to enhance the power of translation.pdf
Videogame localization & technology_ how to enhance the power of translation.pdfinfogdgmi
 
UWB Technology for Enhanced Indoor and Outdoor Positioning in Physiological M...
UWB Technology for Enhanced Indoor and Outdoor Positioning in Physiological M...UWB Technology for Enhanced Indoor and Outdoor Positioning in Physiological M...
UWB Technology for Enhanced Indoor and Outdoor Positioning in Physiological M...UbiTrack UK
 
IaC & GitOps in a Nutshell - a FridayInANuthshell Episode.pdf
IaC & GitOps in a Nutshell - a FridayInANuthshell Episode.pdfIaC & GitOps in a Nutshell - a FridayInANuthshell Episode.pdf
IaC & GitOps in a Nutshell - a FridayInANuthshell Episode.pdfDaniel Santiago Silva Capera
 
The Data Metaverse: Unpacking the Roles, Use Cases, and Tech Trends in Data a...
The Data Metaverse: Unpacking the Roles, Use Cases, and Tech Trends in Data a...The Data Metaverse: Unpacking the Roles, Use Cases, and Tech Trends in Data a...
The Data Metaverse: Unpacking the Roles, Use Cases, and Tech Trends in Data a...Aggregage
 
AI Fame Rush Review – Virtual Influencer Creation In Just Minutes
AI Fame Rush Review – Virtual Influencer Creation In Just MinutesAI Fame Rush Review – Virtual Influencer Creation In Just Minutes
AI Fame Rush Review – Virtual Influencer Creation In Just MinutesMd Hossain Ali
 

Recently uploaded (20)

Computer 10: Lesson 10 - Online Crimes and Hazards
Computer 10: Lesson 10 - Online Crimes and HazardsComputer 10: Lesson 10 - Online Crimes and Hazards
Computer 10: Lesson 10 - Online Crimes and Hazards
 
9 Steps For Building Winning Founding Team
9 Steps For Building Winning Founding Team9 Steps For Building Winning Founding Team
9 Steps For Building Winning Founding Team
 
Crea il tuo assistente AI con lo Stregatto (open source python framework)
Crea il tuo assistente AI con lo Stregatto (open source python framework)Crea il tuo assistente AI con lo Stregatto (open source python framework)
Crea il tuo assistente AI con lo Stregatto (open source python framework)
 
Comparing Sidecar-less Service Mesh from Cilium and Istio
Comparing Sidecar-less Service Mesh from Cilium and IstioComparing Sidecar-less Service Mesh from Cilium and Istio
Comparing Sidecar-less Service Mesh from Cilium and Istio
 
Building Your Own AI Instance (TBLC AI )
Building Your Own AI Instance (TBLC AI )Building Your Own AI Instance (TBLC AI )
Building Your Own AI Instance (TBLC AI )
 
Igniting Next Level Productivity with AI-Infused Data Integration Workflows
Igniting Next Level Productivity with AI-Infused Data Integration WorkflowsIgniting Next Level Productivity with AI-Infused Data Integration Workflows
Igniting Next Level Productivity with AI-Infused Data Integration Workflows
 
Secure your environment with UiPath and CyberArk technologies - Session 1
Secure your environment with UiPath and CyberArk technologies - Session 1Secure your environment with UiPath and CyberArk technologies - Session 1
Secure your environment with UiPath and CyberArk technologies - Session 1
 
Building AI-Driven Apps Using Semantic Kernel.pptx
Building AI-Driven Apps Using Semantic Kernel.pptxBuilding AI-Driven Apps Using Semantic Kernel.pptx
Building AI-Driven Apps Using Semantic Kernel.pptx
 
ADOPTING WEB 3 FOR YOUR BUSINESS: A STEP-BY-STEP GUIDE
ADOPTING WEB 3 FOR YOUR BUSINESS: A STEP-BY-STEP GUIDEADOPTING WEB 3 FOR YOUR BUSINESS: A STEP-BY-STEP GUIDE
ADOPTING WEB 3 FOR YOUR BUSINESS: A STEP-BY-STEP GUIDE
 
activity_diagram_combine_v4_20190827.pdfactivity_diagram_combine_v4_20190827.pdf
activity_diagram_combine_v4_20190827.pdfactivity_diagram_combine_v4_20190827.pdfactivity_diagram_combine_v4_20190827.pdfactivity_diagram_combine_v4_20190827.pdf
activity_diagram_combine_v4_20190827.pdfactivity_diagram_combine_v4_20190827.pdf
 
20230104 - machine vision
20230104 - machine vision20230104 - machine vision
20230104 - machine vision
 
Machine Learning Model Validation (Aijun Zhang 2024).pdf
Machine Learning Model Validation (Aijun Zhang 2024).pdfMachine Learning Model Validation (Aijun Zhang 2024).pdf
Machine Learning Model Validation (Aijun Zhang 2024).pdf
 
COMPUTER 10: Lesson 7 - File Storage and Online Collaboration
COMPUTER 10: Lesson 7 - File Storage and Online CollaborationCOMPUTER 10: Lesson 7 - File Storage and Online Collaboration
COMPUTER 10: Lesson 7 - File Storage and Online Collaboration
 
Designing A Time bound resource download URL
Designing A Time bound resource download URLDesigning A Time bound resource download URL
Designing A Time bound resource download URL
 
Videogame localization & technology_ how to enhance the power of translation.pdf
Videogame localization & technology_ how to enhance the power of translation.pdfVideogame localization & technology_ how to enhance the power of translation.pdf
Videogame localization & technology_ how to enhance the power of translation.pdf
 
UWB Technology for Enhanced Indoor and Outdoor Positioning in Physiological M...
UWB Technology for Enhanced Indoor and Outdoor Positioning in Physiological M...UWB Technology for Enhanced Indoor and Outdoor Positioning in Physiological M...
UWB Technology for Enhanced Indoor and Outdoor Positioning in Physiological M...
 
IaC & GitOps in a Nutshell - a FridayInANuthshell Episode.pdf
IaC & GitOps in a Nutshell - a FridayInANuthshell Episode.pdfIaC & GitOps in a Nutshell - a FridayInANuthshell Episode.pdf
IaC & GitOps in a Nutshell - a FridayInANuthshell Episode.pdf
 
The Data Metaverse: Unpacking the Roles, Use Cases, and Tech Trends in Data a...
The Data Metaverse: Unpacking the Roles, Use Cases, and Tech Trends in Data a...The Data Metaverse: Unpacking the Roles, Use Cases, and Tech Trends in Data a...
The Data Metaverse: Unpacking the Roles, Use Cases, and Tech Trends in Data a...
 
201610817 - edge part1
201610817 - edge part1201610817 - edge part1
201610817 - edge part1
 
AI Fame Rush Review – Virtual Influencer Creation In Just Minutes
AI Fame Rush Review – Virtual Influencer Creation In Just MinutesAI Fame Rush Review – Virtual Influencer Creation In Just Minutes
AI Fame Rush Review – Virtual Influencer Creation In Just Minutes
 

2011.10.14 Apache Giraph - Hortonworks

  • 1. Giraph: Large-scale graph processing infrastructure on Hadoop Avery Ching, Facebook Christian Kunz, Jybe 10/14/2011 @Hortonworks, Sunnyvale, CA
  • 2. Agenda  Why large-scale graph processing?  Existing solutions  Goals  Design and implementation  Bulk synchronous parallel model  Writing an application  Fault-tolerance  Early performance results  Future work / conclusion 2
  • 3. Why scalable graph processing?  Web and social graphs are at immense scale and continuing to grow › In 2008, Google estimated the number of web pages at 1 trillion › At the 2011 F8, Facebook announced it has 800 million monthly active users › In September 2011, Twitter claimed to have over 100 million active monthly users › In March 2011, LinkedIn said it had over 120 million registered users  Relevant and personalized information for users relies strongly on iterative graph ranking algorithms (search results, news, ads, etc.) › In web graphs, page rank and its variants 3
  • 4. Example social graph applications  Popularity rank (page rank) › Can be personalized for a user or “type” of user › Determining popular users, news, jobs, etc.  Shortest paths › Many variants single-source, s-t shortest paths, all-to-all shortest (memory/storage prohibitive) › How are users, groups connected?  Clustering, semi-clustering › Max clique, triangle closure, label propagation algorithms › Finding related people, groups, interests 4
  • 5. Existing solutions  Sequence of map-reduce jobs in Hadoop › Classic map-reduce overheads (job startup/shutdown, reloading data from HDFS, shuffling) › Map-reduce programming model not a good fit for graph algorithms  Google’s Pregel › Requires its own computing infrastructure › Not available (unless you work at Google) › Master is a SPOF  Message passing interface (MPI) › Not fault-tolerant › Too generic 5
  • 6. Giraph goals  Easy deployment on existing big data processing infrastructure › Maintaining a separate graph processing cluster is a lot of overhead for operations  Dynamic resource management › Handle failures gracefully › Integrate new resources when available  Graph-oriented API › Make graph processing code as simple as possible  Open › Leverage the community 6
  • 7. Giraph design Easy deployment on existing big data processing infrastructure  Leverage Hadoop installations around the world for iterative graph processing › Big data today is processed on Hadoop with the Map-Reduce computing model › Map-Reduce with Hadoop is widely deployed outside of Yahoo! as well (i.e. EC2, Cloudera, etc.) Dynamic resource management  Bulk synchronous parallel (BSP) computing model  Fault-tolerant/dynamic graph processing infrastructure › Automatically adjust to available resources on the Hadoop grid › No single point of failure except Hadoop namenode and jobtracker › Relies on ZooKeeper as a fault-tolerant coordination service 7
  • 8. Giraph design (continued) Graph-oriented API  Vertex-centric API to do graph processing (similar to the map() method in Hadoop) in a bulk synchronous parallel computing model › Inspired by Pregel Open  Open-sourced to the Apache Incubator › http://incubator.apache.org/giraph/ 8
  • 9. Bulk synchronous parallel model  Sequential computation on a single physical machine restricts the computational problem domain  BSP is a proposal of a “bridging” model for parallel computation › High-level languages bridged to parallel machines  3 main attributes › Components that process and/or provide storage › Router to deliver point-to-point messages › Synchronization of all or a subset of components through regular intervals (supersteps)  Computation is done when all components are done  Only a model, does not describe an implementation 9
  • 10. Why use BSP?  A relatively simple computational model  Parallelization of computation/messaging during a superstep › Components can only communicate by messages delivered out-of-order in the next superstep  Fault-tolerant/dynamic resource utilization › Supersteps are atomic units of parallel computation › Any superstep can be restarted from a checkpoint (need not be user defined) › A new superstep provides an opportunity for rebalancing of components among available resources 10
  • 11. Maximum vertex value example  All vertices find the maximum value in a strongly connected graph  If 1st superstep or the set a new maximum value from incoming messages, send new value maximum to edges, otherwise vote to halt (gray vertices) 5 5 5 5 1 1 5 5 5 2 2 2 5 5 Superstep 1 Superstep 2 Superstep 3 Superstep 4 11
  • 12. Writing a Giraph application  Every active vertex will call compute() method once during a superstep › Analogous to map() method in Hadoop for a <key, value> tuple  Users chooses 4 types for their implementation of Vertex (I  VertexId, V  VertexValue, E  EdgeValue, M  MsgValue) Map Reduce Giraph public class Mapper< public class Vertex< KEYIN, I extends WritableComparable, VALUEIN, V extends Writable, KEYOUT, E extends Writable, VALUEOUT> { M extends Writable> { void map(KEYIN key, void compute( VALUEIN value, Iterator<M> msgIterator); Context context) } throws IOException, InterruptedException; } 12
  • 13. Basic Giraph API Get/set local vertex values Voting I getVertexId(); void voteToHalt(); V getVertexValue(); boolean isHalted(); void setVertexValue(V vertexValue); SortedMap<I, Edge<I, E>> getOutEdgeMap(); long getSuperstep(); Graph-wide mutation Messaging void addVertexRequest(MutableVertex<I, V, E, M> vertex); void sendMsg(I id, M void removeVertexRequest(I vertexId); msg); void addEdgeRequest(I sourceVertexId, Edge<I, E> edge); void sentMsgToAllEdges( void removeEdgeRequest(I sourceVertexId, I M msg); destVertexId);  Local vertex mutations happen immediately  Vertices “vote” to end the computation › Once all vertices have voted to end the computation, the application is finished  Graph mutations are processed just prior to the next superstep  Sent messages are available at the next superstep during compute 13
  • 14. Page rank example public class SimplePageRankVertex extends Vertex<LongWritable, DoubleWritable, FloatWritable, DoubleWritable> { public void compute(Iterator<DoubleWritable> msgIterator) { if (getSuperstep() >= 1) { double sum = 0; while (msgIterator.hasNext()) { sum += msgIterator.next().get(); } setVertexValue(new DoubleWritable((0.15f / getNumVertices()) + 0.85f * sum); } if (getSuperstep() < 30) { long edges = getOutEdgeIterator().size(); sentMsgToAllEdges(new DoubleWritable(getVertexValue().get() / edges)); } else { voteToHalt(); } } 14
  • 15. Hadoop  Open-source implementation of Map-Reduce and GFS (HDFS)  Meant to solve “big data” challenges such as distributed grep, process log data, sorting, etc.  Includes resource management (JobTracker)  Not good for message passing (every message passing step requires a Map- Reduce job) Input Map Intermediate Reduce Output format tasks files tasks format Split 0 Output 0 Split 1 Split 2 Split 3 Output 1 15
  • 16. Giraph job from Hadoop’s perspective  Giraph has its own InputFormat that calls the user’s VertexInputFormat › Hadoop will start up the workers requested by Giraph, not based on the InputSplit objects generated by the VertexInputFormat  Giraph has its own OutputFormat that will call the user’s VertexOutputFormat › Giraph’s internal structure will write the data with the user’s VertexWriter Giraph’s input Map Giraph’s output format tasks format Split 0 Output 0 Split 1 Split 2 Split 3 Output 1 16
  • 17. Watching a Giraph job in Hadoop 17
  • 18. Thread architecture Map-only job in Hadoop Map Map Map Map Map 0 1 2 3 4 Thread assignment in Giraph Worker Worker Worker Worker Worker Worker Master Worker ZooKeeper 18
  • 19. Thread responsibilities  Master › Only one active master at a time › Runs the VertexInputFormat getSplits() to get the appropriate number of InputSplit objects for the application and writes it to ZooKeeper › Coordinates application • Synchronizes supersteps, end of application • Handles changes within supersteps (i.e. vertex movement, change in number of workers, etc.)  Worker › Reads vertices from InputSplit objects, splitting them into VertexRanges › Executes the compute() method for every Vertex it is assigned once per superstep › Buffers the incoming messages to every Vertex it is assigned for the next superstep  ZooKeeper › Manages a server that Is a part of the ZooKeeper quorum (maintains global application state) 19
  • 20. Vertex distribution VertexRange 0 V Worker V Worker Master V 0 V 0 InputSplit 0 V Workers process V VertexRange 0,1 the InputSplit objects and divide VertexRange 1 V V them further to Prior to every V create V superstep, the VertexRange master assigns objects. every worker 0 or V V more VertexRange objects. They are Master VertexRange 2 V Worker V responsible for the VertexRange 2,3 V 1 V vertices that fall in those VertexRange InputSplit 1 objects. Master uses the V V VertexInputFormat to divide the input VertexRange 3 V V data into InputSplit objects and serializes them to V V ZooKeeper. Worker V V 1 20
  • 21. Worker phases in a superstep Execute Prepare by If desired, compute() If desired, registering exchange on each checkpoint worker vertex vertex and vertices health ranges exchange messages  Master selects one or more of the available workers to use for the superstep  Users can set the checkpoint frequency  Checkpoints are implemented by Giraph (all types implement Writable)  Users can determine how to distribute vertex ranges on the set of available workers  BSP model allows for dynamic resource usage › Every superstep is an atomic unit of computation › Resources can change between supersteps and used accordingly (shrink or grow) 21
  • 22. Fault tolerance  No single point of failure from Giraph threads › With multiple master threads, if the current master dies, a new one will automatically take over. › If a worker thread dies, the application is rolled back to a previously checkpointed superstep. The next superstep will begin with the new amount of workers › If a zookeeper server dies, as long as a quorum remains, the application can proceed  Hadoop single points of failure still exist › Namenode, jobtracker › Restarting manually from a checkpoint is always possible 22
  • 23. Master thread fault tolerance Before failure of active master 0 After failure of active master 0 “Active” “Active” Master 0 Master 0 Active Active Master Master “Spare” State “Active” State Master 1 Master 1 “Spare” “Spare” Master 2 Master 2  One active master, with spare masters taking over in the event of an active master failure  All active master state is stored in ZooKeeper so that a spare master can immediately step in when an active master fails  “Active” master implemented as a queue in ZooKeeper 23
  • 24. Worker thread fault tolerance Superstep i Superstep i+1 Superstep i+2 (no checkpoint) (checkpoint) (no checkpoint) Worker failure! Superstep i+1 Superstep i+2 Superstep i+3 (checkpoint) (no checkpoint) (checkpoint) Worker failure after checkpoint complete! Superstep i+3 Application (no checkpoint) Complete  A single worker failure causes the superstep to fail  In order to disrupt the application, a worker must be registered and chosen for the current superstep  Application reverts to the last committed superstep automatically › Master detects worker failure during any superstep with a ZooKeeper “health” znode › Master chooses the last committed superstep and sends a command through ZooKeeper for all workers to restart from that superstep 24
  • 25. Optional features  Combiners › Similar to Map-Reduce combiners › Users implement a combine() method that can reduce the amount of messages sent and received › Run on both the client side and server side • Client side saves memory and message traffic • Server side save memory  Aggregators › Similar to MPI aggregation routines (i.e. max, min, sum, etc.) › Users can write their own aggregators › Commutative and associate operations that are performed globally › Examples include global communication, monitoring, and statistics 25
  • 26. Early Yahoo! customers  Web of Objects › Currently used for the movie database (10’s of millions of records, run with 400 workers) › Popularity rank, shared connections, personalized page rank  Web map › Next generation page-rank related algorithms will use this framework (250 billion web pages) › Current graph processing solution uses MPI (no fault-tolerance, customized code) 26
  • 27. Page rank benchmarks  Tiberium Tan › Almost 4000 nodes, shared among numerous groups in Yahoo! › Hadoop 0.20.204 (secure Hadoop) › 2x Quad Core 2.4GHz, 24 GB RAM, 1x 6TB HD  org.apache.giraph.benchmark.PageRankBenchmark › Generates data, number of edges, number of vertices, # of supersteps › 1 master/ZooKeeper › 20 supersteps › No checkpoints › 1 random edge per vertex 27
  • 28. Worker scalability (250M vertices) 5000 4500 4000 3500 Total seconds 3000 2500 2000 1500 1000 500 0 0 50 100 150 200 250 300 350 # of workers 28
  • 29. Vertex scalability (300 workers) 2500 2000 Total seconds 1500 1000 500 0 0 100 200 300 400 500 600 # of vertices (in 100's of millions) 29
  • 30. Vertex/worker scalability 2500 600 # of vertices (100's of millions) 500 2000 Total seconds 400 1500 300 1000 200 500 100 0 0 0 50 100 150 200 250 300 350 # of workers 30
  • 31. Apache incubation status  9 committers in all from a variety of relevant companies (LinkedIn, Twitter, Hortonworks, Yahoo!, Facebook, Jybe, etc.)  31 closed issues since incubation, 20 open  92 posts on the user mailing list for the month of September  Probably about a month away from an initial release (guess)  Very interesting tweets (http://twitter.com/#!/search/giraph)  Early project, lots of opportunity to shape the project › Looking for contributors…hint, hint. 31
  • 32. Future work  Improved fault tolerance testing › Unit tests exist, but test a limited subset of failures  More performance testing › Try other edge distributions (map closer to social graphs or web graphs)  Importing some of the currently used graph algorithms into an included library  Storing messages that surpass available memory on disk › Leverage the programming model, maybe even convert to a native Map-Reduce application  Better vertex distribution › Hash and range based distributions  Memory optimization › Primitive types for special cases, user chosen thread counts 32
  • 33. Conclusion  Giraph is a graph processing infrastructure that runs on existing Hadoop infrastructure › Already being used at Yahoo! › Lots of opportunity for new parallel graph algorithms!  Open source › http://incubator.apache.org/giraph/ › https://issues.apache.org/jira/browse/GIRAPH  Questions/comments? › aching@apache.org 33