SlideShare a Scribd company logo
1 of 46
Hands on Apache Flink
How to run, debug and speed up
Flink applications
Robert Metzger
rmetzger@apache.org
@rmetzger_
This talk
• Frequently asked questions + their
answers
• An overview over the tooling in Flink
• An outlook into the future
flink.apache.org 1
“One week of trials and errors
can save up to half an hour of
reading the documentation.”
– Paris Hilton
flink.apache.org 2
WRITE AND TEST YOUR JOB
The first step
flink.apache.org 3
Get started with an empty project
• Generate a skeleton project with Maven
flink.apache.org 4
mvn archetype:generate /
-DarchetypeGroupId=org.apache.flink /
-DarchetypeArtifactId=flink-quickstart-java /
-DarchetypeVersion=0.9-SNAPSHOT
you can also put
“quickstart-scala” here
or “0.8.1”
• No need for manually downloading any
.tgz or .jar files for now
Local Development
• Start Flink in your IDE for local
development & debugging.
flink.apache.org 5
final ExecutionEnvironment env =
ExecutionEnvironment.createLocalEnvironment();
• Use our testing framework
@RunWith(Parameterized.class)
class YourTest extends MultipleProgramsTestBase {
@Test
public void testRunWithConfiguration(){
expectedResult = "1 11n“;
}}
Debugging with the IDE
flink.apache.org 6
RUN YOUR JOB ON A (FAKE)
CLUSTER
Get your hands dirty
flink.apache.org 7
Got no cluster? – Renting options
• Google Compute Engine [1]
• Amazon EMR or any other cloud provider
with preinstalled Hadoop YARN [2]
• Install Flink yourself on the machines
flink.apache.org 8
./bdutil -e extensions/flink/flink_env.sh deploy
[1] http://ci.apache.org/projects/flink/flink-docs-master/setup/gce_setup.html
[2] http://ci.apache.org/projects/flink/flink-docs-master/setup/yarn_setup.html
wget http://stratosphere-bin.amazonaws.com/flink-0.9-SNAPSHOT-bin-hadoop2.tgz
tar xvzf flink-0.9-SNAPSHOT-bin-hadoop2.tgz
cd flink-0.9-SNAPSHOT/
./bin/yarn-session.sh -n 4 -jm 1024 -tm 4096
Got no money?
• Listen closely to this talk and become a
freelance “Big Data Consultant”
• Start a cluster locally in the meantime
flink.apache.org 9
$ tar xzf flink-*.tgz
$ cd flink
$ bin/start-cluster.sh
Starting Job Manager
Starting task manager on host
$ jps
5158 JobManager
5262 TaskManager
assert hasCluster;
• Submitting a job
– /bin/flink (Command Line)
– RemoteExecutionEnvironment
(From a local or remote java app)
– Web Frontend (GUI)
– Per job on YARN (Command Line, directly to
YARN)
– Scala Shell
flink.apache.org 10
Web Frontends – Web Job Client
flink.apache.org 11
Select jobs and
preview plan
Understand Optimizer choices
Web Frontends – Job Manager
flink.apache.org 12
Overall system status
Job execution details
Task Manager resource
utilization
Debugging on a cluster
• Good old system out debugging
– Get a logger
– Start logging
– You can also use System.out.println().
flink.apache.org 13
private static final Logger LOG =
LoggerFactory.getLogger(YourJob.class);
LOG.info("elementCount = {}", elementCount);
Getting logs on a cluster
• Non-YARN (=bare metal installation)
– The logs are located in each TaskManager’s
log/ directory.
– ssh there and read the logs.
• YARN
– Make sure YARN log aggregation is enabled
– Retrieve logs from YARN (once app is
finished)
flink.apache.org 14
$ yarn logs -applicationId <application ID>
Flink Logs
11:42:39,233 INFO org.apache.flink.runtime.jobmanager.JobManager - --------------------------------------------------------------------------------
11:42:39,233 INFO org.apache.flink.runtime.jobmanager.JobManager - Starting JobManager (Version: 0.9-SNAPSHOT, Rev:2e515fc, Date:27.05.2015 @ 11:24:23 CEST)
11:42:39,233 INFO org.apache.flink.runtime.jobmanager.JobManager - Current user: robert
11:42:39,233 INFO org.apache.flink.runtime.jobmanager.JobManager - JVM: OpenJDK 64-Bit Server VM - Oracle Corporation - 1.7/24.75-b04
11:42:39,233 INFO org.apache.flink.runtime.jobmanager.JobManager - Maximum heap size: 736 MiBytes
11:42:39,233 INFO org.apache.flink.runtime.jobmanager.JobManager - JAVA_HOME: (not set)
11:42:39,233 INFO org.apache.flink.runtime.jobmanager.JobManager - JVM Options:
11:42:39,233 INFO org.apache.flink.runtime.jobmanager.JobManager - -XX:MaxPermSize=256m
11:42:39,233 INFO org.apache.flink.runtime.jobmanager.JobManager - -Xms768m
11:42:39,233 INFO org.apache.flink.runtime.jobmanager.JobManager - -Xmx768m
11:42:39,233 INFO org.apache.flink.runtime.jobmanager.JobManager - -Dlog.file=/home/robert/incubator-flink/build-target/bin/../log/flink-robert-jobmanager-robert-da.log
11:42:39,233 INFO org.apache.flink.runtime.jobmanager.JobManager - -Dlog4j.configuration=file:/home/robert/incubator-flink/build-target/bin/../conf/log4j.properties
11:42:39,233 INFO org.apache.flink.runtime.jobmanager.JobManager - -Dlogback.configurationFile=file:/home/robert/incubator-flink/build-target/bin/../conf/logback.xml
11:42:39,233 INFO org.apache.flink.runtime.jobmanager.JobManager - Program Arguments:
11:42:39,233 INFO org.apache.flink.runtime.jobmanager.JobManager - --configDir
11:42:39,233 INFO org.apache.flink.runtime.jobmanager.JobManager - /home/robert/incubator-flink/build-target/bin/../conf
11:42:39,234 INFO org.apache.flink.runtime.jobmanager.JobManager - --executionMode
11:42:39,234 INFO org.apache.flink.runtime.jobmanager.JobManager - local
11:42:39,234 INFO org.apache.flink.runtime.jobmanager.JobManager - --streamingMode
11:42:39,234 INFO org.apache.flink.runtime.jobmanager.JobManager - batch
11:42:39,234 INFO org.apache.flink.runtime.jobmanager.JobManager - --------------------------------------------------------------------------------
11:42:39,469 INFO org.apache.flink.runtime.jobmanager.JobManager - Loading configuration from /home/robert/incubator-flink/build-target/bin/../conf
11:42:39,525 INFO org.apache.flink.runtime.jobmanager.JobManager - Security is not enabled. Starting non-authenticated JobManager.
11:42:39,525 INFO org.apache.flink.runtime.jobmanager.JobManager - Starting JobManager
11:42:39,527 INFO org.apache.flink.runtime.jobmanager.JobManager - Starting JobManager actor system at localhost:6123.
11:42:40,189 INFO akka.event.slf4j.Slf4jLogger - Slf4jLogger started
11:42:40,316 INFO Remoting - Starting remoting
11:42:40,569 INFO Remoting - Remoting started; listening on addresses :[akka.tcp://flink@127.0.0.1:6123]
11:42:40,573 INFO org.apache.flink.runtime.jobmanager.JobManager - Starting JobManager actor
11:42:40,580 INFO org.apache.flink.runtime.blob.BlobServer - Created BLOB server storage directory /tmp/blobStore-50f75dc9-3001-4c1b-bc2a-6658ac21322b
11:42:40,581 INFO org.apache.flink.runtime.blob.BlobServer - Started BLOB server at 0.0.0.0:51194 - max concurrent requests: 50 - max backlog: 1000
11:42:40,613 INFO org.apache.flink.runtime.jobmanager.JobManager - Starting embedded TaskManager for JobManager's LOCAL execution mode
11:42:40,615 INFO org.apache.flink.runtime.jobmanager.JobManager - Starting JobManager at akka://flink/user/jobmanager#205521910.
11:42:40,663 INFO org.apache.flink.runtime.taskmanager.TaskManager - Messages between TaskManager and JobManager have a max timeout of 100000 milliseconds
11:42:40,666 INFO org.apache.flink.runtime.taskmanager.TaskManager - Temporary file directory '/tmp': total 7 GB, usable 7 GB (100.00% usable)
11:42:41,092 INFO org.apache.flink.runtime.io.network.buffer.NetworkBufferPool - Allocated 64 MB for network buffer pool (number of memory segments: 2048, bytes per segment: 32768).
11:42:41,511 INFO org.apache.flink.runtime.taskmanager.TaskManager - Using 0.7 of the currently free heap space for Flink managed memory (461 MB).
11:42:42,520 INFO org.apache.flink.runtime.io.disk.iomanager.IOManager - I/O manager uses directory /tmp/flink-io-4c6f4364-1975-48b7-99d9-a74e4edb7103 for spill files.
11:42:42,523 INFO org.apache.flink.runtime.jobmanager.JobManager - Starting JobManger web frontend
flink.apache.org 15
Build Information
JVM details
Init messages
Get logs of a running YARN
application
flink.apache.org 16
Debugging on a cluster -
Accumulators
• Useful to verify your assumptions about
the data
flink.apache.org 17
class Tokenizer extends RichFlatMapFunction<String, String>>
{
@Override
public void flatMap(String value, Collector<String> out) {
getRuntimeContext()
.getLongCounter("elementCount").add(1L);
// do more stuff.
} }
Use “Rich*Functions” to get RuntimeContext
Debugging on a cluster -
Accumulators
• Where can I get the accumulator results?
– returned by env.execute()
– displayed when executed with /bin/flink
– in the JobManager web frontend
flink.apache.org 18
JobExecutionResult result = env.execute("WordCount");
long ec = result.getAccumulatorResult("elementCount");
Excursion: RichFunctions
• The default functions are SAMs (Single
abstract method). Interfaces with one
method (for Java8 Lambdas)
• There is a “Rich” variant for each function.
– RichFlatMapFunction, …
– Methods
• open(Configuration c) & close()
• getRuntimeContext()
flink.apache.org 19
Excursion: RichFunctions &
RuntimeContext
• The RuntimeContext provides some useful
methods
• getIndexOfThisSubtask () /
getNumberOfParallelSubtasks() – who am
I, and if yes how many?
• getExecutionConfig()
• Accumulators
• DistributedCache
flink.apache.org 20
Attaching a remote debugger to
Flink in a Cluster
flink.apache.org 21
Attaching a debugger to Flink in a
cluster
• Add JVM start option in flink-conf.yaml
env.java.opts: “-agentlib:jdwp=….”
• Open an SSH tunnel to the machine:
ssh -f -N -L 5005:127.0.0.1:5005 user@host
• Use your IDE to start a remote debugging
session
flink.apache.org 22
JOB TUNING
Make it run faster
flink.apache.org 23
Tuning options
• CPU
– Processing slots, threads, …
• Memory
– How to adjust memory usage on the
TaskManager
• I/O
– Specifying temporary directories for spilling
flink.apache.org 24
Tell Flink how many CPUs you
have
• taskmanager.numberOfTaskSlots
– number of parallel job instances
– number of pipelines per TaskManager
• recommended: number of CPU cores
flink.apache.org 25
Map Reduce
Map Reduce
Map Reduce
Map Reduce
Map Reduce
Map Reduce
Map Reduce
Task Manager 1
Slot 1
Slot 2
Slot 3
Task Manager 2
Slot 1
Slot 2
Slot 3
Task Manager 3
Slot 1
Slot 2
Slot 3
Task
Managers: 3
Total number of
processing
slots: 9
flink-config.yaml:
taskmanager.numberOfTaskSlots: 3
(Recommended value: Number of CPU cores)
or
/bin/yarn-session.sh –slots 3 –n 3
Processing slots
Slots – Wordcount with
parallelism=1
flink.apache.org 27
Task Manager 1
Slot 1
Slot 2
Slot 3
Task Manager 2
Slot 1
Slot 2
Slot 3
Task Manager 3
Slot 1
Slot 2
Slot 3
Source ->
flatMap
Reduce Sink
When no argument given,
parallelism.default from
flink-config.yaml is used.
Default value = 1
Slots – Wordcount with higher
parallelism (= 2 here)
flink.apache.org
28Task Manager 1
Slot 1
Slot 2
Slot 3
Task Manager 2
Slot 1
Slot 2
Slot 3
Task Manager 3
Slot 1
Slot 2
Slot 3
Source ->
flatMap
Reduce Sink
Source ->
flatMap
Reduce Sink
Places to set parallelism for a job
flink-config.yaml
parallelism.default: 2
or Flink Client:
./bin/flink -p 2
or ExecutionEnvironment:
env.setParallelism(2)
Slots – Wordcount using all
resources (parallelism = 9)
flink.apache.org 29
Task Manager 1
Slot 1
Slot 2
Slot 3
Task Manager 2
Slot 1
Slot 2
Slot 3
Task Manager 3
Slot 1
Slot 2
Slot 3
Source ->
flatMap
Reduce Sink
Source ->
flatMap
Reduce Sink
Source ->
flatMap
Reduce Sink
Source -
> flatMap
Reduce Sink
Source ->
flatMap
Reduce Sink
Source ->
flatMap
Reduce Sink
Source ->
flatMap
Reduce Sink
Source ->
flatMap
Reduce Sink
Source ->
flatMap
Reduce Sink
Slots – Setting parallelism on a per
operator basis
flink.apache.org 30
Task Manager 1
Slot 1
Slot 2
Slot 3
Task Manager 2
Slot 1
Slot 2
Slot 3
Task Manager 3
Slot 1
Slot 2
Slot 3
Source ->
flatMap
Reduce
Source ->
flatMap
Reduce
Source ->
flatMap
Reduce
Source -
> flatMap
Reduce
Source ->
flatMap
Reduce
Source ->
flatMap
Reduce
Source ->
flatMap
Reduce
Source ->
flatMap
Reduce
Source ->
flatMap
Reduce
The parallelism of each operator can be set individually in the APIs
counts.writeAsCsv(outputPath, "n", " ").setParallelism(1);
Sink
Slots – Setting parallelism on a per
operator basis
flink.apache.org 31
Task Manager 1
Slot 1
Slot 2
Slot 3
Task Manager 2
Slot 1
Slot 2
Slot 3
Task Manager 3
Slot 1
Slot 2
Slot 3
Source ->
flatMap
Reduce
Source ->
flatMap
Reduce
Source ->
flatMap
Reduce
Source -
> flatMap
Reduce
Source ->
flatMap
Reduce
Source ->
flatMap
Reduce
Source ->
flatMap
Reduce
Source ->
flatMap
Reduce
Source ->
flatMap
Reduce
Sink
The data is streamed to this Sink
from all the other slots on the
other TaskManagers
Tuning options
• CPU
– Processing slots, threads, …
• Memory
– How to adjust memory usage on the
TaskManager
• I/O
– Specifying temporary directories for spilling
flink.apache.org 32
flink.apache.org 33
Memory in Flink - Theory
flink.apache.org 34
taskmanager.network.numberOfBuffers
relative: taskmanager.memory.fraction
absolute: taskmanager.memory.size
Memory in Flink - Configuration
taskmanager.heap.mb
or „-tm“ argument for bin/yarn-session.sh
Memory in Flink - OOM
flink.apache.org 35
2015-02-20 11:22:54 INFO JobClient:345 - java.lang.OutOfMemoryError: Java heap space
at org.apache.flink.runtime.io.network.serialization.DataOutputSerializer.resize(DataOutputSerializer.java:249)
at org.apache.flink.runtime.io.network.serialization.DataOutputSerializer.write(DataOutputSerializer.java:93)
at org.apache.flink.api.java.typeutils.runtime.DataOutputViewStream.write(DataOutputViewStream.java:39)
at com.esotericsoftware.kryo.io.Output.flush(Output.java:163)
at com.esotericsoftware.kryo.io.Output.require(Output.java:142)
at com.esotericsoftware.kryo.io.Output.writeBoolean(Output.java:613)
at com.twitter.chill.java.BitSetSerializer.write(BitSetSerializer.java:42)
at com.twitter.chill.java.BitSetSerializer.write(BitSetSerializer.java:29)
at com.esotericsoftware.kryo.Kryo.writeClassAndObject(Kryo.java:599)
at org.apache.flink.api.java.typeutils.runtime.KryoSerializer.serialize(KryoSerializer.java:155)
at org.apache.flink.api.scala.typeutils.CaseClassSerializer.serialize(CaseClassSerializer.scala:91)
at org.apache.flink.api.scala.typeutils.CaseClassSerializer.serialize(CaseClassSerializer.scala:30)
at org.apache.flink.runtime.plugable.SerializationDelegate.write(SerializationDelegate.java:51)
at
org.apache.flink.runtime.io.network.serialization.SpanningRecordSerializer.addRecord(SpanningRecordSerializer.java:76
at org.apache.flink.runtime.io.network.api.RecordWriter.emit(RecordWriter.java:82)
at org.apache.flink.runtime.operators.shipping.OutputCollector.collect(OutputCollector.java:88)
at org.apache.flink.api.scala.GroupedDataSet$$anon$2.reduce(GroupedDataSet.scala:262)
at org.apache.flink.runtime.operators.GroupReduceDriver.run(GroupReduceDriver.java:124)
at org.apache.flink.runtime.operators.RegularPactTask.run(RegularPactTask.java:493)
at org.apache.flink.runtime.operators.RegularPactTask.invoke(RegularPactTask.java:360)
at org.apache.flink.runtime.execution.RuntimeEnvironment.run(RuntimeEnvironment.java:257)
at java.lang.Thread.run(Thread.java:745)
Memory is missing
here
Reduce managed
memory
reduce
taskmanager.
memory.fraction
Memory in Flink – Network buffers
flink.apache.org 36
Memory is missing
here
Managed memory
will shrink
automatically
Error: java.lang.Exception: Failed to deploy the task CHAIN
Reduce(org.okkam.flink.maintenance.deduplication.blocking.RemoveDuplicateReduceGr
oupFunction) ->
Combine(org.apache.flink.api.java.operators.DistinctOperator$DistinctFunction) (15/28) -
execution #0 to slot SubSlot 5 (cab978f80c0cb7071136cd755e971be9 (5) -
ALLOCATED/ALIVE):
org.apache.flink.runtime.io.network.InsufficientResourcesException: okkam-nano-
2.okkam.it has not enough buffers to safely execute CHAIN
Reduce(org.okkam.flink.maintenance.deduplication.blocking.RemoveDuplicateReduceGr
oupFunction) ->
Combine(org.apache.flink.api.java.operators.DistinctOperator$DistinctFunction)
(36 buffers missing)
increase „taskmanager.network.numberOfBuffers“
What are these buffers needed for?
flink.apache.org 37
TaskManager 1
Slot 2
Map Reduce
Slot 1
TaskManager 2
Slot 2
Slot 1
A small Flink cluster with 4 processing slots (on 2 Task Managers)
A simple MapReduce Job in Flink:
What are these buffers needed for?
flink.apache.org 38
Map Reduce job with a parallelism of 2 and 2 processing slots per Machine
TaskManager 1 TaskManager 2
Slot1Slot2
Map
Map
Reduce
Reduce
Map
Map
Reduce
Reduce
Map
Map
Reduce
Reduce
Map
Map
Reduce
Reduce
Slot1Slot2
Network buffer
8 buffers for outgoing
data 8 buffers for incoming
data
What are these buffers needed for?
flink.apache.org 39
Map Reduce job with a parallelism of 2 and 2 processing slots per Machine
TaskManager 1 TaskManager 2
Slot1Slot2
Map
Map
Reduce
Reduce
Map
Map
Reduce
Reduce
Map
Map
Reduce
Reduce
Map
Map
Reduce
Reduce
Tuning options
• CPU
– Processing slots, threads, …
• Memory
– How to adjust memory usage on the
TaskManager
• I/O
– Specifying temporary directories for spilling
flink.apache.org 40
Tuning options
• Memory
– How to adjust memory usage on the
TaskManager
• CPU
– Processing slots, threads, …
• I/O
– Specifying temporary directories for spilling
flink.apache.org 41
Disk I/O
• Sometimes your data doesn’t fit into main
memory, so we have to spill to disk
– taskmanager.tmp.dirs: /mnt/disk1,/mnt/disk2
• Use real local disks only (no tmpfs or
NAS)
flink.apache.org 42
Reader
Thread
Disk 1
Writer
Thread
Reader
Thread
Writer
Thread
Disk 2
Task Manager
Outlook
• Per job monitoring & metrics
• Less configuration values with dynamic
memory management
• Download operator results to debug them
locally
flink.apache.org 43
Join our community
• RTFM (= read the documentation)
• Mailing lists
– Subscribe: user-subscribe@flink.apache.org
– Ask: user@flink.apache.org
• Stack Overflow
– tag with “flink” so that we get an email
notification ;)
• IRC: freenode#flink
• Read the code, its open source 
flink.apache.org 44
Flink Forward registration & call for
abstracts is open now
flink.apache.org 45
• 12/13 October 2015
• Kulturbrauerei Berlin
• With Flink Workshops / Trainings!

