Apache Samza

Martin Kleppmann

Definition vehicles, or the writes of records to a database. is an open source frame- jobs are long- work for distributed processing of high- running processes that continuously volume event streams. Its primary design consume one or more event streams, goal is to support high throughput for a invoking some application logic on wide range of processing patterns, while every event, producing derived output providing operational robustness at the streams, and potentially writing output massive scale required by Internet com- to databases for subsequent querying. panies. Samza achieves this goal through While a batch process or a database a small number of carefully designed ab- query typically reads the state of a stractions: partitioned logs for messag- dataset at one point in time, and then ing, fault-tolerant local state, and cluster- finishes, a stream processor is never based task scheduling. finished: it continually awaits the arrival of new events, and it only shuts down when terminated by an administrator. Many tasks can be naturally ex- Overview pressed as stream processing jobs, for example: Stream processing is playing an increas- • aggregating occurrences of events, ingly important part of the data man- e.g., counting how many times a agement needs of many organizations. particular item has been viewed; Event streams can represent many kinds • computing the rate of certain events, of data, for example, the activity of users e.g., for system diagnostics, report- on a website, the movement of goods or ing, and abuse prevention;

1 2 Martin Kleppmann

• enriching events with information the scalability of Samza is directly at- from a database, e.g., extending user tributable to the choice of these founda- click events with information about tional abstractions. The remainder of this the user who performed the action; article provides further detail on those • joining related events, e.g., joining an design decisions and their practical con- event describing an email that was sequences. sent with any events describing the user clicking links in that email; • updating caches, materialized views, and search indexes, e.g., maintaining Partitioned Log Processing an external full-text search index over text in a database; A Samza job consists of a set of Java • using machine learning systems Virtual Machine (JVM) instances, called to classify events, e.g., for spam tasks, that each processes a subset of filtering. the input data. The code running in each JVM comprises the Samza framework Apache Samza, an open source and user code that implements the re- stream processing framework, can be quired application-specific functionality. used for any of the above applications The primary API for user code is the (Kleppmann and Kreps, 2015; Noghabi Java interface StreamTask, which de- et al, 2017). It was originally developed fines a method process(). Figure 1 at LinkedIn, then donated to the Apache shows two examples of user classes im- Software Foundation in 2013, and plementing the StreamTask interface. became a top-level Apache project in Once a Samza job is deployed and 2015. Samza is now used in production initialized, the framework calls the at many Internet companies, including process() method once for every LinkedIn (Paramasivam, 2016), Netflix message in any of the input streams. (Netflix Technology Blog, 2016), Uber The execution of this method may have (Chen, 2016; Hermann and Del Balso, various effects, including querying 2017), and TripAdvisor (Calisi, 2016). or updating local state and sending Samza is designed for usage scenar- messages to output streams. This model ios that require very high throughput: of computation is closely analogous to in some production settings, it pro- a map task in the well-known MapRe- cesses millions of messages per second duce programming model (Dean and or trillions of events per day (Feng, Ghemawat, 2004), with the difference 2015; Paramasivam, 2016; Noghabi that a Samza job’s input is typically et al, 2017). Consequently, the design never-ending (unbounded). of Samza prioritizes scalability and Similarly to MapReduce, each Samza operational robustness above most other task is a single-threaded process that it- concerns. erates over a sequence of input records. The core of Samza consists of sev- The inputs to a Samza job are partitioned eral fairly low-level abstractions, on top into disjoint subsets, and each input par- of which high-level operators have been tition is assigned to exactly one process- built (Pathirage et al, 2016). However, ing task. More than one partition may be the core abstractions have been carefully assigned to the same processing task, in designed for operational robustness, and Apache Samza 3

