SlideShare a Scribd company logo
1 of 52
MICROSERVICES WITH KAFKA
—
AN INTRODUCTION TO KAFKA STREAMS APIWITH A REAL-LIFE EXAMPLE
Alexis Seigneurin
Who I am
• Software engineer for 15+ years
• Consultant at Ippon USA, previously at Ippon France
• Favorite subjects: Spark, Kafka, Machine Learning, Scala
• Spark certified
• @aseigneurin
• 200+ software engineers in France, the US and Australia
• In the US: NYC, DC, RVA
• Digital, Big Data and Cloud applications
• Java & Agile expertise
• Open-source projects: JHipster,Tatami, etc.
• @ipponusa
The project
The project
• Analyze records from customers → Send notifications
• High volume of data
• 25 millions records per day in average + with seasonal peaks
• Need to keep at least 60 days of history = 1.5 Billion records
• Need an hybrid platform
• Batch processing for some types of analysis
• Streaming for other analyses
• Hybrid team
• Data Scientists: more familiar with Python
• Software Engineers: Java
Architecture - Real time platform
• New detectors are implemented by Data Scientists all the time
• Need the implementations to be independent from each other
• One Spark Streaming job per detector
• Microservice-inspired architecture
• Diamond-shaped
• Upstream jobs are written in Scala
• Core is made of multiple Python jobs, one per detector
• Downstream jobs are written in Scala
• Plumbing between the jobs → Kafka
1/2
Architecture - Real time platform 2/2
Modularity
• One Spark job per detector
• Hot deployments: can roll out new detectors (= new jobs) without
stopping existing jobs
• Can roll out updated code without affecting other jobs
• Able to measure the resources consumed by a single job
• Shared services are provided by upstream and
downstream jobs
Consuming Kafka messages with Spark Streaming
(and why you probably shouldn’t do it)
Spark 1.x + Kafka?
• Spark has become the de-facto processing framework
• Provides APIs for multiple programming languages
• Python → Data Scientists
• Scala/Java → Software Engineers
• Supports batch jobs and streaming jobs, incl. support for Kafka…
Consuming from Kafka
• Connecting Spark to Kafka, 2 methods:
• Receiver-based approach: not ideal for parallelism
• Direct approach: better for parallelism but have to deal with Kafka
offsets
Spark +
Kakfa
problem
s
Dealing with Kafka offsets
• Default: consumes from the end of the Kafka topic (or the
beginning)
• Documentation → Use checkpoints
• Tasks have to be Serializable (not always possible: dependent libraries)
• Harder to deploy the application (classes are serialized) → run a new instance
in parallel and kill the first one (harder to automate; messages consumed
twice)
• Requires a shared file system (HDFS, S3) → big latency on these FS that
forces to increase the micro-batch interval
1/2Spark +
Kakfa
problem
s
Dealing with Kafka offsets
• Dealing with Kafka offsets
• Solution: deal with offsets in the Spark Streaming application
• Write the offsets to a reliable storage: ZooKeeper, Kafka…
• Write after processing the data
• Read the offsets on startup (if no offsets, start from the end)
• blog.ippon.tech/spark-kafka-achieving-zero-data-loss/
2/2Spark +
Kakfa
problem
s
Micro-batches
Spark streaming processes events in micro-batches
• Impact on latency
• Spark Streaming micro-batches → hard to achieve sub-second latency
• See spark.apache.org/docs/latest/streaming-programming-guide.html#task-launching-overheads
• Total latency of the system = sum of the latencies of each stage
• In this use case, events are independent from each other - no need for windowing computation → a
real streaming framework would be more appropriate
• Impact on memory usage
• Kafka+Spark using the direct approach = 1 RDD partition per Kafka partition
• If you start the Spark with lots of unprocessed data in Kafka, RDD partitions can exceed the size of
the memory
Spark +
Kakfa
problem
s
Allocation of resources in Spark
• With Spark Streaming, resources (CPU & memory) are allocated per job
• Resources are allocated when the job is submitted and cannot be updated on the
fly
• Have to allocate 1 core to the Driver of the job → unused resource
• Have to allocate extra resources to each job to handle variations in traffic →
unused resources
• For peak periods, easy to add new Spark Workers but jobs have to restarted
• Idea to be tested:
• Over allocation of real resources, e.g let Spark know it has 6 cores on a 4-cores server
Spark +
Kakfa
problem
s
Python code in production
• Data Scientists know Python →They can contribute
• But shipping code written by Data Scientists is not ideal
• Need production-grade code (error handling, logging…)
• Code is less tested than Scala code
• Harder to deploy than a JAR file → PythonVirtual Environments
• blog.cloudera.com/blog/2015/09/how-to-prepare-your-apache-
hadoop-cluster-for-pyspark-jobs/
Spark +
Kakfa
problem
s
Resilience of Spark Jobs
• Spark Streaming application = 1 Driver + 1 Application
• Application = N Executors
• If an Executor dies → restarted (seamless)
• If the Driver dies, the whole Application must be restarted
• Scala/Java jobs → “supervised” mode
• Python jobs → not supported with Spark Standalone
Spark +
Kakfa
problem
s
Writing to Kafka
• Spark Streaming comes with a library to read from Kafka
but none to write to Kafka!
• Flink or Kafka Streams do that out-of-the-box
• Cloudera provides an open-source library:
• github.com/cloudera/spark-kafka-writer
• (Has been removed by now!)
Spark +
Kakfa
problem
s
Spark 2.x + Kafka?
• New API: Structured Streaming
• Still ALPHA in 2.1
• Support is improving…
Kafka Streams
Kafka Streams
docs.confluent.io/3.2.0/streams/index.html
• “powerful, easy-to-use library for building highly scalable, fault-tolerant, distributed
stream processing applications on top of Apache Kafka”
• Works with Kafka ≥ 0.10
• No cluster needed: Kafka is the cluster manager (consumer groups)
• Natively consumes messages from Kafka (and handles offsets)
• Natively pushes produced messages to Kafka
• Processes messages one at a time → low latency, low footprint
• Java library (works best in Java, can work in Scala)
• Read text from a topic
• Process the text:
• Only keep messages containing the “a” character
• Capitalize the text
• Output the result to another topic
Quick Example 1/3
• Create a regular Java application (with a main)
• Add the Kafka Streams dependency:
<dependency>

<groupId>org.apache.kafka</groupId>

<artifactId>kafka-streams</artifactId>

<version>0.10.2.1</version>

</dependency>
• Add the Kafka Streams code (next slide)
• Build and run the JAR
Quick Example 2/3
Properties props = new Properties();

props.put(StreamsConfig.APPLICATION_ID_CONFIG, “text-transformer");

props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");

props.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, "localhost:2181");

props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, "8");

props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");


KStreamBuilder builder = new KStreamBuilder();


builder.stream(Serdes.String(), Serdes.String(), "text-input")

.filter((key, value) -> value.contains("a"))

.mapValues(text -> text.toUpperCase())