More Related Content

What's hot

Apache kafka performance(throughput) - without data loss and guaranteeing dat...
Apache kafka performance(throughput) - without data loss and guaranteeing dat...Apache kafka performance(throughput) - without data loss and guaranteeing dat...
Apache kafka performance(throughput) - without data loss and guaranteeing dat...SANG WON PARK
 
Spark shuffle introduction
Spark shuffle introductionSpark shuffle introduction
Spark shuffle introductioncolorant
 
Tame the small files problem and optimize data layout for streaming ingestion...
Tame the small files problem and optimize data layout for streaming ingestion...Tame the small files problem and optimize data layout for streaming ingestion...
Tame the small files problem and optimize data layout for streaming ingestion...Flink Forward
 
Kafka to the Maxka - (Kafka Performance Tuning)
Kafka to the Maxka - (Kafka Performance Tuning)Kafka to the Maxka - (Kafka Performance Tuning)
Kafka to the Maxka - (Kafka Performance Tuning)DataWorks Summit
 
Apache Flink internals
Apache Flink internalsApache Flink internals
Apache Flink internalsKostas Tzoumas
 
The Top Five Mistakes Made When Writing Streaming Applications with Mark Grov...
The Top Five Mistakes Made When Writing Streaming Applications with Mark Grov...The Top Five Mistakes Made When Writing Streaming Applications with Mark Grov...
The Top Five Mistakes Made When Writing Streaming Applications with Mark Grov...Databricks
 