class SplitWords implements StreamTask { class CountWords implements StreamTask, InitableTask { static final SystemStream WORD_STREAM = new SystemStream("kafka", "words"); private KeyValueStore store;

public void process( public void init(Config config, IncomingMessageEnvelope in, TaskContext context) { MessageCollector out, store = (KeyValueStore) TaskCoordinator _) { context.getStore("word-counts"); } String str = (String) in.getMessage(); public void process( for (String word : str.split(" ")) { IncomingMessageEnvelope in, out.send( MessageCollector out, new OutgoingMessageEnvelope( TaskCoordinator _) { WORD_STREAM, word, 1)); } String word = (String) in.getKey(); } Integer inc = (Integer) in.getMessage(); } Integer count = store.get(word); if (count == null) count = 0; store.put(word, count + inc); } }

Fig. 1 The two operators of a streaming word-frequency counter using Samza’s StreamTask API (Image source: Kleppmann and Kreps (2015), c 2015 IEEE, reused with permission)

Partition 0 “hello world” “hello samza” Split “hello” “hello” “interesting” Count

Partition 1 “samza is interesting” Split “world” “samza” “samza” “is” Count

Kafka topic strings Samza job Kafka topic words Samza job SplitWords CountWords

Fig. 2 A Samza task consumes input from one partition, but can send output to any partition (Image source: Kleppmann and Kreps (2015), c 2015 IEEE, reused with permission)

Output stream “hello” “hello” “interesting” Count —hello“ : 1 —hello“ : 2 —interesting“ : 1

Output stream “world” “samza” “samza” “is” Count —world“ : 1 —samza“ : 1 —samza“ : 2 —is“ : 1

Kafka topic words Samza job CountWords Kafka topic word_counts