.to(Serdes.String(), Serdes.String(), "text-output");



KafkaStreams streams = new KafkaStreams(builder, props);

streams.start();
Quick Example 3/3
• Application ID = Kafka
consumer group
• Threads for parallel processing
(relates to partitions)
• Topic to read from + key/
value deserializers
• Transformations: map, filter…
• Topic to write to + key/value
serializers
ProcessorTopology
• Need to define one or more processor
topologies
• Two APIs to define topologies:
• DSL (preferred): map(), filter(), to()…
• Processor API (low level): implement the
Processor interface then connect source processors,
stream processors and sink processors together
Parallelism (one process)
• Kafka Streams creates 1 task per partition in the input topic
• A task is an instance of the topology
• Tasks are independent from each other
• The number of processing threads is determined by the
developer
props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, "8");
• Tasks are distributed between threads
1/2
Parallelism (one process)
• 3 partitions → 3 tasks
• The tasks are distributed
to the 2 threads
2/2
Parallelism (multiple processes)
• With multiple processes (multiple instances of the JVM),
each consumer process is assigned a portion of the
partitions
→ Consumer group
• Reassignment of partitions occurs:
• When a new consumer joins the group
• When a consumer dies
→Tasks are created/deleted accordingly
1/2
Parallelism (multiple processes)
• Partitions are assigned to
2 consumers
• 3 partitions → 3 tasks
(as before)
• Each thread has one task
→ Improved parallelism
2/2
KStream vs, KTable
KStream is a stream of records
• Records are independent from each other
• (Do not use log compaction)
Example:


KStreamBuilder builder = new KStreamBuilder();

KStream<String, String> stream = builder.stream(Serdes.String(),
Serdes.String(), "input-topic");
Example (inspired from the
documentation):
• Sum values as records arrive
• Records:
• (alice, 1) = 1
• (charlie, 1) = 2
• (alice, 3) = 5
• → Adds to (alice, 1)
KStream vs, KTable
KTable is a change log stream
• New records with the same key are an
update of previously received records
for the same key
• Keys are required
• Requires a state store
Example:
KStreamBuilder builder = new KStreamBuilder();

KTable<String, String> table = builder.table(Serdes.String(),
Serdes.String(), "input-topic", "store-name");
Example (inspired from the
documentation):
• Sum values as records arrive
• Records:
• (alice, 1) = 1
• (charlie, 1) = 2
• (alice, 3) = 4
• → Replaces (alice, 1)
map / mapValues Apply a transformation to the records
flatMap / flatMapValues
Apply a transformation to the records and create 0/1/n
records per input record
filter Apply a predicate
groupBy / groupByKey Group the records. Followed by a call to reduce, aggregate or
count
join / leftJoin / outerJoin Joins 2 KStreams / KTables
to Writes the records to a Kafka topic
through
Writes the records to a Kafka topic and builds a new
KStream / KTable from this topic
API 1/2
State Stores
• Some operations require to store a state
• KTables (by definition, they need to keep previously received values)
• Aggregations (groupBy / groupByKey)
• Windowing operations
• One state store per task (RocksDB or a hash map)
• Backed by internal topics for recovery → fault tolerance
• Can be queried internally
Aggregations
• Aggregations are performed by keys
• Records with a null key are ignored
• Repartition the data using an internal topic if need be
• Aggregations can be windowed or non-windowed
• Aggregating a KStream generates a KTable
builder.stream(dummySerde, accountSerde, "accounts")

.map((_, account) => (account.accountId, account))

.to(stringSerde, accountSerde, "accounts-by-id")



builder.stream(stringSerde, accountSerde, "accounts-by-id")
Joins
• Types of joins: inner / outer / left join
• Operands:
• KTable to KTable → Non-windowed
• KStream to KTable → Non-windowed
• KStream to KStream → Windowed (to avoid an infinitely growing result)
• Data must be co-partitioned
• Repartition using an internal topic if need be
Deploying and Running
• Assemble a JAR (maven-shade plugin)
• Run the JAR as a regular Java application (java -cp …)
• Make sure all instances are in the same consumer group
(same application ID)
Running
Topic “AUTH-JSON” with 4 partitions
Application ID = “auth-converter”
Log on the first instance:
11:00:22,331 ...AbstractCoordinator - Successfully joined group auth-converter with generation 1
11:00:22,332 ...ConsumerCoordinator - Setting newly assigned partitions [AUTH_JSON-2, AUTH_JSON-1,
AUTH_JSON-3, AUTH_JSON-0] for group auth-converter
Running - Scaling up
Start a new instance:
Log on the first instance:
11:01:31,402 ...AbstractCoordinator - Successfully joined group auth-converter with generation 2
11:01:31,404 ...ConsumerCoordinator - Setting newly assigned partitions [AUTH_JSON-2, AUTH_JSON-3] for group
auth-converter
11:01:31,390 ...ConsumerCoordinator - Revoking previously assigned partitions [AUTH_JSON-2, AUTH_JSON-1,
AUTH_JSON-3, AUTH_JSON-0] for group auth-converter
11:01:31,401 ...ConsumerCoordinator - Setting newly assigned partitions [AUTH_JSON-1, AUTH_JSON-0] for group
auth-converter
Running - Scaling down
Kill one of the instances
Log on the remaining instance:
11:02:13,410 ...ConsumerCoordinator - Revoking previously assigned partitions [AUTH_JSON-1, AUTH_JSON-0] for
group auth-converter
11:02:13,415 ...ConsumerCoordinator - Setting newly assigned partitions [AUTH_JSON-2, AUTH_JSON-1,
AUTH_JSON-3, AUTH_JSON-0] for group auth-converter
Delivery semantics
• At least once
• No messages will be lost
• Messages can be processed a second time when failure happens
→ Make your system idempotent
• Exactly once planned for Kafka 0.11
• KIP-98 - Exactly Once Delivery andTransactional Messaging
• KIP-129: Streams Exactly-Once Semantics
Kafka Streams on this project
Migration
• Conversion of Spark / Scala code
• Upgraded from Scala 2.10 to 2.11 and enabled the -Xexperimental flag of
the Scala compiler so that Scala lambdas are converted into Java lambdas
(SAM support)
• Removed lots of specific code to read from / write to Kafka (supported
out-of-the-box with Kafka Streams)
• API similar to the RDD API →Very straightforward conversion (no need to
call foreachRDD, so even better!)
• Conversion of Spark / Python code: not attempted
Metrics
• Kafka Streams doesn’t have a UI to display metrics (e.g.
number of records processed)
• Used Dropwizard Metrics (metrics.dropwizard.io)
• Java API to calculate metrics and send them to various sinks
• Used InfluxDB to store the metrics
• Graphite compatible
• Used Grafana to display the metrics as graphs
Metrics aggregation
• Each instance reports its own metrics → Need to aggregate metrics
• Specific reporter to send Dropwizard Metrics to a Kafka topic
• Kafka topic to collect metrics
• 1 partition
• Key = instance ID (e.g. app-1, app-2…)
• Value = monotonic metric
• Kafka Streams app to aggregate metrics
• Input is a KTable (new values replace previous values)
• Send aggregated metrics to InfluxDB
Kafka Streams app to aggregate metrics
KTable<String, CounterMetric> metricsStream = builder.table(appIdSerde, metricSerde, "metrics", "raw-metrics");