Demystifying flink memory allocation and tuning - Roshan Naik, Uber
Demystifying flink memory allocation and tuning - Roshan Naik, UberDemystifying flink memory allocation and tuning - Roshan Naik, Uber
Demystifying flink memory allocation and tuning - Roshan Naik, UberFlink Forward
 
Productizing Structured Streaming Jobs
Productizing Structured Streaming JobsProductizing Structured Streaming Jobs
Productizing Structured Streaming JobsDatabricks
 
Introduction to Apache Flink - Fast and reliable big data processing
Introduction to Apache Flink - Fast and reliable big data processingIntroduction to Apache Flink - Fast and reliable big data processing
Introduction to Apache Flink - Fast and reliable big data processingTill Rohrmann
 
Step-by-Step Introduction to Apache Flink
Step-by-Step Introduction to Apache Flink Step-by-Step Introduction to Apache Flink
Step-by-Step Introduction to Apache Flink Slim Baltagi
 
Apache Flink: API, runtime, and project roadmap
Apache Flink: API, runtime, and project roadmapApache Flink: API, runtime, and project roadmap
Apache Flink: API, runtime, and project roadmapKostas Tzoumas
 
Simplifying Change Data Capture using Databricks Delta
Simplifying Change Data Capture using Databricks DeltaSimplifying Change Data Capture using Databricks Delta
Simplifying Change Data Capture using Databricks DeltaDatabricks
 
Autoscaling Flink with Reactive Mode
Autoscaling Flink with Reactive ModeAutoscaling Flink with Reactive Mode
Autoscaling Flink with Reactive ModeFlink Forward
 
Introduction to Apache Flink
Introduction to Apache FlinkIntroduction to Apache Flink
Introduction to Apache Flinkdatamantra
 
Apache Flink and what it is used for
Apache Flink and what it is used forApache Flink and what it is used for
Apache Flink and what it is used forAljoscha Krettek
 
Introduction to Apache Flink
Introduction to Apache FlinkIntroduction to Apache Flink
Introduction to Apache Flinkmxmxm
 
Easy, scalable, fault tolerant stream processing with structured streaming - ...
Easy, scalable, fault tolerant stream processing with structured streaming - ...Easy, scalable, fault tolerant stream processing with structured streaming - ...
Easy, scalable, fault tolerant stream processing with structured streaming - ...Databricks
 
Kafka monitoring using Prometheus and Grafana
Kafka monitoring using Prometheus and GrafanaKafka monitoring using Prometheus and Grafana
Kafka monitoring using Prometheus and Grafanawonyong hwang
 

What's hot (20)

Apache kafka performance(throughput) - without data loss and guaranteeing dat...
Apache kafka performance(throughput) - without data loss and guaranteeing dat...Apache kafka performance(throughput) - without data loss and guaranteeing dat...
Apache kafka performance(throughput) - without data loss and guaranteeing dat...
 
Spark shuffle introduction
Spark shuffle introductionSpark shuffle introduction
Spark shuffle introduction
 
Tame the small files problem and optimize data layout for streaming ingestion...
Tame the small files problem and optimize data layout for streaming ingestion...Tame the small files problem and optimize data layout for streaming ingestion...
Tame the small files problem and optimize data layout for streaming ingestion...
 
Apache NiFi in the Hadoop Ecosystem
Apache NiFi in the Hadoop Ecosystem Apache NiFi in the Hadoop Ecosystem
Apache NiFi in the Hadoop Ecosystem
 
Kafka to the Maxka - (Kafka Performance Tuning)
Kafka to the Maxka - (Kafka Performance Tuning)Kafka to the Maxka - (Kafka Performance Tuning)
Kafka to the Maxka - (Kafka Performance Tuning)
 
Apache Flink internals
Apache Flink internalsApache Flink internals
Apache Flink internals
 
The Top Five Mistakes Made When Writing Streaming Applications with Mark Grov...
The Top Five Mistakes Made When Writing Streaming Applications with Mark Grov...The Top Five Mistakes Made When Writing Streaming Applications with Mark Grov...
The Top Five Mistakes Made When Writing Streaming Applications with Mark Grov...
 
Demystifying flink memory allocation and tuning - Roshan Naik, Uber
Demystifying flink memory allocation and tuning - Roshan Naik, UberDemystifying flink memory allocation and tuning - Roshan Naik, Uber
Demystifying flink memory allocation and tuning - Roshan Naik, Uber
 
Productizing Structured Streaming Jobs
Productizing Structured Streaming JobsProductizing Structured Streaming Jobs
Productizing Structured Streaming Jobs
 
Introduction to Apache Flink - Fast and reliable big data processing
Introduction to Apache Flink - Fast and reliable big data processingIntroduction to Apache Flink - Fast and reliable big data processing
Introduction to Apache Flink - Fast and reliable big data processing
 
Step-by-Step Introduction to Apache Flink
Step-by-Step Introduction to Apache Flink Step-by-Step Introduction to Apache Flink
Step-by-Step Introduction to Apache Flink
 
Apache Flink: API, runtime, and project roadmap
Apache Flink: API, runtime, and project roadmapApache Flink: API, runtime, and project roadmap
Apache Flink: API, runtime, and project roadmap
 
Simplifying Change Data Capture using Databricks Delta
Simplifying Change Data Capture using Databricks DeltaSimplifying Change Data Capture using Databricks Delta
Simplifying Change Data Capture using Databricks Delta
 
Autoscaling Flink with Reactive Mode
Autoscaling Flink with Reactive ModeAutoscaling Flink with Reactive Mode
Autoscaling Flink with Reactive Mode
 
Apache Spark Architecture
Apache Spark ArchitectureApache Spark Architecture
Apache Spark Architecture
 
Introduction to Apache Flink
Introduction to Apache FlinkIntroduction to Apache Flink
Introduction to Apache Flink
 
Apache Flink and what it is used for
Apache Flink and what it is used forApache Flink and what it is used for
Apache Flink and what it is used for
 