Fig. 3 A task’s local state is made durable by emitting a changelog of key-value pairs to Kafka (Image source: Kleppmann and Kreps (2015), c 2015 IEEE, reused with permission) 4 Martin Kleppmann which case the processing of those par- this log available for applications to titions is interleaved on the task thread. consume (Das et al, 2012; Qiao et al, However, the number of partitions in the 2013). Processing the stream of writes input determines the job’s maximum de- to a database enables jobs to maintain gree of parallelism. external indexes or materialized views The log interface assumes that each onto data in a database and is especially partition of the input is a totally ordered relevant in conjunction with Samza’s sequence of records and that each record support for local state (see Section is associated with a monotonically in- “Fault-tolerant Local State”). creasing sequence number or identifier While every partition of an input (known as offset). Since the records in stream is assigned to one particular task each partition are read sequentially, a of a Samza job, the output partitions job can track its progress by periodically are not bound to tasks. That is, when a writing the offset of the last read record StreamTask emits output messages, to durable storage. If a stream processing it can assign them to any partition of task is restarted, it resumes consuming the output stream. This fact can be the input from the last recorded offset. used to group related data items into Most commonly, Samza is used the same partition: for example, in the in conjunction with word-counting application illustrated (see separate article on Kafka). Kafka in Figure 2, the SplitWords task provides a partitioned, fault-tolerant log chooses the output partition for each that allows publishers to append mes- word based on a hash of the word. This sages to a log partition, and consumers ensures that when different tasks en- (subscribers) to sequentially read the counter occurrences of the same word, messages in a log partition (Wang et al, they are all written to the same output 2015; Kreps et al, 2011; Goodhope partition, from where a downstream job et al, 2012). Kafka also allows stream can read and aggregate the occurrences. processing jobs to reprocess previously When stream tasks are composed seen records by resetting the consumer into multistage processing pipelines, the offset to an earlier position, a fact that is output of one task becomes the input to useful during recovery from failures. another task. Unlike many other stream However, Samza’s stream interface is processing frameworks, Samza does not pluggable: besides Kafka, it can use any implement its own message transport storage or messaging system as input, layer to deliver messages between provided that the system can adhere to stream operators. Instead, Kafka is used the partitioned log interface. By default, for this purpose; since Kafka writes all Samza can also read files from the messages to disk, it provides a large Hadoop Distributed Filesystem (HDFS) buffer between stages of the processing as input, in a way that parallels MapRe- pipeline, limited only by the available duce jobs, at competitive performance disk space on the Kafka brokers. (Noghabi et al, 2017). At LinkedIn, Typically, Kafka is configured to re- Samza is commonly deployed with tain several days or weeks worth of mes- Databus inputs: Databus is a change sages in each topic. Thus, if one stage data capture technology that records the of a processing pipeline fails or begins log of writes to a database and makes to run slow, Kafka can simply buffer the Apache Samza 5 input to that stage while leaving ample point, an additional consumer can be at- time for the problem to be resolved. Un- tached to inspect the message flow. like system designs based on backpres- sure, which require a producer to slow down if the consumer cannot keep up, the failure of one Samza job does not af- Fault-tolerant Local State fect any upstream jobs that produce its inputs. This fact is crucial for the robust Stateless stream processing, in which operation of large-scale systems, since it any message can be processed inde- provides fault containment: as far as pos- pendently from any other message, is sible, a fault in one part of the system easy to implement and scale. However, does not negatively impact other parts of many important applications require that the system. stream processing tasks maintain state. Messages are dropped only if the For example: failed or slow processing stage is not • when performing a join between two repaired within the retention period of streams, a task must maintain an in- the Kafka topic. In this case, dropping dex of messages seen on each input messages is desirable because it isolates within some time window, in order to the fault: the alternative – retaining find messages matching the join con- messages indefinitely until the job is dition when they arrive; repaired – would lead to resource ex- • when computing a rate (number of haustion (running out of memory or disk events per time interval) or aggrega- space), which would cause a cascading tion (e.g., sum of a particular field), failure affecting unrelated parts of the a task must maintain the current ag- system. gregate value and update it based on Thus, Samza’s design of using incoming events; Kafka’s on-disk logs for message trans- • when processing an event requires port is a crucial factor in its scalability: a database query to look up some in a large organization, it is often the related data (e.g., looking up a user case that an event stream produced by record for the user who performed one team’s job is consumed by one or the action in the event), the database more jobs that are administered by other can also be regarded as stream teams. The jobs may be operating at processor state. different levels of maturity: for example, a stream produced by an important Many stream processing frameworks production job may be consumed by use transient state that is kept in mem- several unreliable experimental jobs. ory in the processing task, for exam- Using Kafka as a buffer between jobs ple, in a hash table. However, such state ensures that adding an unreliable con- is lost when a task crashes or when a sumer does not negatively impact the processing job is restarted (e.g., to de- more important jobs in the system. ploy a new version). To make the state Finally, an additional benefit of using fault-tolerant, some frameworks such as Kafka for message transport is that every periodically write check- message stream in the system is accessi- points of the in-memory state to durable ble for debugging and monitoring: at any storage (Carbone et al, 2015); this ap- 6 Martin Kleppmann proach is reasonable when the state is state, it remains significantly faster than small, but it becomes expensive as the accessing a remote database (Noghabi state grows (Noghabi et al, 2017). et al, 2017). Another approach, used, for exam- If a job is cleanly shut down and ple, by , is to use an restarted, for example, to deploy a new external database or key-value store version, Samza’s host affinity feature for any processor state that needs to tries to launch each StreamTask instance be fault-tolerant. This approach carries on the machine that has the appropriate a severe performance penalty: due to RocksDB store on its local disk (subject network latency, accessing a database to available resources). Thus, in most on another node is orders of magnitude cases the state survives task restart with- slower than accessing local in-process out any further action. However, in some state (Noghabi et al, 2017). Moreover, cases – for example, if a processing a high-throughput stream processor can node suffers a full system failure – the easily overwhelm the external database state on the local disk may be lost or with queries; if the database is shared rendered inaccessible. with other applications, such overload In order to survive the loss of local risks harming the performance of other disk storage, Samza again relies on applications to the point that they Kafka. For each store containing state become unavailable (Kreps, 2014). of a stream task, Samza creates a Kafka In response to these problems, Samza topic called a changelog that serves as pioneered an approach to managing a replication log for the store. Every state in a stream task that avoids the write to the local RocksDB store is also problems of both checkpointing and encoded as a message and published remote databases. Samza’s approach to this topic, as illustrated in Figure 3. to providing fault-tolerant local state These writes can be performed asyn- has subsequently been adopted in the chronously in batches, enabling much Kafka Streams framework (see article greater throughput than synchronous on Apache Kafka). random-access requests to a remote data Samza allows each task to main- store. The write queue only needs to tain state on the local disk of the be flushed when the offsets of input processing node, with an in-memory streams are written to durable storage, cache for frequently accessed items. as described in the last section. By default, Samza uses RocksDB, When a Samza task needs to recover an embedded key-value store that is its state after the loss of local storage, loaded into the JVM process of the it reads all messages in the appropriate stream task, but other storage engines partition of the changelog topic and can also be plugged in its place. In applies them to a new RocksDB store. Figure 1, the CountWords task When this process completes, the result accesses this managed state through is a new copy of the store that contains the KeyValueStore interface. For the same data as the store that was workloads with good locality, Samza’s lost. Since Kafka replicates all data RocksDB with cache provides per- across multiple nodes, it is suitable for formance close to in-memory stores; fault-tolerant durable storage of this for random-access workloads on large changelog. Apache Samza 7