KStream<String, CounterMetric> metricValueStream = metricsStream

.groupBy((key, value) -> new KeyValue<>(value.getName(), value), metricNameSerde, metricSerde)

.reduce(CounterMetric::add, CounterMetric::subtract, "aggregates")

.toStream()
.to(metricNameSerde, metricSerde, "metrics-agg");



// --- Second topology



GraphiteReporter graphite = GraphiteReporter.builder()

.hostname("localhost")

.port(2003)

.build();



KStream<String, CounterMetric> aggMetricsStream = builder.stream(metricNameSerde, metricSerde, "metrics-agg");

aggMetricsStream.foreach((key, metric) -> graphite.send(metric));
Metrics
Aggregated metric
Send data into Kafka (1M records)
Start consumer 1
Start consumer 2
Aggregated metric (from consumers 1 and 2)
Stop consumer 2
Delta = records processed twice
Results
Pros
• Simpler code (no manual handling of offsets)
• Simpler packaging (no dependencies to exclude, less
dependency version conflicts)
• Much lower latency: from seconds to milliseconds
• Reduced memory footprint
• Easier scaling
• Improved stability when restarting the application
Cons
• No UI
• No centralized logs → Use ELK or equivalent…
• No centralized metrics → Aggregate metrics
• Have to use an intermediate topic if grouping the
data by a value that is not the key
Summary
&
Conclusion
Summary
• Very easy to build pipelines on top of Kafka
• Great fit for micro-services
• Compared to Spark Streaming:
• Better for realtime apps than Spark Streaming
• Lower latency, lower memory footprint, easier scaling
• Lower level: good for prod, lacks a UI for dev
• Compared to a standard Kafka consumer:
• Higher level: faster to build a sophisticated app
• Less control for very fine-grained consumption
Thank you!
@aseigneurin

More Related Content

What's hot

Kafka Summit NYC 2017 - Scalable Real-Time Complex Event Processing @ Uber
Kafka Summit NYC 2017 - Scalable Real-Time Complex Event Processing @ UberKafka Summit NYC 2017 - Scalable Real-Time Complex Event Processing @ Uber
Kafka Summit NYC 2017 - Scalable Real-Time Complex Event Processing @ Uber
confluent
 
A Marriage of Lambda and Kappa: Supporting Iterative Development of an Event ...
A Marriage of Lambda and Kappa: Supporting Iterative Development of an Event ...A Marriage of Lambda and Kappa: Supporting Iterative Development of an Event ...
A Marriage of Lambda and Kappa: Supporting Iterative Development of an Event ...
confluent
 

What's hot (20)

Kafka Summit NYC 2017 - Apache Kafka in the Enterprise: What if it Fails?
Kafka Summit NYC 2017 - Apache Kafka in the Enterprise: What if it Fails? Kafka Summit NYC 2017 - Apache Kafka in the Enterprise: What if it Fails?
Kafka Summit NYC 2017 - Apache Kafka in the Enterprise: What if it Fails?
 
Kafka Streams for Java enthusiasts
Kafka Streams for Java enthusiastsKafka Streams for Java enthusiasts
Kafka Streams for Java enthusiasts
 
Building Large-Scale Stream Infrastructures Across Multiple Data Centers with...
Building Large-Scale Stream Infrastructures Across Multiple Data Centers with...Building Large-Scale Stream Infrastructures Across Multiple Data Centers with...
Building Large-Scale Stream Infrastructures Across Multiple Data Centers with...
 
Performance Tuning RocksDB for Kafka Streams’ State Stores
Performance Tuning RocksDB for Kafka Streams’ State StoresPerformance Tuning RocksDB for Kafka Streams’ State Stores
Performance Tuning RocksDB for Kafka Streams’ State Stores
 
Kafka Summit NYC 2017 - Scalable Real-Time Complex Event Processing @ Uber
Kafka Summit NYC 2017 - Scalable Real-Time Complex Event Processing @ UberKafka Summit NYC 2017 - Scalable Real-Time Complex Event Processing @ Uber
Kafka Summit NYC 2017 - Scalable Real-Time Complex Event Processing @ Uber
 
ksqlDB: A Stream-Relational Database System
ksqlDB: A Stream-Relational Database SystemksqlDB: A Stream-Relational Database System
ksqlDB: A Stream-Relational Database System
 
Deploying Confluent Platform for Production
Deploying Confluent Platform for ProductionDeploying Confluent Platform for Production
Deploying Confluent Platform for Production
 
Building Realtim Data Pipelines with Kafka Connect and Spark Streaming
Building Realtim Data Pipelines with Kafka Connect and Spark StreamingBuilding Realtim Data Pipelines with Kafka Connect and Spark Streaming
Building Realtim Data Pipelines with Kafka Connect and Spark Streaming
 
Introduction to apache kafka, confluent and why they matter
Introduction to apache kafka, confluent and why they matterIntroduction to apache kafka, confluent and why they matter
Introduction to apache kafka, confluent and why they matter
 
Hadoop summit - Scaling Uber’s Real-Time Infra for Trillion Events per Day
Hadoop summit - Scaling Uber’s Real-Time Infra for  Trillion Events per DayHadoop summit - Scaling Uber’s Real-Time Infra for  Trillion Events per Day
Hadoop summit - Scaling Uber’s Real-Time Infra for Trillion Events per Day
 
How to Lock Down Apache Kafka and Keep Your Streams Safe
How to Lock Down Apache Kafka and Keep Your Streams SafeHow to Lock Down Apache Kafka and Keep Your Streams Safe
How to Lock Down Apache Kafka and Keep Your Streams Safe
 
A Marriage of Lambda and Kappa: Supporting Iterative Development of an Event ...
A Marriage of Lambda and Kappa: Supporting Iterative Development of an Event ...A Marriage of Lambda and Kappa: Supporting Iterative Development of an Event ...
A Marriage of Lambda and Kappa: Supporting Iterative Development of an Event ...
 
Flink at netflix paypal speaker series
Flink at netflix   paypal speaker seriesFlink at netflix   paypal speaker series
Flink at netflix paypal speaker series
 
What's the time? ...and why? (Mattias Sax, Confluent) Kafka Summit SF 2019
What's the time? ...and why? (Mattias Sax, Confluent) Kafka Summit SF 2019What's the time? ...and why? (Mattias Sax, Confluent) Kafka Summit SF 2019
What's the time? ...and why? (Mattias Sax, Confluent) Kafka Summit SF 2019
 
Cross the streams thanks to Kafka and Flink (Christophe Philemotte, Digazu) K...
Cross the streams thanks to Kafka and Flink (Christophe Philemotte, Digazu) K...Cross the streams thanks to Kafka and Flink (Christophe Philemotte, Digazu) K...
Cross the streams thanks to Kafka and Flink (Christophe Philemotte, Digazu) K...
 