Introduction to Apache Flink
Introduction to Apache FlinkIntroduction to Apache Flink
Introduction to Apache Flink
 
Easy, scalable, fault tolerant stream processing with structured streaming - ...
Easy, scalable, fault tolerant stream processing with structured streaming - ...Easy, scalable, fault tolerant stream processing with structured streaming - ...
Easy, scalable, fault tolerant stream processing with structured streaming - ...
 
Kafka monitoring using Prometheus and Grafana
Kafka monitoring using Prometheus and GrafanaKafka monitoring using Prometheus and Grafana
Kafka monitoring using Prometheus and Grafana
 

Similar to Apache Flink Hands On

Riga Dev Day - Automated Android Continuous Integration
Riga Dev Day - Automated Android Continuous IntegrationRiga Dev Day - Automated Android Continuous Integration
Riga Dev Day - Automated Android Continuous IntegrationNicolas Fränkel
 
Spark on Yarn
Spark on YarnSpark on Yarn
Spark on YarnQubole
 
How eBay does Automatic Outage Planning
How eBay does Automatic Outage PlanningHow eBay does Automatic Outage Planning
How eBay does Automatic Outage PlanningCA | Automic Software
 
Introduction to Laravel Framework (5.2)
Introduction to Laravel Framework (5.2)Introduction to Laravel Framework (5.2)
Introduction to Laravel Framework (5.2)Viral Solani
 
Ansible benelux meetup - Amsterdam 27-5-2015
Ansible benelux meetup - Amsterdam 27-5-2015Ansible benelux meetup - Amsterdam 27-5-2015
Ansible benelux meetup - Amsterdam 27-5-2015Pavel Chunyayev
 
Spark 2.x Troubleshooting Guide
Spark 2.x Troubleshooting GuideSpark 2.x Troubleshooting Guide
Spark 2.x Troubleshooting GuideIBM
 
Why scala is not my ideal language and what I can do with this
Why scala is not my ideal language and what I can do with thisWhy scala is not my ideal language and what I can do with this
Why scala is not my ideal language and what I can do with thisRuslan Shevchenko
 
RichFaces - Testing on Mobile Devices
RichFaces - Testing on Mobile DevicesRichFaces - Testing on Mobile Devices
RichFaces - Testing on Mobile DevicesPavol Pitoňák
 
Release with confidence
Release with confidenceRelease with confidence
Release with confidenceJohn Congdon
 
Performance tuning with zend framework
Performance tuning with zend frameworkPerformance tuning with zend framework
Performance tuning with zend frameworkAlan Seiden
 
The power of linux advanced tracer [POUG18]
The power of linux advanced tracer [POUG18]The power of linux advanced tracer [POUG18]
The power of linux advanced tracer [POUG18]Mahmoud Hatem
 
Using apache spark for processing trillions of records each day at Datadog
Using apache spark for processing trillions of records each day at DatadogUsing apache spark for processing trillions of records each day at Datadog
Using apache spark for processing trillions of records each day at DatadogVadim Semenov
 
Spark summit2014 techtalk - testing spark
Spark summit2014 techtalk - testing sparkSpark summit2014 techtalk - testing spark
Spark summit2014 techtalk - testing sparkAnu Shetty
 
Performance Profiling in Rust
Performance Profiling in RustPerformance Profiling in Rust
Performance Profiling in RustInfluxData
 
Audit your reactive applications
Audit your reactive applicationsAudit your reactive applications
Audit your reactive applicationsOCTO Technology
 
Byteman and The Jokre, Sanne Grinovero (JBoss by RedHat)
Byteman and The Jokre, Sanne Grinovero (JBoss by RedHat)Byteman and The Jokre, Sanne Grinovero (JBoss by RedHat)
Byteman and The Jokre, Sanne Grinovero (JBoss by RedHat)OpenBlend society
 
Container orchestration from theory to practice
Container orchestration from theory to practiceContainer orchestration from theory to practice
Container orchestration from theory to practiceDocker, Inc.
 
Automated Java Deployments With Rpm
Automated Java Deployments With RpmAutomated Java Deployments With Rpm
Automated Java Deployments With RpmMartin Jackson
 

Similar to Apache Flink Hands On (20)

Riga Dev Day - Automated Android Continuous Integration
Riga Dev Day - Automated Android Continuous IntegrationRiga Dev Day - Automated Android Continuous Integration
Riga Dev Day - Automated Android Continuous Integration
 
Spark on Yarn
Spark on YarnSpark on Yarn
Spark on Yarn
 
How eBay does Automatic Outage Planning
How eBay does Automatic Outage PlanningHow eBay does Automatic Outage Planning
How eBay does Automatic Outage Planning
 
Introduction to Laravel Framework (5.2)
Introduction to Laravel Framework (5.2)Introduction to Laravel Framework (5.2)
Introduction to Laravel Framework (5.2)
 
Ansible benelux meetup - Amsterdam 27-5-2015
Ansible benelux meetup - Amsterdam 27-5-2015Ansible benelux meetup - Amsterdam 27-5-2015
Ansible benelux meetup - Amsterdam 27-5-2015
 
Spark 2.x Troubleshooting Guide
Spark 2.x Troubleshooting GuideSpark 2.x Troubleshooting Guide
Spark 2.x Troubleshooting Guide
 
infra-as-code
infra-as-codeinfra-as-code
infra-as-code
 
Why scala is not my ideal language and what I can do with this
Why scala is not my ideal language and what I can do with thisWhy scala is not my ideal language and what I can do with this
Why scala is not my ideal language and what I can do with this
 
RichFaces - Testing on Mobile Devices
RichFaces - Testing on Mobile DevicesRichFaces - Testing on Mobile Devices
RichFaces - Testing on Mobile Devices
 
Release with confidence
Release with confidenceRelease with confidence
Release with confidence
 
Performance tuning with zend framework
Performance tuning with zend frameworkPerformance tuning with zend framework
Performance tuning with zend framework
 
The power of linux advanced tracer [POUG18]
The power of linux advanced tracer [POUG18]The power of linux advanced tracer [POUG18]
The power of linux advanced tracer [POUG18]
 
Using apache spark for processing trillions of records each day at Datadog
Using apache spark for processing trillions of records each day at DatadogUsing apache spark for processing trillions of records each day at Datadog
Using apache spark for processing trillions of records each day at Datadog
 
ESIGate dev meeting #4 21-11-2013
ESIGate dev meeting #4 21-11-2013ESIGate dev meeting #4 21-11-2013
ESIGate dev meeting #4 21-11-2013
 
Spark summit2014 techtalk - testing spark
Spark summit2014 techtalk - testing sparkSpark summit2014 techtalk - testing spark
Spark summit2014 techtalk - testing spark
 
Performance Profiling in Rust
Performance Profiling in RustPerformance Profiling in Rust
Performance Profiling in Rust
 
Audit your reactive applications
Audit your reactive applicationsAudit your reactive applications
Audit your reactive applications
 
Byteman and The Jokre, Sanne Grinovero (JBoss by RedHat)
Byteman and The Jokre, Sanne Grinovero (JBoss by RedHat)Byteman and The Jokre, Sanne Grinovero (JBoss by RedHat)
Byteman and The Jokre, Sanne Grinovero (JBoss by RedHat)
 
Container orchestration from theory to practice
Container orchestration from theory to practiceContainer orchestration from theory to practice
Container orchestration from theory to practice
 
Automated Java Deployments With Rpm
Automated Java Deployments With RpmAutomated Java Deployments With Rpm
Automated Java Deployments With Rpm
 

More from Robert Metzger

How to Contribute to Apache Flink (and Flink at the Apache Software Foundation)
How to Contribute to Apache Flink (and Flink at the Apache Software Foundation)How to Contribute to Apache Flink (and Flink at the Apache Software Foundation)
How to Contribute to Apache Flink (and Flink at the Apache Software Foundation)Robert Metzger
 
Apache Flink @ Tel Aviv / Herzliya Meetup
Apache Flink @ Tel Aviv / Herzliya MeetupApache Flink @ Tel Aviv / Herzliya Meetup
Apache Flink @ Tel Aviv / Herzliya MeetupRobert Metzger
 
Apache Flink Community Updates November 2016 @ Berlin Meetup
Apache Flink Community Updates November 2016 @ Berlin MeetupApache Flink Community Updates November 2016 @ Berlin Meetup
Apache Flink Community Updates November 2016 @ Berlin MeetupRobert Metzger
 
A Data Streaming Architecture with Apache Flink (berlin Buzzwords 2016)
A Data Streaming Architecture with Apache Flink (berlin Buzzwords 2016)A Data Streaming Architecture with Apache Flink (berlin Buzzwords 2016)
A Data Streaming Architecture with Apache Flink (berlin Buzzwords 2016)Robert Metzger
 
Community Update May 2016 (January - May) | Berlin Apache Flink Meetup
Community Update May 2016 (January - May) | Berlin Apache Flink MeetupCommunity Update May 2016 (January - May) | Berlin Apache Flink Meetup
Community Update May 2016 (January - May) | Berlin Apache Flink MeetupRobert Metzger
 
GOTO Night Amsterdam - Stream processing with Apache Flink
GOTO Night Amsterdam - Stream processing with Apache FlinkGOTO Night Amsterdam - Stream processing with Apache Flink
GOTO Night Amsterdam - Stream processing with Apache FlinkRobert Metzger
 
QCon London - Stream Processing with Apache Flink
QCon London - Stream Processing with Apache FlinkQCon London - Stream Processing with Apache Flink
QCon London - Stream Processing with Apache FlinkRobert Metzger
 