If a stream task repeatedly writes ployed to a shared pool of machines, new values for the same key in its with each multi-core machine typically local storage, the changelog contains running a mixture of tasks from several many redundant messages, since only different jobs. the most recent value for a given key This architecture requires infrastruc- is required in order to restore local ture for managing resources and for storage. To remove this redundancy, deploying the code of processing jobs Samza uses a Kafka feature called log to the machines on which it is to be compaction on the changelog topic. run. Some frameworks, such as Storm With log compaction enabled, Kafka and Flink, have built-in mechanisms for runs a background process that searches resource management and deployment. for messages with the same key and However, frameworks that perform discards all but the most recent of those their own task scheduling and cluster messages. Thus, whenever a key in the management generally require a static store is overwritten with a new value, the assignment of computing resources – old value is eventually removed from the potentially even dedicated machines – changelog. However, any key that is not before any jobs can be deployed to the overwritten is retained indefinitely by cluster. This static resource allocation Kafka. This compaction process, which leads to inefficiencies in machine uti- is very similar to internal processes in lization and limits the ability to scale on log-structured storage engines, ensures demand (Kulkarni et al, 2015). that the storage cost and recovery time By contrast, Samza relies on existing from a changelog corresponds to the cluster management software, which size of the state, independently of the allows Samza jobs to share a pool of total number of messages ever sent to machines with non-Samza applica- the changelog (Kleppmann, 2017). tions. Samza supports two modes of distributed operation: • A job can be deployed to a cluster Cluster-based Task Scheduling managed by YARN (Vavilapalli et al, 2013). YARN is a general-purpose resource scheduler When a new stream processing job is and cluster manager that can run started, it must be allocated computing stream processors, MapReduce batch resources: CPU cores, RAM, disk jobs, data analytics engines, and var- space, and network bandwidth. Those ious other applications on a shared resources may need to be adjusted from cluster. Samza jobs can be deployed time to time as load varies and reclaimed to existing YARN clusters without when a job is shut down. requiring any special cluster-level At large organizations, hundreds or configuration or resource allocation. thousands of jobs need to run concur- • Samza also supports a stand-alone rently. At such scale, it is not practical to mode in which a job’s JVM instances manually assign resources: task schedul- are deployed and executed through ing and resource allocation must be au- some external process that is not tomated. To maximize hardware utiliza- under Samza’s control. In this case, tion, many jobs and applications are de- the instances use Apache ZooKeeper 8 Martin Kleppmann