Siphon - Near Real Time Databus Using Kafka, Eric Boyd, Nitin Kumar
Siphon - Near Real Time Databus Using Kafka, Eric Boyd, Nitin KumarSiphon - Near Real Time Databus Using Kafka, Eric Boyd, Nitin Kumar
Siphon - Near Real Time Databus Using Kafka, Eric Boyd, Nitin Kumar
 
Spring Kafka beyond the basics - Lessons learned on our Kafka journey (Tim va...
Spring Kafka beyond the basics - Lessons learned on our Kafka journey (Tim va...Spring Kafka beyond the basics - Lessons learned on our Kafka journey (Tim va...
Spring Kafka beyond the basics - Lessons learned on our Kafka journey (Tim va...
 
Kafka Summit NYC 2017 - Building Advanced Streaming Applications using the La...
Kafka Summit NYC 2017 - Building Advanced Streaming Applications using the La...Kafka Summit NYC 2017 - Building Advanced Streaming Applications using the La...
Kafka Summit NYC 2017 - Building Advanced Streaming Applications using the La...
 
Kafka Summit NYC 2017 - Data Processing at LinkedIn with Apache Kafka
Kafka Summit NYC 2017 - Data Processing at LinkedIn with Apache KafkaKafka Summit NYC 2017 - Data Processing at LinkedIn with Apache Kafka
Kafka Summit NYC 2017 - Data Processing at LinkedIn with Apache Kafka
 
Kafka Summit NYC 2017 - Cloud Native Data Streaming Microservices with Spring...
Kafka Summit NYC 2017 - Cloud Native Data Streaming Microservices with Spring...Kafka Summit NYC 2017 - Cloud Native Data Streaming Microservices with Spring...
Kafka Summit NYC 2017 - Cloud Native Data Streaming Microservices with Spring...
 

Similar to Kafka Summit NYC 2017 Introduction to Kafka Streams with a Real-life Example

Building High-Throughput, Low-Latency Pipelines in Kafka
Building High-Throughput, Low-Latency Pipelines in KafkaBuilding High-Throughput, Low-Latency Pipelines in Kafka
Building High-Throughput, Low-Latency Pipelines in Kafka
confluent
 

Similar to Kafka Summit NYC 2017 Introduction to Kafka Streams with a Real-life Example (20)

Lessons Learned: Using Spark and Microservices
Lessons Learned: Using Spark and MicroservicesLessons Learned: Using Spark and Microservices
Lessons Learned: Using Spark and Microservices
 
Apache Kafka - Scalable Message-Processing and more !
Apache Kafka - Scalable Message-Processing and more !Apache Kafka - Scalable Message-Processing and more !
Apache Kafka - Scalable Message-Processing and more !
 
Fundamentals of Apache Kafka
Fundamentals of Apache KafkaFundamentals of Apache Kafka
Fundamentals of Apache Kafka
 
Real time Analytics with Apache Kafka and Apache Spark
Real time Analytics with Apache Kafka and Apache SparkReal time Analytics with Apache Kafka and Apache Spark
Real time Analytics with Apache Kafka and Apache Spark
 
Apache kafka
Apache kafkaApache kafka
Apache kafka
 
Scala and Spark are Ideal for Big Data
Scala and Spark are Ideal for Big DataScala and Spark are Ideal for Big Data
Scala and Spark are Ideal for Big Data
 
Global Big Data Conference Sept 2014 AWS Kinesis Spark Streaming Approximatio...
Global Big Data Conference Sept 2014 AWS Kinesis Spark Streaming Approximatio...Global Big Data Conference Sept 2014 AWS Kinesis Spark Streaming Approximatio...
Global Big Data Conference Sept 2014 AWS Kinesis Spark Streaming Approximatio...
 
Productionizing Spark and the REST Job Server- Evan Chan
Productionizing Spark and the REST Job Server- Evan ChanProductionizing Spark and the REST Job Server- Evan Chan
Productionizing Spark and the REST Job Server- Evan Chan
 
Apache Spark Tutorial
Apache Spark TutorialApache Spark Tutorial
Apache Spark Tutorial
 
Productionizing Spark and the Spark Job Server
Productionizing Spark and the Spark Job ServerProductionizing Spark and the Spark Job Server
Productionizing Spark and the Spark Job Server
 
East Bay Java User Group Oct 2014 Spark Streaming Kinesis Machine Learning
 East Bay Java User Group Oct 2014 Spark Streaming Kinesis Machine Learning East Bay Java User Group Oct 2014 Spark Streaming Kinesis Machine Learning
East Bay Java User Group Oct 2014 Spark Streaming Kinesis Machine Learning
 
Building High-Throughput, Low-Latency Pipelines in Kafka
Building High-Throughput, Low-Latency Pipelines in KafkaBuilding High-Throughput, Low-Latency Pipelines in Kafka
Building High-Throughput, Low-Latency Pipelines in Kafka
 
Introduction to Kafka Streams Presentation
Introduction to Kafka Streams PresentationIntroduction to Kafka Streams Presentation
Introduction to Kafka Streams Presentation
 
Fundamentals and Architecture of Apache Kafka
Fundamentals and Architecture of Apache KafkaFundamentals and Architecture of Apache Kafka
Fundamentals and Architecture of Apache Kafka
 
Kafka Explainaton
Kafka ExplainatonKafka Explainaton
Kafka Explainaton
 
Spark - The Ultimate Scala Collections by Martin Odersky
Spark - The Ultimate Scala Collections by Martin OderskySpark - The Ultimate Scala Collections by Martin Odersky
Spark - The Ultimate Scala Collections by Martin Odersky
 
Flink Forward San Francisco 2019: Moving from Lambda and Kappa Architectures ...
Flink Forward San Francisco 2019: Moving from Lambda and Kappa Architectures ...Flink Forward San Francisco 2019: Moving from Lambda and Kappa Architectures ...
Flink Forward San Francisco 2019: Moving from Lambda and Kappa Architectures ...
 
14th Athens Big Data Meetup - Landoop Workshop - Apache Kafka Entering The St...
14th Athens Big Data Meetup - Landoop Workshop - Apache Kafka Entering The St...14th Athens Big Data Meetup - Landoop Workshop - Apache Kafka Entering The St...
14th Athens Big Data Meetup - Landoop Workshop - Apache Kafka Entering The St...
 
Tuning and Monitoring Deep Learning on Apache Spark
Tuning and Monitoring Deep Learning on Apache SparkTuning and Monitoring Deep Learning on Apache Spark
Tuning and Monitoring Deep Learning on Apache Spark
 
DataConf.TW2018: Develop Kafka Streams Application on Your Laptop
DataConf.TW2018: Develop Kafka Streams Application on Your LaptopDataConf.TW2018: Develop Kafka Streams Application on Your Laptop
DataConf.TW2018: Develop Kafka Streams Application on Your Laptop
 

More from confluent

More from confluent (20)

Catch the Wave: SAP Event-Driven and Data Streaming for the Intelligence Ente...
Catch the Wave: SAP Event-Driven and Data Streaming for the Intelligence Ente...Catch the Wave: SAP Event-Driven and Data Streaming for the Intelligence Ente...
Catch the Wave: SAP Event-Driven and Data Streaming for the Intelligence Ente...
 
Santander Stream Processing with Apache Flink
Santander Stream Processing with Apache FlinkSantander Stream Processing with Apache Flink
Santander Stream Processing with Apache Flink
 
Unlocking the Power of IoT: A comprehensive approach to real-time insights
Unlocking the Power of IoT: A comprehensive approach to real-time insightsUnlocking the Power of IoT: A comprehensive approach to real-time insights
Unlocking the Power of IoT: A comprehensive approach to real-time insights
 
Workshop híbrido: Stream Processing con Flink
Workshop híbrido: Stream Processing con FlinkWorkshop híbrido: Stream Processing con Flink
Workshop híbrido: Stream Processing con Flink
 
Industry 4.0: Building the Unified Namespace with Confluent, HiveMQ and Spark...
Industry 4.0: Building the Unified Namespace with Confluent, HiveMQ and Spark...Industry 4.0: Building the Unified Namespace with Confluent, HiveMQ and Spark...
Industry 4.0: Building the Unified Namespace with Confluent, HiveMQ and Spark...
 
AWS Immersion Day Mapfre - Confluent
AWS Immersion Day Mapfre   -   ConfluentAWS Immersion Day Mapfre   -   Confluent
AWS Immersion Day Mapfre - Confluent
 
Eventos y Microservicios - Santander TechTalk
Eventos y Microservicios - Santander TechTalkEventos y Microservicios - Santander TechTalk
Eventos y Microservicios - Santander TechTalk
 
Q&A with Confluent Experts: Navigating Networking in Confluent Cloud
Q&A with Confluent Experts: Navigating Networking in Confluent CloudQ&A with Confluent Experts: Navigating Networking in Confluent Cloud
Q&A with Confluent Experts: Navigating Networking in Confluent Cloud
 
Citi TechTalk Session 2: Kafka Deep Dive
Citi TechTalk Session 2: Kafka Deep DiveCiti TechTalk Session 2: Kafka Deep Dive
Citi TechTalk Session 2: Kafka Deep Dive
 
Build real-time streaming data pipelines to AWS with Confluent
Build real-time streaming data pipelines to AWS with ConfluentBuild real-time streaming data pipelines to AWS with Confluent
Build real-time streaming data pipelines to AWS with Confluent
 
Q&A with Confluent Professional Services: Confluent Service Mesh
Q&A with Confluent Professional Services: Confluent Service MeshQ&A with Confluent Professional Services: Confluent Service Mesh
Q&A with Confluent Professional Services: Confluent Service Mesh
 
Citi Tech Talk: Event Driven Kafka Microservices
Citi Tech Talk: Event Driven Kafka MicroservicesCiti Tech Talk: Event Driven Kafka Microservices
Citi Tech Talk: Event Driven Kafka Microservices
 
Confluent & GSI Webinars series - Session 3
Confluent & GSI Webinars series - Session 3Confluent & GSI Webinars series - Session 3
Confluent & GSI Webinars series - Session 3
 
Citi Tech Talk: Messaging Modernization
Citi Tech Talk: Messaging ModernizationCiti Tech Talk: Messaging Modernization
Citi Tech Talk: Messaging Modernization
 
Citi Tech Talk: Data Governance for streaming and real time data
Citi Tech Talk: Data Governance for streaming and real time dataCiti Tech Talk: Data Governance for streaming and real time data
Citi Tech Talk: Data Governance for streaming and real time data
 
Confluent & GSI Webinars series: Session 2
Confluent & GSI Webinars series: Session 2Confluent & GSI Webinars series: Session 2
Confluent & GSI Webinars series: Session 2
 
Data In Motion Paris 2023
Data In Motion Paris 2023Data In Motion Paris 2023
Data In Motion Paris 2023
 
Confluent Partner Tech Talk with Synthesis
Confluent Partner Tech Talk with SynthesisConfluent Partner Tech Talk with Synthesis
Confluent Partner Tech Talk with Synthesis
 
The Future of Application Development - API Days - Melbourne 2023
The Future of Application Development - API Days - Melbourne 2023The Future of Application Development - API Days - Melbourne 2023
The Future of Application Development - API Days - Melbourne 2023
 
The Playful Bond Between REST And Data Streams
The Playful Bond Between REST And Data StreamsThe Playful Bond Between REST And Data Streams
The Playful Bond Between REST And Data Streams
 

Recently uploaded

Abortion Pill Prices Tembisa [(+27832195400*)] 🏥 Women's Abortion Clinic in T...
Abortion Pill Prices Tembisa [(+27832195400*)] 🏥 Women's Abortion Clinic in T...Abortion Pill Prices Tembisa [(+27832195400*)] 🏥 Women's Abortion Clinic in T...
Abortion Pill Prices Tembisa [(+27832195400*)] 🏥 Women's Abortion Clinic in T...
Medical / Health Care (+971588192166) Mifepristone and Misoprostol tablets 200mg
 
%+27788225528 love spells in Knoxville Psychic Readings, Attraction spells,Br...
%+27788225528 love spells in Knoxville Psychic Readings, Attraction spells,Br...%+27788225528 love spells in Knoxville Psychic Readings, Attraction spells,Br...
%+27788225528 love spells in Knoxville Psychic Readings, Attraction spells,Br...
masabamasaba
 
Large-scale Logging Made Easy: Meetup at Deutsche Bank 2024
Large-scale Logging Made Easy: Meetup at Deutsche Bank 2024Large-scale Logging Made Easy: Meetup at Deutsche Bank 2024
Large-scale Logging Made Easy: Meetup at Deutsche Bank 2024
VictoriaMetrics
 
%+27788225528 love spells in Colorado Springs Psychic Readings, Attraction sp...
%+27788225528 love spells in Colorado Springs Psychic Readings, Attraction sp...%+27788225528 love spells in Colorado Springs Psychic Readings, Attraction sp...
%+27788225528 love spells in Colorado Springs Psychic Readings, Attraction sp...
masabamasaba
 

Recently uploaded (20)

tonesoftg
tonesoftgtonesoftg
tonesoftg
 
WSO2CON 2024 - API Management Usage at La Poste and Its Impact on Business an...
WSO2CON 2024 - API Management Usage at La Poste and Its Impact on Business an...WSO2CON 2024 - API Management Usage at La Poste and Its Impact on Business an...
WSO2CON 2024 - API Management Usage at La Poste and Its Impact on Business an...
 
Abortion Pill Prices Tembisa [(+27832195400*)] 🏥 Women's Abortion Clinic in T...
Abortion Pill Prices Tembisa [(+27832195400*)] 🏥 Women's Abortion Clinic in T...Abortion Pill Prices Tembisa [(+27832195400*)] 🏥 Women's Abortion Clinic in T...
Abortion Pill Prices Tembisa [(+27832195400*)] 🏥 Women's Abortion Clinic in T...
 
%in kaalfontein+277-882-255-28 abortion pills for sale in kaalfontein
%in kaalfontein+277-882-255-28 abortion pills for sale in kaalfontein%in kaalfontein+277-882-255-28 abortion pills for sale in kaalfontein
%in kaalfontein+277-882-255-28 abortion pills for sale in kaalfontein
 
VTU technical seminar 8Th Sem on Scikit-learn
VTU technical seminar 8Th Sem on Scikit-learnVTU technical seminar 8Th Sem on Scikit-learn
VTU technical seminar 8Th Sem on Scikit-learn
 
%+27788225528 love spells in Knoxville Psychic Readings, Attraction spells,Br...
%+27788225528 love spells in Knoxville Psychic Readings, Attraction spells,Br...%+27788225528 love spells in Knoxville Psychic Readings, Attraction spells,Br...
%+27788225528 love spells in Knoxville Psychic Readings, Attraction spells,Br...
 
MarTech Trend 2024 Book : Marketing Technology Trends (2024 Edition) How Data...
MarTech Trend 2024 Book : Marketing Technology Trends (2024 Edition) How Data...MarTech Trend 2024 Book : Marketing Technology Trends (2024 Edition) How Data...
MarTech Trend 2024 Book : Marketing Technology Trends (2024 Edition) How Data...
 
Large-scale Logging Made Easy: Meetup at Deutsche Bank 2024
Large-scale Logging Made Easy: Meetup at Deutsche Bank 2024Large-scale Logging Made Easy: Meetup at Deutsche Bank 2024
Large-scale Logging Made Easy: Meetup at Deutsche Bank 2024
 
WSO2Con2024 - WSO2's IAM Vision: Identity-Led Digital Transformation
WSO2Con2024 - WSO2's IAM Vision: Identity-Led Digital TransformationWSO2Con2024 - WSO2's IAM Vision: Identity-Led Digital Transformation
WSO2Con2024 - WSO2's IAM Vision: Identity-Led Digital Transformation
 
What Goes Wrong with Language Definitions and How to Improve the Situation
What Goes Wrong with Language Definitions and How to Improve the SituationWhat Goes Wrong with Language Definitions and How to Improve the Situation
What Goes Wrong with Language Definitions and How to Improve the Situation
 
%in Midrand+277-882-255-28 abortion pills for sale in midrand
%in Midrand+277-882-255-28 abortion pills for sale in midrand%in Midrand+277-882-255-28 abortion pills for sale in midrand
%in Midrand+277-882-255-28 abortion pills for sale in midrand
 
Artyushina_Guest lecture_YorkU CS May 2024.pptx
Artyushina_Guest lecture_YorkU CS May 2024.pptxArtyushina_Guest lecture_YorkU CS May 2024.pptx
Artyushina_Guest lecture_YorkU CS May 2024.pptx
 
%in ivory park+277-882-255-28 abortion pills for sale in ivory park
%in ivory park+277-882-255-28 abortion pills for sale in ivory park %in ivory park+277-882-255-28 abortion pills for sale in ivory park
%in ivory park+277-882-255-28 abortion pills for sale in ivory park
 
WSO2CON2024 - It's time to go Platformless
WSO2CON2024 - It's time to go PlatformlessWSO2CON2024 - It's time to go Platformless
WSO2CON2024 - It's time to go Platformless
 
%+27788225528 love spells in Colorado Springs Psychic Readings, Attraction sp...
%+27788225528 love spells in Colorado Springs Psychic Readings, Attraction sp...%+27788225528 love spells in Colorado Springs Psychic Readings, Attraction sp...
%+27788225528 love spells in Colorado Springs Psychic Readings, Attraction sp...
 
WSO2CON 2024 Slides - Open Source to SaaS
WSO2CON 2024 Slides - Open Source to SaaSWSO2CON 2024 Slides - Open Source to SaaS
WSO2CON 2024 Slides - Open Source to SaaS
 
Architecture decision records - How not to get lost in the past
Architecture decision records - How not to get lost in the pastArchitecture decision records - How not to get lost in the past
Architecture decision records - How not to get lost in the past
 
AI & Machine Learning Presentation Template
AI & Machine Learning Presentation TemplateAI & Machine Learning Presentation Template
AI & Machine Learning Presentation Template
 
Announcing Codolex 2.0 from GDK Software
Announcing Codolex 2.0 from GDK SoftwareAnnouncing Codolex 2.0 from GDK Software
Announcing Codolex 2.0 from GDK Software
 
WSO2CON 2024 - Building the API First Enterprise – Running an API Program, fr...
WSO2CON 2024 - Building the API First Enterprise – Running an API Program, fr...WSO2CON 2024 - Building the API First Enterprise – Running an API Program, fr...
WSO2CON 2024 - Building the API First Enterprise – Running an API Program, fr...
 

Kafka Summit NYC 2017 Introduction to Kafka Streams with a Real-life Example

  • 1. MICROSERVICES WITH KAFKA — AN INTRODUCTION TO KAFKA STREAMS APIWITH A REAL-LIFE EXAMPLE Alexis Seigneurin
  • 2. Who I am • Software engineer for 15+ years • Consultant at Ippon USA, previously at Ippon France • Favorite subjects: Spark, Kafka, Machine Learning, Scala • Spark certified • @aseigneurin
  • 3. • 200+ software engineers in France, the US and Australia • In the US: NYC, DC, RVA • Digital, Big Data and Cloud applications • Java & Agile expertise • Open-source projects: JHipster,Tatami, etc. • @ipponusa
  • 5. The project • Analyze records from customers → Send notifications • High volume of data • 25 millions records per day in average + with seasonal peaks • Need to keep at least 60 days of history = 1.5 Billion records • Need an hybrid platform • Batch processing for some types of analysis • Streaming for other analyses • Hybrid team • Data Scientists: more familiar with Python • Software Engineers: Java
  • 6. Architecture - Real time platform • New detectors are implemented by Data Scientists all the time • Need the implementations to be independent from each other • One Spark Streaming job per detector • Microservice-inspired architecture • Diamond-shaped • Upstream jobs are written in Scala • Core is made of multiple Python jobs, one per detector • Downstream jobs are written in Scala • Plumbing between the jobs → Kafka 1/2
  • 7. Architecture - Real time platform 2/2
  • 8. Modularity • One Spark job per detector • Hot deployments: can roll out new detectors (= new jobs) without stopping existing jobs • Can roll out updated code without affecting other jobs • Able to measure the resources consumed by a single job • Shared services are provided by upstream and downstream jobs
  • 9. Consuming Kafka messages with Spark Streaming (and why you probably shouldn’t do it)
  • 10. Spark 1.x + Kafka? • Spark has become the de-facto processing framework • Provides APIs for multiple programming languages • Python → Data Scientists • Scala/Java → Software Engineers • Supports batch jobs and streaming jobs, incl. support for Kafka…
  • 11. Consuming from Kafka • Connecting Spark to Kafka, 2 methods: • Receiver-based approach: not ideal for parallelism • Direct approach: better for parallelism but have to deal with Kafka offsets Spark + Kakfa problem s
  • 12. Dealing with Kafka offsets • Default: consumes from the end of the Kafka topic (or the beginning) • Documentation → Use checkpoints • Tasks have to be Serializable (not always possible: dependent libraries) • Harder to deploy the application (classes are serialized) → run a new instance in parallel and kill the first one (harder to automate; messages consumed twice) • Requires a shared file system (HDFS, S3) → big latency on these FS that forces to increase the micro-batch interval 1/2Spark + Kakfa problem s
  • 13. Dealing with Kafka offsets • Dealing with Kafka offsets • Solution: deal with offsets in the Spark Streaming application • Write the offsets to a reliable storage: ZooKeeper, Kafka… • Write after processing the data • Read the offsets on startup (if no offsets, start from the end) • blog.ippon.tech/spark-kafka-achieving-zero-data-loss/ 2/2Spark + Kakfa problem s
  • 14. Micro-batches Spark streaming processes events in micro-batches • Impact on latency • Spark Streaming micro-batches → hard to achieve sub-second latency • See spark.apache.org/docs/latest/streaming-programming-guide.html#task-launching-overheads • Total latency of the system = sum of the latencies of each stage • In this use case, events are independent from each other - no need for windowing computation → a real streaming framework would be more appropriate • Impact on memory usage • Kafka+Spark using the direct approach = 1 RDD partition per Kafka partition • If you start the Spark with lots of unprocessed data in Kafka, RDD partitions can exceed the size of the memory Spark + Kakfa problem s
  • 15. Allocation of resources in Spark • With Spark Streaming, resources (CPU & memory) are allocated per job • Resources are allocated when the job is submitted and cannot be updated on the fly • Have to allocate 1 core to the Driver of the job → unused resource • Have to allocate extra resources to each job to handle variations in traffic → unused resources • For peak periods, easy to add new Spark Workers but jobs have to restarted • Idea to be tested: • Over allocation of real resources, e.g let Spark know it has 6 cores on a 4-cores server Spark + Kakfa problem s
  • 16. Python code in production • Data Scientists know Python →They can contribute • But shipping code written by Data Scientists is not ideal • Need production-grade code (error handling, logging…) • Code is less tested than Scala code • Harder to deploy than a JAR file → PythonVirtual Environments • blog.cloudera.com/blog/2015/09/how-to-prepare-your-apache- hadoop-cluster-for-pyspark-jobs/ Spark + Kakfa problem s
  • 17. Resilience of Spark Jobs • Spark Streaming application = 1 Driver + 1 Application • Application = N Executors • If an Executor dies → restarted (seamless) • If the Driver dies, the whole Application must be restarted • Scala/Java jobs → “supervised” mode • Python jobs → not supported with Spark Standalone Spark + Kakfa problem s
  • 18. Writing to Kafka • Spark Streaming comes with a library to read from Kafka but none to write to Kafka! • Flink or Kafka Streams do that out-of-the-box • Cloudera provides an open-source library: • github.com/cloudera/spark-kafka-writer • (Has been removed by now!) Spark + Kakfa problem s
  • 19. Spark 2.x + Kafka? • New API: Structured Streaming • Still ALPHA in 2.1 • Support is improving…
  • 21. Kafka Streams docs.confluent.io/3.2.0/streams/index.html • “powerful, easy-to-use library for building highly scalable, fault-tolerant, distributed stream processing applications on top of Apache Kafka” • Works with Kafka ≥ 0.10 • No cluster needed: Kafka is the cluster manager (consumer groups) • Natively consumes messages from Kafka (and handles offsets) • Natively pushes produced messages to Kafka • Processes messages one at a time → low latency, low footprint • Java library (works best in Java, can work in Scala)
  • 22. • Read text from a topic • Process the text: • Only keep messages containing the “a” character • Capitalize the text • Output the result to another topic Quick Example 1/3
  • 23. • Create a regular Java application (with a main) • Add the Kafka Streams dependency: <dependency>
 <groupId>org.apache.kafka</groupId>
 <artifactId>kafka-streams</artifactId>
 <version>0.10.2.1</version>
 </dependency> • Add the Kafka Streams code (next slide) • Build and run the JAR Quick Example 2/3
  • 24. Properties props = new Properties();
 props.put(StreamsConfig.APPLICATION_ID_CONFIG, “text-transformer");
 props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");
 props.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, "localhost:2181");
 props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, "8");
 props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); 
 KStreamBuilder builder = new KStreamBuilder(); 
 builder.stream(Serdes.String(), Serdes.String(), "text-input")
 .filter((key, value) -> value.contains("a"))
 .mapValues(text -> text.toUpperCase())
 .to(Serdes.String(), Serdes.String(), "text-output");
 
 KafkaStreams streams = new KafkaStreams(builder, props);
 streams.start(); Quick Example 3/3 • Application ID = Kafka consumer group • Threads for parallel processing (relates to partitions) • Topic to read from + key/ value deserializers • Transformations: map, filter… • Topic to write to + key/value serializers
  • 25. ProcessorTopology • Need to define one or more processor topologies • Two APIs to define topologies: • DSL (preferred): map(), filter(), to()… • Processor API (low level): implement the Processor interface then connect source processors, stream processors and sink processors together
  • 26. Parallelism (one process) • Kafka Streams creates 1 task per partition in the input topic • A task is an instance of the topology • Tasks are independent from each other • The number of processing threads is determined by the developer props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, "8"); • Tasks are distributed between threads 1/2
  • 27. Parallelism (one process) • 3 partitions → 3 tasks • The tasks are distributed to the 2 threads 2/2
  • 28. Parallelism (multiple processes) • With multiple processes (multiple instances of the JVM), each consumer process is assigned a portion of the partitions → Consumer group • Reassignment of partitions occurs: • When a new consumer joins the group • When a consumer dies →Tasks are created/deleted accordingly 1/2
  • 29. Parallelism (multiple processes) • Partitions are assigned to 2 consumers • 3 partitions → 3 tasks (as before) • Each thread has one task → Improved parallelism 2/2
  • 30. KStream vs, KTable KStream is a stream of records • Records are independent from each other • (Do not use log compaction) Example: 
 KStreamBuilder builder = new KStreamBuilder();
 KStream<String, String> stream = builder.stream(Serdes.String(), Serdes.String(), "input-topic"); Example (inspired from the documentation): • Sum values as records arrive • Records: • (alice, 1) = 1 • (charlie, 1) = 2 • (alice, 3) = 5 • → Adds to (alice, 1)
  • 31. KStream vs, KTable KTable is a change log stream • New records with the same key are an update of previously received records for the same key • Keys are required • Requires a state store Example: KStreamBuilder builder = new KStreamBuilder();
 KTable<String, String> table = builder.table(Serdes.String(), Serdes.String(), "input-topic", "store-name"); Example (inspired from the documentation): • Sum values as records arrive • Records: • (alice, 1) = 1 • (charlie, 1) = 2 • (alice, 3) = 4 • → Replaces (alice, 1)
  • 32. map / mapValues Apply a transformation to the records flatMap / flatMapValues Apply a transformation to the records and create 0/1/n records per input record filter Apply a predicate groupBy / groupByKey Group the records. Followed by a call to reduce, aggregate or count join / leftJoin / outerJoin Joins 2 KStreams / KTables to Writes the records to a Kafka topic through Writes the records to a Kafka topic and builds a new KStream / KTable from this topic API 1/2
  • 33. State Stores • Some operations require to store a state • KTables (by definition, they need to keep previously received values) • Aggregations (groupBy / groupByKey) • Windowing operations • One state store per task (RocksDB or a hash map) • Backed by internal topics for recovery → fault tolerance • Can be queried internally
  • 34. Aggregations • Aggregations are performed by keys • Records with a null key are ignored • Repartition the data using an internal topic if need be • Aggregations can be windowed or non-windowed • Aggregating a KStream generates a KTable builder.stream(dummySerde, accountSerde, "accounts")
 .map((_, account) => (account.accountId, account))
 .to(stringSerde, accountSerde, "accounts-by-id")
 
 builder.stream(stringSerde, accountSerde, "accounts-by-id")
  • 35. Joins • Types of joins: inner / outer / left join • Operands: • KTable to KTable → Non-windowed • KStream to KTable → Non-windowed • KStream to KStream → Windowed (to avoid an infinitely growing result) • Data must be co-partitioned • Repartition using an internal topic if need be
  • 36. Deploying and Running • Assemble a JAR (maven-shade plugin) • Run the JAR as a regular Java application (java -cp …) • Make sure all instances are in the same consumer group (same application ID)
  • 37. Running Topic “AUTH-JSON” with 4 partitions Application ID = “auth-converter” Log on the first instance: 11:00:22,331 ...AbstractCoordinator - Successfully joined group auth-converter with generation 1 11:00:22,332 ...ConsumerCoordinator - Setting newly assigned partitions [AUTH_JSON-2, AUTH_JSON-1, AUTH_JSON-3, AUTH_JSON-0] for group auth-converter
  • 38. Running - Scaling up Start a new instance: Log on the first instance: 11:01:31,402 ...AbstractCoordinator - Successfully joined group auth-converter with generation 2 11:01:31,404 ...ConsumerCoordinator - Setting newly assigned partitions [AUTH_JSON-2, AUTH_JSON-3] for group auth-converter 11:01:31,390 ...ConsumerCoordinator - Revoking previously assigned partitions [AUTH_JSON-2, AUTH_JSON-1, AUTH_JSON-3, AUTH_JSON-0] for group auth-converter 11:01:31,401 ...ConsumerCoordinator - Setting newly assigned partitions [AUTH_JSON-1, AUTH_JSON-0] for group auth-converter
  • 39. Running - Scaling down Kill one of the instances Log on the remaining instance: 11:02:13,410 ...ConsumerCoordinator - Revoking previously assigned partitions [AUTH_JSON-1, AUTH_JSON-0] for group auth-converter 11:02:13,415 ...ConsumerCoordinator - Setting newly assigned partitions [AUTH_JSON-2, AUTH_JSON-1, AUTH_JSON-3, AUTH_JSON-0] for group auth-converter
  • 40. Delivery semantics • At least once • No messages will be lost • Messages can be processed a second time when failure happens → Make your system idempotent • Exactly once planned for Kafka 0.11 • KIP-98 - Exactly Once Delivery andTransactional Messaging • KIP-129: Streams Exactly-Once Semantics
  • 41. Kafka Streams on this project
  • 42. Migration • Conversion of Spark / Scala code • Upgraded from Scala 2.10 to 2.11 and enabled the -Xexperimental flag of the Scala compiler so that Scala lambdas are converted into Java lambdas (SAM support) • Removed lots of specific code to read from / write to Kafka (supported out-of-the-box with Kafka Streams) • API similar to the RDD API →Very straightforward conversion (no need to call foreachRDD, so even better!) • Conversion of Spark / Python code: not attempted
  • 43. Metrics • Kafka Streams doesn’t have a UI to display metrics (e.g. number of records processed) • Used Dropwizard Metrics (metrics.dropwizard.io) • Java API to calculate metrics and send them to various sinks • Used InfluxDB to store the metrics • Graphite compatible • Used Grafana to display the metrics as graphs
  • 44.
  • 45. Metrics aggregation • Each instance reports its own metrics → Need to aggregate metrics • Specific reporter to send Dropwizard Metrics to a Kafka topic • Kafka topic to collect metrics • 1 partition • Key = instance ID (e.g. app-1, app-2…) • Value = monotonic metric • Kafka Streams app to aggregate metrics • Input is a KTable (new values replace previous values) • Send aggregated metrics to InfluxDB
  • 46. Kafka Streams app to aggregate metrics KTable<String, CounterMetric> metricsStream = builder.table(appIdSerde, metricSerde, "metrics", "raw-metrics");
 KStream<String, CounterMetric> metricValueStream = metricsStream
 .groupBy((key, value) -> new KeyValue<>(value.getName(), value), metricNameSerde, metricSerde)
 .reduce(CounterMetric::add, CounterMetric::subtract, "aggregates")
 .toStream() .to(metricNameSerde, metricSerde, "metrics-agg");
 
 // --- Second topology
 
 GraphiteReporter graphite = GraphiteReporter.builder()
 .hostname("localhost")
 .port(2003)
 .build();
 
 KStream<String, CounterMetric> aggMetricsStream = builder.stream(metricNameSerde, metricSerde, "metrics-agg");
 aggMetricsStream.foreach((key, metric) -> graphite.send(metric));
  • 48. Send data into Kafka (1M records) Start consumer 1 Start consumer 2 Aggregated metric (from consumers 1 and 2) Stop consumer 2 Delta = records processed twice
  • 49. Results Pros • Simpler code (no manual handling of offsets) • Simpler packaging (no dependencies to exclude, less dependency version conflicts) • Much lower latency: from seconds to milliseconds • Reduced memory footprint • Easier scaling • Improved stability when restarting the application Cons • No UI • No centralized logs → Use ELK or equivalent… • No centralized metrics → Aggregate metrics • Have to use an intermediate topic if grouping the data by a value that is not the key
  • 51. Summary • Very easy to build pipelines on top of Kafka • Great fit for micro-services • Compared to Spark Streaming: • Better for realtime apps than Spark Streaming • Lower latency, lower memory footprint, easier scaling • Lower level: good for prod, lacks a UI for dev • Compared to a standard Kafka consumer: • Higher level: faster to build a sophisticated app • Less control for very fine-grained consumption