January 2016 Flink Community Update & Roadmap 2016
January 2016 Flink Community Update & Roadmap 2016January 2016 Flink Community Update & Roadmap 2016
January 2016 Flink Community Update & Roadmap 2016Robert Metzger
 
Flink Community Update December 2015: Year in Review
Flink Community Update December 2015: Year in ReviewFlink Community Update December 2015: Year in Review
Flink Community Update December 2015: Year in ReviewRobert Metzger
 
Apache Flink Meetup Munich (November 2015): Flink Overview, Architecture, Int...
Apache Flink Meetup Munich (November 2015): Flink Overview, Architecture, Int...Apache Flink Meetup Munich (November 2015): Flink Overview, Architecture, Int...
Apache Flink Meetup Munich (November 2015): Flink Overview, Architecture, Int...Robert Metzger
 
Chicago Flink Meetup: Flink's streaming architecture
Chicago Flink Meetup: Flink's streaming architectureChicago Flink Meetup: Flink's streaming architecture
Chicago Flink Meetup: Flink's streaming architectureRobert Metzger
 
Flink September 2015 Community Update
Flink September 2015 Community UpdateFlink September 2015 Community Update
Flink September 2015 Community UpdateRobert Metzger
 
Architecture of Flink's Streaming Runtime @ ApacheCon EU 2015
Architecture of Flink's Streaming Runtime @ ApacheCon EU 2015Architecture of Flink's Streaming Runtime @ ApacheCon EU 2015
Architecture of Flink's Streaming Runtime @ ApacheCon EU 2015Robert Metzger
 
Click-Through Example for Flink’s KafkaConsumer Checkpointing
Click-Through Example for Flink’s KafkaConsumer CheckpointingClick-Through Example for Flink’s KafkaConsumer Checkpointing
Click-Through Example for Flink’s KafkaConsumer CheckpointingRobert Metzger
 
August Flink Community Update
August Flink Community UpdateAugust Flink Community Update
August Flink Community UpdateRobert Metzger
 
Flink Cummunity Update July (Berlin Meetup)
Flink Cummunity Update July (Berlin Meetup)Flink Cummunity Update July (Berlin Meetup)
Flink Cummunity Update July (Berlin Meetup)Robert Metzger
 
Apache Flink First Half of 2015 Community Update
Apache Flink First Half of 2015 Community UpdateApache Flink First Half of 2015 Community Update
Apache Flink First Half of 2015 Community UpdateRobert Metzger
 
Apache Flink Deep-Dive @ Hadoop Summit 2015 in San Jose, CA
Apache Flink Deep-Dive @ Hadoop Summit 2015 in San Jose, CAApache Flink Deep-Dive @ Hadoop Summit 2015 in San Jose, CA
Apache Flink Deep-Dive @ Hadoop Summit 2015 in San Jose, CARobert Metzger
 
Berlin Apache Flink Meetup May 2015, Community Update
Berlin Apache Flink Meetup May 2015, Community UpdateBerlin Apache Flink Meetup May 2015, Community Update
Berlin Apache Flink Meetup May 2015, Community UpdateRobert Metzger
 

More from Robert Metzger (20)

How to Contribute to Apache Flink (and Flink at the Apache Software Foundation)
How to Contribute to Apache Flink (and Flink at the Apache Software Foundation)How to Contribute to Apache Flink (and Flink at the Apache Software Foundation)
How to Contribute to Apache Flink (and Flink at the Apache Software Foundation)
 
dA Platform Overview
dA Platform OverviewdA Platform Overview
dA Platform Overview
 
Apache Flink @ Tel Aviv / Herzliya Meetup
Apache Flink @ Tel Aviv / Herzliya MeetupApache Flink @ Tel Aviv / Herzliya Meetup
Apache Flink @ Tel Aviv / Herzliya Meetup
 
Apache Flink Community Updates November 2016 @ Berlin Meetup
Apache Flink Community Updates November 2016 @ Berlin MeetupApache Flink Community Updates November 2016 @ Berlin Meetup
Apache Flink Community Updates November 2016 @ Berlin Meetup
 
A Data Streaming Architecture with Apache Flink (berlin Buzzwords 2016)
A Data Streaming Architecture with Apache Flink (berlin Buzzwords 2016)A Data Streaming Architecture with Apache Flink (berlin Buzzwords 2016)
A Data Streaming Architecture with Apache Flink (berlin Buzzwords 2016)
 
Community Update May 2016 (January - May) | Berlin Apache Flink Meetup
Community Update May 2016 (January - May) | Berlin Apache Flink MeetupCommunity Update May 2016 (January - May) | Berlin Apache Flink Meetup
Community Update May 2016 (January - May) | Berlin Apache Flink Meetup
 
GOTO Night Amsterdam - Stream processing with Apache Flink
GOTO Night Amsterdam - Stream processing with Apache FlinkGOTO Night Amsterdam - Stream processing with Apache Flink
GOTO Night Amsterdam - Stream processing with Apache Flink
 
QCon London - Stream Processing with Apache Flink
QCon London - Stream Processing with Apache FlinkQCon London - Stream Processing with Apache Flink
QCon London - Stream Processing with Apache Flink
 
January 2016 Flink Community Update & Roadmap 2016
January 2016 Flink Community Update & Roadmap 2016January 2016 Flink Community Update & Roadmap 2016
January 2016 Flink Community Update & Roadmap 2016
 
Flink Community Update December 2015: Year in Review
Flink Community Update December 2015: Year in ReviewFlink Community Update December 2015: Year in Review
Flink Community Update December 2015: Year in Review
 
Apache Flink Meetup Munich (November 2015): Flink Overview, Architecture, Int...
Apache Flink Meetup Munich (November 2015): Flink Overview, Architecture, Int...Apache Flink Meetup Munich (November 2015): Flink Overview, Architecture, Int...
Apache Flink Meetup Munich (November 2015): Flink Overview, Architecture, Int...
 
Chicago Flink Meetup: Flink's streaming architecture
Chicago Flink Meetup: Flink's streaming architectureChicago Flink Meetup: Flink's streaming architecture
Chicago Flink Meetup: Flink's streaming architecture
 
Flink September 2015 Community Update
Flink September 2015 Community UpdateFlink September 2015 Community Update
Flink September 2015 Community Update
 
Architecture of Flink's Streaming Runtime @ ApacheCon EU 2015
Architecture of Flink's Streaming Runtime @ ApacheCon EU 2015Architecture of Flink's Streaming Runtime @ ApacheCon EU 2015
Architecture of Flink's Streaming Runtime @ ApacheCon EU 2015
 
Click-Through Example for Flink’s KafkaConsumer Checkpointing
Click-Through Example for Flink’s KafkaConsumer CheckpointingClick-Through Example for Flink’s KafkaConsumer Checkpointing
Click-Through Example for Flink’s KafkaConsumer Checkpointing
 
August Flink Community Update
August Flink Community UpdateAugust Flink Community Update
August Flink Community Update
 
Flink Cummunity Update July (Berlin Meetup)
Flink Cummunity Update July (Berlin Meetup)Flink Cummunity Update July (Berlin Meetup)
Flink Cummunity Update July (Berlin Meetup)
 
Apache Flink First Half of 2015 Community Update
Apache Flink First Half of 2015 Community UpdateApache Flink First Half of 2015 Community Update
Apache Flink First Half of 2015 Community Update
 
Apache Flink Deep-Dive @ Hadoop Summit 2015 in San Jose, CA
Apache Flink Deep-Dive @ Hadoop Summit 2015 in San Jose, CAApache Flink Deep-Dive @ Hadoop Summit 2015 in San Jose, CA
Apache Flink Deep-Dive @ Hadoop Summit 2015 in San Jose, CA
 
Berlin Apache Flink Meetup May 2015, Community Update
Berlin Apache Flink Meetup May 2015, Community UpdateBerlin Apache Flink Meetup May 2015, Community Update
Berlin Apache Flink Meetup May 2015, Community Update
 

Recently uploaded

Human Factors of XR: Using Human Factors to Design XR Systems
Human Factors of XR: Using Human Factors to Design XR SystemsHuman Factors of XR: Using Human Factors to Design XR Systems
Human Factors of XR: Using Human Factors to Design XR SystemsMark Billinghurst
 
Tampa BSides - Chef's Tour of Microsoft Security Adoption Framework (SAF)
Tampa BSides - Chef's Tour of Microsoft Security Adoption Framework (SAF)Tampa BSides - Chef's Tour of Microsoft Security Adoption Framework (SAF)
Tampa BSides - Chef's Tour of Microsoft Security Adoption Framework (SAF)Mark Simos
 
Integration and Automation in Practice: CI/CD in Mule Integration and Automat...
Integration and Automation in Practice: CI/CD in Mule Integration and Automat...Integration and Automation in Practice: CI/CD in Mule Integration and Automat...
Integration and Automation in Practice: CI/CD in Mule Integration and Automat...Patryk Bandurski
 
Kotlin Multiplatform & Compose Multiplatform - Starter kit for pragmatics
Kotlin Multiplatform & Compose Multiplatform - Starter kit for pragmaticsKotlin Multiplatform & Compose Multiplatform - Starter kit for pragmatics
Kotlin Multiplatform & Compose Multiplatform - Starter kit for pragmaticscarlostorres15106
 
The Future of Software Development - Devin AI Innovative Approach.pdf
The Future of Software Development - Devin AI Innovative Approach.pdfThe Future of Software Development - Devin AI Innovative Approach.pdf
The Future of Software Development - Devin AI Innovative Approach.pdfSeasiaInfotech2
 
"ML in Production",Oleksandr Bagan
"ML in Production",Oleksandr Bagan"ML in Production",Oleksandr Bagan
"ML in Production",Oleksandr BaganFwdays
 