(Junqueira et al, 2011) to coordi- processes and that a resource-intensive nate their work, such as assigning process cannot starve others. partitions of the input streams. Samza isolates stream processing jobs from each other in several different The stand-alone mode allows Samza ways. By using Kafka’s on-disk logs to be integrated with an organization’s as a large buffer between producers existing deployment and cluster man- and consumers of a stream, instead of agement tools, or with cloud computing backpressure, Samza ensures that a slow platforms: for example, Netflix runs or failed consumer does not affect up- Samza jobs directly as EC2 instances on stream jobs. By providing fault-tolerant Amazon Web Services (AWS), relying local state as a common abstraction, on the existing cloud facilities for re- Samza improves performance and source allocation (Paramasivam, 2016). avoids reliance on external databases Moreover, Samza’s cluster management that might be overloaded by high query interface is pluggable, enabling further volume. Finally, by integrating with integrations with other technologies YARN and other cluster managers, such as Mesos (Hindman et al, 2011). Samza builds upon existing resource With large deployments, an impor- scheduling and isolation technology that tant concern is resource isolation, that allows a cluster to be shared between is, ensuring that each process receives many different applications without the resources it requested and that a risking resource starvation. misbehaving process cannot starve colocated processes of resources. When running in YARN, Samza supports the Linux cgroups feature to enforce limits References on the CPU and memory use of stream processing tasks. In virtual machine Calisi L (2016) How to convert legacy Hadoop environments such as EC2, resource Map/Reduce ETL systems to Samza isolation is enforced by the hypervisor. Streaming. URL https://www.youtube.com/ watch?v=KQ5OnL2hMBY Carbone P, Katsifodimos A, Ewen S, Markl V, Haridi S, Tzoumas K (2015) Apache Summary Flink: Stream and batch processing in a sin- gle engine. IEEE Data Engineering Bulletin 38(4):28–38, URL http://sites.computer.org/ Apache Samza is a stream processing debull/A15dec/p28.pdf framework that is designed to provide Chen S (2016) Scalable complex event pro- cessing on Samza @Uber. URL https: high throughput and operational ro- //www.slideshare.net/ShuyiChen2/scalable- bustness at very large scale. Efficient complex-event-processing-on-samza-uber resource utilization requires a mixture Das S, Botev C, Surlaker K, Ghosh B, Varadara- of different jobs to share a multi-tenant jan B, Nagaraj S, Zhang D, Gao L, West- computing infrastructure. In such an erman J, Ganti P, Shkolnik B, Topiwala S, Pachev A, Somasundaram N, Subramaniam environment, the primary challenge S (2012) All aboard the Databus! LinkedIn’s in providing robust operation is fault scalable consistent change data capture plat- isolation, that is, ensuring that a faulty form. In: 3rd ACM Symposium on Cloud process cannot disrupt correctly running Computing (SoCC), DOI 10.1145/2391229. 2391247 Apache Samza 9

Dean J, Ghemawat S (2004) MapReduce: Sim- MOD), pp 239–250, DOI 10.1145/2723372. plified data processing on large clusters. In: 2723374 6th USENIX Symposium on Operating Sys- Netflix Technology Blog (2016) Kafka tem Design and Implementation (OSDI) inside Keystone pipeline. URL Feng T (2015) Benchmarking Apache Samza: http://techblog.netflix.com/2016/04/kafka- 1.2 million messages per second on a single inside-keystone-pipeline.html node. URL http://engineering.linkedin.com/ Noghabi SA, Paramasivam K, Pan Y, Ramesh performance/benchmarking-apache-samza- N, Bringhurst J, Gupta I, Campbell RH 12-million-messages-second-single-node (2017) Samza: Stateful scalable stream Goodhope K, Koshy J, Kreps J, Narkhede N, processing at LinkedIn. Proceedings of Park R, Rao J, Ye VY (2012) Building the VLDB Endowment 10(12):1634–1645, LinkedIn’s real-time activity data pipeline. DOI 10.14778/3137765.3137770 IEEE Data Engineering Bulletin 35(2):33– Paramasivam K (2016) Stream processing 45, URL http://sites.computer.org/debull/ with Apache Samza – current and future. A12june/A12JUN-CD.pdf URL https://engineering.linkedin.com/blog/ Hermann J, Del Balso M (2017) 2016/01/whats-new-samza Meet Michelangelo: Uber’s ma- Pathirage M, Hyde J, Pan Y, Plale B (2016) chine learning platform. URL SamzaSQL: Scalable fast data management https://eng.uber.com/michelangelo/ with streaming sql. In: IEEE International Hindman B, Konwinski A, Zaharia M, Ghodsi Workshop on High-Performance Big Data A, Joseph AD, Katz R, Shenker S, Stoica I Computing (HPBDC), pp 1627–1636, DOI (2011) Mesos: A platform for fine-grained 10.1109/IPDPSW.2016.141 resource sharing in the data center. In: 8th Qiao L, Auradar A, Beaver C, Brandt G, Gandhi USENIX Symposium on Networked Sys- M, Gopalakrishna K, Ip W, Jgadish S, Lu tems Design and Implementation (NSDI) S, Pachev A, Ramesh A, Surlaker K, Sebas- Junqueira FP, Reed BC, Serafini M (2011) Zab: tian A, Shanbhag R, Subramaniam S, Sun High-performance broadcast for primary- Y, Topiwala S, Tran C, Westerman J, Zhang backup systems. In: 41st IEEE/IFIP Interna- D, Das S, Quiggle T, Schulman B, Ghosh tional Conference on Dependable Systems B, Curtis A, Seeliger O, Zhang Z (2013) and Networks (DSN), pp 245–256, DOI On brewing fresh Espresso: LinkedIn’s dis- 10.1109/DSN.2011.5958223 tributed data serving platform. In: ACM In- Kleppmann M (2017) Designing Data-Intensive ternational Conference on Management of Applications. O’Reilly Media, ISBN 978-1- Data (SIGMOD), pp 1135–1146, DOI 10. 4493-7332-0 1145/2463676.2465298 Kleppmann M, Kreps J (2015) Kafka, Samza Vavilapalli VK, Murthy AC, Douglas C, Agar- and the Unix philosophy of distributed data. wal S, Konar M, Evans R, Graves T, Lowe J, IEEE Data Engineering Bulletin 38(4):4– Shah H, Seth S, Saha B, Curino C, O’Malley 14, URL http://sites.computer.org/debull/ O, Radia S, Reed B, Baldeschwieler E A15dec/p4.pdf (2013) Apache Hadoop YARN: Yet another Kreps J (2014) Why local state is a funda- resource negotiator. In: 4th ACM Sympo- mental primitive in stream processing. sium on Cloud Computing (SoCC), DOI URL https://www.oreilly.com/ideas/why- 10.1145/2523616.2523633 local-state-is-a-fundamental-primitive-in- Wang G, Koshy J, Subramanian S, Paramasi- stream-processing vam K, Zadeh M, Narkhede N, Rao J, Kreps Kreps J, Narkhede N, Rao J (2011) Kafka: a dis- J, Stein J (2015) Building a replicated log- tributed messaging system for log process- ging system with Apache Kafka. Proceed- ing. In: 6th International Workshop on Net- ings of the VLDB Endowment 8(12):1654– working Meets Databases (NetDB) 1655, DOI 10.14778/2824032.2824063 Kulkarni S, Bhagat N, Fu M, Kedigehalli V, Kellogg C, Mittal S, Patel JM, Ramasamy K, Taneja S (2015) Heron: Stream processing at scale. In: ACM International Conference on Management of Data (SIG- 10 Martin Kleppmann Cross-References

Apache Flink Apache Kafka Continuous Queries Publish/Subscribe