Gen AI in Business - Global Trends Report 2024.pdf
Gen AI in Business - Global Trends Report 2024.pdfGen AI in Business - Global Trends Report 2024.pdf
Gen AI in Business - Global Trends Report 2024.pdfAddepto
 
Vector Databases 101 - An introduction to the world of Vector Databases
Vector Databases 101 - An introduction to the world of Vector DatabasesVector Databases 101 - An introduction to the world of Vector Databases
Vector Databases 101 - An introduction to the world of Vector DatabasesZilliz
 
CloudStudio User manual (basic edition):
CloudStudio User manual (basic edition):CloudStudio User manual (basic edition):
CloudStudio User manual (basic edition):comworks
 
My INSURER PTE LTD - Insurtech Innovation Award 2024
My INSURER PTE LTD - Insurtech Innovation Award 2024My INSURER PTE LTD - Insurtech Innovation Award 2024
My INSURER PTE LTD - Insurtech Innovation Award 2024The Digital Insurer
 
"Debugging python applications inside k8s environment", Andrii Soldatenko
"Debugging python applications inside k8s environment", Andrii Soldatenko"Debugging python applications inside k8s environment", Andrii Soldatenko
"Debugging python applications inside k8s environment", Andrii SoldatenkoFwdays
 
Anypoint Exchange: It’s Not Just a Repo!
Anypoint Exchange: It’s Not Just a Repo!Anypoint Exchange: It’s Not Just a Repo!
Anypoint Exchange: It’s Not Just a Repo!Manik S Magar
 
My Hashitalk Indonesia April 2024 Presentation
My Hashitalk Indonesia April 2024 PresentationMy Hashitalk Indonesia April 2024 Presentation
My Hashitalk Indonesia April 2024 PresentationRidwan Fadjar
 
SIP trunking in Janus @ Kamailio World 2024
SIP trunking in Janus @ Kamailio World 2024SIP trunking in Janus @ Kamailio World 2024
SIP trunking in Janus @ Kamailio World 2024Lorenzo Miniero
 
New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024
New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024
New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024BookNet Canada
 
Commit 2024 - Secret Management made easy
Commit 2024 - Secret Management made easyCommit 2024 - Secret Management made easy
Commit 2024 - Secret Management made easyAlfredo García Lavilla
 
WordPress Websites for Engineers: Elevate Your Brand
WordPress Websites for Engineers: Elevate Your BrandWordPress Websites for Engineers: Elevate Your Brand
WordPress Websites for Engineers: Elevate Your Brandgvaughan
 
Training state-of-the-art general text embedding
Training state-of-the-art general text embeddingTraining state-of-the-art general text embedding
Training state-of-the-art general text embeddingZilliz
 
Vertex AI Gemini Prompt Engineering Tips
Vertex AI Gemini Prompt Engineering TipsVertex AI Gemini Prompt Engineering Tips
Vertex AI Gemini Prompt Engineering TipsMiki Katsuragi
 
DevEX - reference for building teams, processes, and platforms
DevEX - reference for building teams, processes, and platformsDevEX - reference for building teams, processes, and platforms
DevEX - reference for building teams, processes, and platformsSergiu Bodiu
 

Recently uploaded (20)

Human Factors of XR: Using Human Factors to Design XR Systems
Human Factors of XR: Using Human Factors to Design XR SystemsHuman Factors of XR: Using Human Factors to Design XR Systems
Human Factors of XR: Using Human Factors to Design XR Systems
 
Tampa BSides - Chef's Tour of Microsoft Security Adoption Framework (SAF)
Tampa BSides - Chef's Tour of Microsoft Security Adoption Framework (SAF)Tampa BSides - Chef's Tour of Microsoft Security Adoption Framework (SAF)
Tampa BSides - Chef's Tour of Microsoft Security Adoption Framework (SAF)
 
Integration and Automation in Practice: CI/CD in Mule Integration and Automat...
Integration and Automation in Practice: CI/CD in Mule Integration and Automat...Integration and Automation in Practice: CI/CD in Mule Integration and Automat...
Integration and Automation in Practice: CI/CD in Mule Integration and Automat...
 
Kotlin Multiplatform & Compose Multiplatform - Starter kit for pragmatics
Kotlin Multiplatform & Compose Multiplatform - Starter kit for pragmaticsKotlin Multiplatform & Compose Multiplatform - Starter kit for pragmatics
Kotlin Multiplatform & Compose Multiplatform - Starter kit for pragmatics
 
The Future of Software Development - Devin AI Innovative Approach.pdf
The Future of Software Development - Devin AI Innovative Approach.pdfThe Future of Software Development - Devin AI Innovative Approach.pdf
The Future of Software Development - Devin AI Innovative Approach.pdf
 
"ML in Production",Oleksandr Bagan
"ML in Production",Oleksandr Bagan"ML in Production",Oleksandr Bagan
"ML in Production",Oleksandr Bagan
 
Gen AI in Business - Global Trends Report 2024.pdf
Gen AI in Business - Global Trends Report 2024.pdfGen AI in Business - Global Trends Report 2024.pdf
Gen AI in Business - Global Trends Report 2024.pdf
 
Vector Databases 101 - An introduction to the world of Vector Databases
Vector Databases 101 - An introduction to the world of Vector DatabasesVector Databases 101 - An introduction to the world of Vector Databases
Vector Databases 101 - An introduction to the world of Vector Databases
 
CloudStudio User manual (basic edition):
CloudStudio User manual (basic edition):CloudStudio User manual (basic edition):
CloudStudio User manual (basic edition):
 
My INSURER PTE LTD - Insurtech Innovation Award 2024
My INSURER PTE LTD - Insurtech Innovation Award 2024My INSURER PTE LTD - Insurtech Innovation Award 2024
My INSURER PTE LTD - Insurtech Innovation Award 2024
 
"Debugging python applications inside k8s environment", Andrii Soldatenko
"Debugging python applications inside k8s environment", Andrii Soldatenko"Debugging python applications inside k8s environment", Andrii Soldatenko
"Debugging python applications inside k8s environment", Andrii Soldatenko
 
Anypoint Exchange: It’s Not Just a Repo!
Anypoint Exchange: It’s Not Just a Repo!Anypoint Exchange: It’s Not Just a Repo!
Anypoint Exchange: It’s Not Just a Repo!
 
My Hashitalk Indonesia April 2024 Presentation
My Hashitalk Indonesia April 2024 PresentationMy Hashitalk Indonesia April 2024 Presentation
My Hashitalk Indonesia April 2024 Presentation
 
SIP trunking in Janus @ Kamailio World 2024
SIP trunking in Janus @ Kamailio World 2024SIP trunking in Janus @ Kamailio World 2024
SIP trunking in Janus @ Kamailio World 2024
 
New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024
New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024
New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024
 
Commit 2024 - Secret Management made easy
Commit 2024 - Secret Management made easyCommit 2024 - Secret Management made easy
Commit 2024 - Secret Management made easy
 
WordPress Websites for Engineers: Elevate Your Brand
WordPress Websites for Engineers: Elevate Your BrandWordPress Websites for Engineers: Elevate Your Brand
WordPress Websites for Engineers: Elevate Your Brand
 
Training state-of-the-art general text embedding
Training state-of-the-art general text embeddingTraining state-of-the-art general text embedding
Training state-of-the-art general text embedding
 
Vertex AI Gemini Prompt Engineering Tips
Vertex AI Gemini Prompt Engineering TipsVertex AI Gemini Prompt Engineering Tips
Vertex AI Gemini Prompt Engineering Tips
 
DevEX - reference for building teams, processes, and platforms
DevEX - reference for building teams, processes, and platformsDevEX - reference for building teams, processes, and platforms
DevEX - reference for building teams, processes, and platforms
 

Apache Flink Hands On

  • 1. Hands on Apache Flink How to run, debug and speed up Flink applications Robert Metzger rmetzger@apache.org @rmetzger_
  • 2. This talk • Frequently asked questions + their answers • An overview over the tooling in Flink • An outlook into the future flink.apache.org 1
  • 3. “One week of trials and errors can save up to half an hour of reading the documentation.” – Paris Hilton flink.apache.org 2
  • 4. WRITE AND TEST YOUR JOB The first step flink.apache.org 3
  • 5. Get started with an empty project • Generate a skeleton project with Maven flink.apache.org 4 mvn archetype:generate / -DarchetypeGroupId=org.apache.flink / -DarchetypeArtifactId=flink-quickstart-java / -DarchetypeVersion=0.9-SNAPSHOT you can also put “quickstart-scala” here or “0.8.1” • No need for manually downloading any .tgz or .jar files for now
  • 6. Local Development • Start Flink in your IDE for local development & debugging. flink.apache.org 5 final ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); • Use our testing framework @RunWith(Parameterized.class) class YourTest extends MultipleProgramsTestBase { @Test public void testRunWithConfiguration(){ expectedResult = "1 11n“; }}
  • 7. Debugging with the IDE flink.apache.org 6
  • 8. RUN YOUR JOB ON A (FAKE) CLUSTER Get your hands dirty flink.apache.org 7
  • 9. Got no cluster? – Renting options • Google Compute Engine [1] • Amazon EMR or any other cloud provider with preinstalled Hadoop YARN [2] • Install Flink yourself on the machines flink.apache.org 8 ./bdutil -e extensions/flink/flink_env.sh deploy [1] http://ci.apache.org/projects/flink/flink-docs-master/setup/gce_setup.html [2] http://ci.apache.org/projects/flink/flink-docs-master/setup/yarn_setup.html wget http://stratosphere-bin.amazonaws.com/flink-0.9-SNAPSHOT-bin-hadoop2.tgz tar xvzf flink-0.9-SNAPSHOT-bin-hadoop2.tgz cd flink-0.9-SNAPSHOT/ ./bin/yarn-session.sh -n 4 -jm 1024 -tm 4096
  • 10. Got no money? • Listen closely to this talk and become a freelance “Big Data Consultant” • Start a cluster locally in the meantime flink.apache.org 9 $ tar xzf flink-*.tgz $ cd flink $ bin/start-cluster.sh Starting Job Manager Starting task manager on host $ jps 5158 JobManager 5262 TaskManager
  • 11. assert hasCluster; • Submitting a job – /bin/flink (Command Line) – RemoteExecutionEnvironment (From a local or remote java app) – Web Frontend (GUI) – Per job on YARN (Command Line, directly to YARN) – Scala Shell flink.apache.org 10
  • 12. Web Frontends – Web Job Client flink.apache.org 11 Select jobs and preview plan Understand Optimizer choices
  • 13. Web Frontends – Job Manager flink.apache.org 12 Overall system status Job execution details Task Manager resource utilization
  • 14. Debugging on a cluster • Good old system out debugging – Get a logger – Start logging – You can also use System.out.println(). flink.apache.org 13 private static final Logger LOG = LoggerFactory.getLogger(YourJob.class); LOG.info("elementCount = {}", elementCount);
  • 15. Getting logs on a cluster • Non-YARN (=bare metal installation) – The logs are located in each TaskManager’s log/ directory. – ssh there and read the logs. • YARN – Make sure YARN log aggregation is enabled – Retrieve logs from YARN (once app is finished) flink.apache.org 14 $ yarn logs -applicationId <application ID>
  • 16. Flink Logs 11:42:39,233 INFO org.apache.flink.runtime.jobmanager.JobManager - -------------------------------------------------------------------------------- 11:42:39,233 INFO org.apache.flink.runtime.jobmanager.JobManager - Starting JobManager (Version: 0.9-SNAPSHOT, Rev:2e515fc, Date:27.05.2015 @ 11:24:23 CEST) 11:42:39,233 INFO org.apache.flink.runtime.jobmanager.JobManager - Current user: robert 11:42:39,233 INFO org.apache.flink.runtime.jobmanager.JobManager - JVM: OpenJDK 64-Bit Server VM - Oracle Corporation - 1.7/24.75-b04 11:42:39,233 INFO org.apache.flink.runtime.jobmanager.JobManager - Maximum heap size: 736 MiBytes 11:42:39,233 INFO org.apache.flink.runtime.jobmanager.JobManager - JAVA_HOME: (not set) 11:42:39,233 INFO org.apache.flink.runtime.jobmanager.JobManager - JVM Options: 11:42:39,233 INFO org.apache.flink.runtime.jobmanager.JobManager - -XX:MaxPermSize=256m 11:42:39,233 INFO org.apache.flink.runtime.jobmanager.JobManager - -Xms768m 11:42:39,233 INFO org.apache.flink.runtime.jobmanager.JobManager - -Xmx768m 11:42:39,233 INFO org.apache.flink.runtime.jobmanager.JobManager - -Dlog.file=/home/robert/incubator-flink/build-target/bin/../log/flink-robert-jobmanager-robert-da.log 11:42:39,233 INFO org.apache.flink.runtime.jobmanager.JobManager - -Dlog4j.configuration=file:/home/robert/incubator-flink/build-target/bin/../conf/log4j.properties 11:42:39,233 INFO org.apache.flink.runtime.jobmanager.JobManager - -Dlogback.configurationFile=file:/home/robert/incubator-flink/build-target/bin/../conf/logback.xml 11:42:39,233 INFO org.apache.flink.runtime.jobmanager.JobManager - Program Arguments: 11:42:39,233 INFO org.apache.flink.runtime.jobmanager.JobManager - --configDir 11:42:39,233 INFO org.apache.flink.runtime.jobmanager.JobManager - /home/robert/incubator-flink/build-target/bin/../conf 11:42:39,234 INFO org.apache.flink.runtime.jobmanager.JobManager - --executionMode 11:42:39,234 INFO org.apache.flink.runtime.jobmanager.JobManager - local 11:42:39,234 INFO org.apache.flink.runtime.jobmanager.JobManager - --streamingMode 11:42:39,234 INFO org.apache.flink.runtime.jobmanager.JobManager - batch 11:42:39,234 INFO org.apache.flink.runtime.jobmanager.JobManager - -------------------------------------------------------------------------------- 11:42:39,469 INFO org.apache.flink.runtime.jobmanager.JobManager - Loading configuration from /home/robert/incubator-flink/build-target/bin/../conf 11:42:39,525 INFO org.apache.flink.runtime.jobmanager.JobManager - Security is not enabled. Starting non-authenticated JobManager. 11:42:39,525 INFO org.apache.flink.runtime.jobmanager.JobManager - Starting JobManager 11:42:39,527 INFO org.apache.flink.runtime.jobmanager.JobManager - Starting JobManager actor system at localhost:6123. 11:42:40,189 INFO akka.event.slf4j.Slf4jLogger - Slf4jLogger started 11:42:40,316 INFO Remoting - Starting remoting 11:42:40,569 INFO Remoting - Remoting started; listening on addresses :[akka.tcp://flink@127.0.0.1:6123] 11:42:40,573 INFO org.apache.flink.runtime.jobmanager.JobManager - Starting JobManager actor 11:42:40,580 INFO org.apache.flink.runtime.blob.BlobServer - Created BLOB server storage directory /tmp/blobStore-50f75dc9-3001-4c1b-bc2a-6658ac21322b 11:42:40,581 INFO org.apache.flink.runtime.blob.BlobServer - Started BLOB server at 0.0.0.0:51194 - max concurrent requests: 50 - max backlog: 1000 11:42:40,613 INFO org.apache.flink.runtime.jobmanager.JobManager - Starting embedded TaskManager for JobManager's LOCAL execution mode 11:42:40,615 INFO org.apache.flink.runtime.jobmanager.JobManager - Starting JobManager at akka://flink/user/jobmanager#205521910. 11:42:40,663 INFO org.apache.flink.runtime.taskmanager.TaskManager - Messages between TaskManager and JobManager have a max timeout of 100000 milliseconds 11:42:40,666 INFO org.apache.flink.runtime.taskmanager.TaskManager - Temporary file directory '/tmp': total 7 GB, usable 7 GB (100.00% usable) 11:42:41,092 INFO org.apache.flink.runtime.io.network.buffer.NetworkBufferPool - Allocated 64 MB for network buffer pool (number of memory segments: 2048, bytes per segment: 32768). 11:42:41,511 INFO org.apache.flink.runtime.taskmanager.TaskManager - Using 0.7 of the currently free heap space for Flink managed memory (461 MB). 11:42:42,520 INFO org.apache.flink.runtime.io.disk.iomanager.IOManager - I/O manager uses directory /tmp/flink-io-4c6f4364-1975-48b7-99d9-a74e4edb7103 for spill files. 11:42:42,523 INFO org.apache.flink.runtime.jobmanager.JobManager - Starting JobManger web frontend flink.apache.org 15 Build Information JVM details Init messages
  • 17. Get logs of a running YARN application flink.apache.org 16
  • 18. Debugging on a cluster - Accumulators • Useful to verify your assumptions about the data flink.apache.org 17 class Tokenizer extends RichFlatMapFunction<String, String>> { @Override public void flatMap(String value, Collector<String> out) { getRuntimeContext() .getLongCounter("elementCount").add(1L); // do more stuff. } } Use “Rich*Functions” to get RuntimeContext
  • 19. Debugging on a cluster - Accumulators • Where can I get the accumulator results? – returned by env.execute() – displayed when executed with /bin/flink – in the JobManager web frontend flink.apache.org 18 JobExecutionResult result = env.execute("WordCount"); long ec = result.getAccumulatorResult("elementCount");
  • 20. Excursion: RichFunctions • The default functions are SAMs (Single abstract method). Interfaces with one method (for Java8 Lambdas) • There is a “Rich” variant for each function. – RichFlatMapFunction, … – Methods • open(Configuration c) & close() • getRuntimeContext() flink.apache.org 19
  • 21. Excursion: RichFunctions & RuntimeContext • The RuntimeContext provides some useful methods • getIndexOfThisSubtask () / getNumberOfParallelSubtasks() – who am I, and if yes how many? • getExecutionConfig() • Accumulators • DistributedCache flink.apache.org 20
  • 22. Attaching a remote debugger to Flink in a Cluster flink.apache.org 21
  • 23. Attaching a debugger to Flink in a cluster • Add JVM start option in flink-conf.yaml env.java.opts: “-agentlib:jdwp=….” • Open an SSH tunnel to the machine: ssh -f -N -L 5005:127.0.0.1:5005 user@host • Use your IDE to start a remote debugging session flink.apache.org 22
  • 24. JOB TUNING Make it run faster flink.apache.org 23
  • 25. Tuning options • CPU – Processing slots, threads, … • Memory – How to adjust memory usage on the TaskManager • I/O – Specifying temporary directories for spilling flink.apache.org 24
  • 26. Tell Flink how many CPUs you have • taskmanager.numberOfTaskSlots – number of parallel job instances – number of pipelines per TaskManager • recommended: number of CPU cores flink.apache.org 25 Map Reduce Map Reduce Map Reduce Map Reduce Map Reduce Map Reduce Map Reduce
  • 27. Task Manager 1 Slot 1 Slot 2 Slot 3 Task Manager 2 Slot 1 Slot 2 Slot 3 Task Manager 3 Slot 1 Slot 2 Slot 3 Task Managers: 3 Total number of processing slots: 9 flink-config.yaml: taskmanager.numberOfTaskSlots: 3 (Recommended value: Number of CPU cores) or /bin/yarn-session.sh –slots 3 –n 3 Processing slots
  • 28. Slots – Wordcount with parallelism=1 flink.apache.org 27 Task Manager 1 Slot 1 Slot 2 Slot 3 Task Manager 2 Slot 1 Slot 2 Slot 3 Task Manager 3 Slot 1 Slot 2 Slot 3 Source -> flatMap Reduce Sink When no argument given, parallelism.default from flink-config.yaml is used. Default value = 1
  • 29. Slots – Wordcount with higher parallelism (= 2 here) flink.apache.org 28Task Manager 1 Slot 1 Slot 2 Slot 3 Task Manager 2 Slot 1 Slot 2 Slot 3 Task Manager 3 Slot 1 Slot 2 Slot 3 Source -> flatMap Reduce Sink Source -> flatMap Reduce Sink Places to set parallelism for a job flink-config.yaml parallelism.default: 2 or Flink Client: ./bin/flink -p 2 or ExecutionEnvironment: env.setParallelism(2)
  • 30. Slots – Wordcount using all resources (parallelism = 9) flink.apache.org 29 Task Manager 1 Slot 1 Slot 2 Slot 3 Task Manager 2 Slot 1 Slot 2 Slot 3 Task Manager 3 Slot 1 Slot 2 Slot 3 Source -> flatMap Reduce Sink Source -> flatMap Reduce Sink Source -> flatMap Reduce Sink Source - > flatMap Reduce Sink Source -> flatMap Reduce Sink Source -> flatMap Reduce Sink Source -> flatMap Reduce Sink Source -> flatMap Reduce Sink Source -> flatMap Reduce Sink
  • 31. Slots – Setting parallelism on a per operator basis flink.apache.org 30 Task Manager 1 Slot 1 Slot 2 Slot 3 Task Manager 2 Slot 1 Slot 2 Slot 3 Task Manager 3 Slot 1 Slot 2 Slot 3 Source -> flatMap Reduce Source -> flatMap Reduce Source -> flatMap Reduce Source - > flatMap Reduce Source -> flatMap Reduce Source -> flatMap Reduce Source -> flatMap Reduce Source -> flatMap Reduce Source -> flatMap Reduce The parallelism of each operator can be set individually in the APIs counts.writeAsCsv(outputPath, "n", " ").setParallelism(1); Sink
  • 32. Slots – Setting parallelism on a per operator basis flink.apache.org 31 Task Manager 1 Slot 1 Slot 2 Slot 3 Task Manager 2 Slot 1 Slot 2 Slot 3 Task Manager 3 Slot 1 Slot 2 Slot 3 Source -> flatMap Reduce Source -> flatMap Reduce Source -> flatMap Reduce Source - > flatMap Reduce Source -> flatMap Reduce Source -> flatMap Reduce Source -> flatMap Reduce Source -> flatMap Reduce Source -> flatMap Reduce Sink The data is streamed to this Sink from all the other slots on the other TaskManagers
  • 33. Tuning options • CPU – Processing slots, threads, … • Memory – How to adjust memory usage on the TaskManager • I/O – Specifying temporary directories for spilling flink.apache.org 32
  • 35. flink.apache.org 34 taskmanager.network.numberOfBuffers relative: taskmanager.memory.fraction absolute: taskmanager.memory.size Memory in Flink - Configuration taskmanager.heap.mb or „-tm“ argument for bin/yarn-session.sh
  • 36. Memory in Flink - OOM flink.apache.org 35 2015-02-20 11:22:54 INFO JobClient:345 - java.lang.OutOfMemoryError: Java heap space at org.apache.flink.runtime.io.network.serialization.DataOutputSerializer.resize(DataOutputSerializer.java:249) at org.apache.flink.runtime.io.network.serialization.DataOutputSerializer.write(DataOutputSerializer.java:93) at org.apache.flink.api.java.typeutils.runtime.DataOutputViewStream.write(DataOutputViewStream.java:39) at com.esotericsoftware.kryo.io.Output.flush(Output.java:163) at com.esotericsoftware.kryo.io.Output.require(Output.java:142) at com.esotericsoftware.kryo.io.Output.writeBoolean(Output.java:613) at com.twitter.chill.java.BitSetSerializer.write(BitSetSerializer.java:42) at com.twitter.chill.java.BitSetSerializer.write(BitSetSerializer.java:29) at com.esotericsoftware.kryo.Kryo.writeClassAndObject(Kryo.java:599) at org.apache.flink.api.java.typeutils.runtime.KryoSerializer.serialize(KryoSerializer.java:155) at org.apache.flink.api.scala.typeutils.CaseClassSerializer.serialize(CaseClassSerializer.scala:91) at org.apache.flink.api.scala.typeutils.CaseClassSerializer.serialize(CaseClassSerializer.scala:30) at org.apache.flink.runtime.plugable.SerializationDelegate.write(SerializationDelegate.java:51) at org.apache.flink.runtime.io.network.serialization.SpanningRecordSerializer.addRecord(SpanningRecordSerializer.java:76 at org.apache.flink.runtime.io.network.api.RecordWriter.emit(RecordWriter.java:82) at org.apache.flink.runtime.operators.shipping.OutputCollector.collect(OutputCollector.java:88) at org.apache.flink.api.scala.GroupedDataSet$$anon$2.reduce(GroupedDataSet.scala:262) at org.apache.flink.runtime.operators.GroupReduceDriver.run(GroupReduceDriver.java:124) at org.apache.flink.runtime.operators.RegularPactTask.run(RegularPactTask.java:493) at org.apache.flink.runtime.operators.RegularPactTask.invoke(RegularPactTask.java:360) at org.apache.flink.runtime.execution.RuntimeEnvironment.run(RuntimeEnvironment.java:257) at java.lang.Thread.run(Thread.java:745) Memory is missing here Reduce managed memory reduce taskmanager. memory.fraction
  • 37. Memory in Flink – Network buffers flink.apache.org 36 Memory is missing here Managed memory will shrink automatically Error: java.lang.Exception: Failed to deploy the task CHAIN Reduce(org.okkam.flink.maintenance.deduplication.blocking.RemoveDuplicateReduceGr oupFunction) -> Combine(org.apache.flink.api.java.operators.DistinctOperator$DistinctFunction) (15/28) - execution #0 to slot SubSlot 5 (cab978f80c0cb7071136cd755e971be9 (5) - ALLOCATED/ALIVE): org.apache.flink.runtime.io.network.InsufficientResourcesException: okkam-nano- 2.okkam.it has not enough buffers to safely execute CHAIN Reduce(org.okkam.flink.maintenance.deduplication.blocking.RemoveDuplicateReduceGr oupFunction) -> Combine(org.apache.flink.api.java.operators.DistinctOperator$DistinctFunction) (36 buffers missing) increase „taskmanager.network.numberOfBuffers“
  • 38. What are these buffers needed for? flink.apache.org 37 TaskManager 1 Slot 2 Map Reduce Slot 1 TaskManager 2 Slot 2 Slot 1 A small Flink cluster with 4 processing slots (on 2 Task Managers) A simple MapReduce Job in Flink:
  • 39. What are these buffers needed for? flink.apache.org 38 Map Reduce job with a parallelism of 2 and 2 processing slots per Machine TaskManager 1 TaskManager 2 Slot1Slot2 Map Map Reduce Reduce Map Map Reduce Reduce Map Map Reduce Reduce Map Map Reduce Reduce Slot1Slot2 Network buffer 8 buffers for outgoing data 8 buffers for incoming data
  • 40. What are these buffers needed for? flink.apache.org 39 Map Reduce job with a parallelism of 2 and 2 processing slots per Machine TaskManager 1 TaskManager 2 Slot1Slot2 Map Map Reduce Reduce Map Map Reduce Reduce Map Map Reduce Reduce Map Map Reduce Reduce
  • 41. Tuning options • CPU – Processing slots, threads, … • Memory – How to adjust memory usage on the TaskManager • I/O – Specifying temporary directories for spilling flink.apache.org 40
  • 42. Tuning options • Memory – How to adjust memory usage on the TaskManager • CPU – Processing slots, threads, … • I/O – Specifying temporary directories for spilling flink.apache.org 41
  • 43. Disk I/O • Sometimes your data doesn’t fit into main memory, so we have to spill to disk – taskmanager.tmp.dirs: /mnt/disk1,/mnt/disk2 • Use real local disks only (no tmpfs or NAS) flink.apache.org 42 Reader Thread Disk 1 Writer Thread Reader Thread Writer Thread Disk 2 Task Manager
  • 44. Outlook • Per job monitoring & metrics • Less configuration values with dynamic memory management • Download operator results to debug them locally flink.apache.org 43
  • 45. Join our community • RTFM (= read the documentation) • Mailing lists – Subscribe: user-subscribe@flink.apache.org – Ask: user@flink.apache.org • Stack Overflow – tag with “flink” so that we get an email notification ;) • IRC: freenode#flink • Read the code, its open source  flink.apache.org 44
  • 46. Flink Forward registration & call for abstracts is open now flink.apache.org 45 • 12/13 October 2015 • Kulturbrauerei Berlin • With Flink Workshops / Trainings!

Editor's Notes

  1. My goal: Everybody finds a new, useful feature of flink in this talk!
  2. scripts, no typing required
  3. An entire slide about cloud computing without having “cloud” on it
  4. bin/start-cluster.sh is also the option for those with Flink “on premise”
  5. this way you can also start multiple threads per disk