mirror of
https://github.com/elastic/logstash.git
synced 2025-04-24 06:37:19 -04:00
parent
e5cc9c407d
commit
fc128fe7ff
3 changed files with 58 additions and 75 deletions
BIN
docs/static/images/pipeline_correct_load.png
vendored
Normal file
BIN
docs/static/images/pipeline_correct_load.png
vendored
Normal file
Binary file not shown.
After Width: | Height: | Size: 118 KiB |
BIN
docs/static/images/pipeline_overload.png
vendored
Normal file
BIN
docs/static/images/pipeline_overload.png
vendored
Normal file
Binary file not shown.
After Width: | Height: | Size: 113 KiB |
133
docs/static/life-of-an-event.asciidoc
vendored
133
docs/static/life-of-an-event.asciidoc
vendored
|
@ -81,95 +81,78 @@ For more information about the available codecs, see
|
|||
[float]
|
||||
=== Fault Tolerance
|
||||
|
||||
Events are passed from stage to stage using internal queues implemented with a
|
||||
Ruby `SizedQueue`. A `SizedQueue` has a maximum number of items it can contain.
|
||||
When the queue is at maximum capacity, all writes to the queue are blocked.
|
||||
Logstash keeps all events in main memory during processing. Logstash responds to a SIGTERM by attempting to halt inputs and waiting for pending events to finish processing before shutting down. When the pipeline cannot be flushed due to a stuck output or filter, Logstash waits indefinitely. For example, when a pipeline sends output to a database that is unreachable by the Logstash instance, the instance waits indefinitely after receiving a SIGTERM.
|
||||
|
||||
Logstash sets the size of each queue to 20. This means a maximum of 20 events
|
||||
can be pending for the next stage, which helps prevent data loss and keeps
|
||||
Logstash from acting as a data storage system. These internal queues are not
|
||||
intended for storing messages long-term.
|
||||
To enable Logstash to detect these situations and terminate with a stalled pipeline, use the `--allow-unsafe-shutdown` flag.
|
||||
|
||||
The small queue sizes mean that Logstash simply blocks and stalls safely when
|
||||
there's a heavy load or temporary pipeline problems. The alternatives would be
|
||||
to either have an unlimited queue or drop messages when there's a problem. An
|
||||
unlimited queue can grow unbounded and eventually exceed memory, causing a crash
|
||||
that loses all of the queued messages. In most cases, dropping messages outright
|
||||
is equally undesirable.
|
||||
|
||||
An output can fail or have problems due to downstream issues, such as a full
|
||||
disk, permissions problems, temporary network failures, or service outages. Most
|
||||
outputs keep retrying to ship events affected by the failure.
|
||||
|
||||
If an output is failing, the output thread waits until the output is able to
|
||||
successfully send the message. The output stops reading from the output queue,
|
||||
which means the queue can fill up with events.
|
||||
|
||||
When the output queue is full, filters are blocked because they cannot write new
|
||||
events to the output queue. While they are blocked from writing to the output
|
||||
queue, filters stop reading from the filter queue. Eventually, this can cause
|
||||
the filter queue (input -> filter) to fill up.
|
||||
|
||||
A full filter queue blocks inputs from writing to the filters. This causes all
|
||||
inputs to stop processing data from wherever they're getting new events.
|
||||
|
||||
In ideal circumstances, this behaves similarly to when the tcp window closes to
|
||||
0. No new data is sent because the receiver hasn't finished processing the
|
||||
current queue of data, but as soon as the downstream (output) problem is
|
||||
resolved, messages start flowing again.
|
||||
WARNING: Unsafe shutdowns, force-kills of the Logstash process, or crashes of the Logstash process for any other reason result in data loss. Shut down Logstash safely whenever possible.
|
||||
|
||||
[float]
|
||||
=== Thread Model
|
||||
==== Execution Model
|
||||
|
||||
The thread model in Logstash is currently:
|
||||
The Logstash pipeline coordinates the execution of inputs, filters, and outputs. The following schematic sketches the data flow of a pipeline:
|
||||
|
||||
[source,js]
|
||||
----------------------------------
|
||||
input threads | filter worker threads | output worker
|
||||
----------------------------------
|
||||
---------------------------------------------------
|
||||
input threads | pipeline worker threads
|
||||
---------------------------------------------------
|
||||
|
||||
Filters are optional, so if you have no filters defined it is simply:
|
||||
Pipelines in the current release of Logstash process filtering and output in the same thread. Prior to the 2.2 release of Logstash, filtering and output were distinct stages handled by distinct threads.
|
||||
This change to the Logstash architecture improves performance and enables future persistence capabilities. The new pipeline substantially improves thread liveness, decreases resource usage, and increases throughput. The current Logstash pipeline is a micro-batching pipeline, which is inherently more efficient than a one-at-a-time approach. These efficiencies come in many places, two of the more prominent ones being a reduction in contention and a consequent improvement in thread liveness. These efficiencies are especially noticeable on many-core machines.
|
||||
|
||||
[source,js]
|
||||
----------------------------------
|
||||
input threads | output worker
|
||||
----------------------------------
|
||||
Each `input {}` statement in the Logstash configuration file runs in its own thread. Inputs write events to a common Java https://docs.oracle.com/javase/7/docs/api/java/util/concurrent/SynchronousQueue.html[SynchronousQueue]. This queue holds no events, instead transferring each pushed event to a free worker, blocking if all workers are busy. Each pipeline worker thread takes a batch of events off this queue, creating a buffer per worker, runs the batch of events through the configured filters, then runs the filtered events through any outputs. The size of the batch and number of pipeline worker threads are configurable. The following pseudocode illustrates the process flow:
|
||||
|
||||
Each input runs in a thread by itself. This prevents busier inputs from being
|
||||
blocked by slower ones. It also allows for easier containment of scope because
|
||||
each input has a thread.
|
||||
[source,ruby]
|
||||
synchronous_queue = SynchronousQueue.new
|
||||
inputs.each do |input|
|
||||
Thread.new do
|
||||
input.run(synchronous_queue)
|
||||
end
|
||||
end
|
||||
num_pipeline_workers.times do
|
||||
Thread.new do
|
||||
while true
|
||||
batch = take_batch(synchronous_queue, batch_size, batch_delay)
|
||||
output_batch(filter_batch(batch))
|
||||
end
|
||||
end
|
||||
end
|
||||
wait_for_threads_to_terminate()
|
||||
|
||||
The filter thread model is a 'worker' model where each worker receives an event
|
||||
and applies all filters, in order, before sending it on to the output queue.
|
||||
This allows scalability across CPUs because many filters are CPU intensive
|
||||
(permitting that we have thread safety).
|
||||
There are three configurable options in the pipeline, `--pipeline-workers`, `--pipeline-batch-size`, and `--pipeline-batch-delay`.
|
||||
The `--pipeline-workers` or `-w` parameter determines how many threads to run for filter and output processing. If you find that events are backing up, or that the CPU is not saturated, consider increasing the value of this parameter to make better use of available processing power. Good results can even be found increasing this number past the number of available processors as these threads may spend significant time in an I/O wait state when writing to external systems. Legal values for this parameter are positive integers.
|
||||
|
||||
The default number of filter workers is 1, but you can increase this number by
|
||||
specifying the '-w' flag when you run the Logstash agent.
|
||||
The `--pipeline-batch-size` or `-b` parameter defines the maximum number of events an individual worker thread collects before attempting to execute filters and outputs. Larger batch sizes are generally more efficient, but increase memory overhead. Some hardware configurations require you to increase JVM heap size by setting the `LS_HEAP_SIZE` variable to avoid performance degradation with this option. Values of this parameter in excess of the optimum range cause performance degradation due to frequent garbage collection or JVM crashes related to out-of-memory exceptions. Output plugins can process each batch as a logical unit. The Elasticsearch output, for example, issues https://www.elastic.co/guide/en/elasticsearch/reference/current/docs-bulk.html[bulk requests] for each batch received. Tuning the `-b` parameter adjusts the size of bulk requests sent to Elasticsearch.
|
||||
|
||||
The output worker model is currently a single thread. Outputs receive events in
|
||||
the order the outputs are defined in the config file.
|
||||
|
||||
Outputs might decide to temporarily buffer events before publishing them. One
|
||||
example of this is the `elasticsearch` output, which buffers events and flushes
|
||||
them all at once using a separate thread. This mechanism (buffering many events
|
||||
and writing in a separate thread) can improve performance because it prevents
|
||||
the Logstash pipeline from being stalled waiting for a response from
|
||||
elasticsearch.
|
||||
The `--pipeline-batch-delay` option rarely needs to be tuned. This option adjusts the latency of the Logstash pipeline. Pipeline batch delay is the maximum amount of time in milliseconds that Logstash waits for new messages after receiving an event in the current pipeline worker thread. After this time elapses, Logstash beings to execute filters and outputs.The maximum time that Logstash waits between receiving an event and processing that event in a filter is the product of the `pipeline_batch_delay` and `pipeline_batch_size` settings.
|
||||
|
||||
[float]
|
||||
=== Resource Usage
|
||||
==== Notes on Pipeline Configuration and Performance
|
||||
|
||||
Logstash typically has at least 3 threads (2 if you have no filters). One input
|
||||
thread, one filter worker thread, and one output thread. If you see Logstash
|
||||
using multiple CPUs, this is likely why. If you want to know more about what
|
||||
each thread is doing, you should read this article:
|
||||
http://www.semicomplete.com/blog/geekery/debugging-java-performance.html[Debugging Java Performance].
|
||||
Threads in Java have names and you can use `jstack` and `top` to figure out who
|
||||
is using what resources.
|
||||
The total number of inflight events is determined by the product of the `pipeline_workers` and `pipeline_batch_size` parameters. This product is referred to as the _inflight count_. Keep the value of the inflight count in mind as you adjust the `pipeline_workers` and `pipeline_batch_size` parameters. Pipelines that intermittently receive large events at irregular intervals require sufficient memory to handle these spikes. Configure the `LS_HEAP_SIZE` option accordingly.
|
||||
|
||||
On Linux platforms, Logstash labels all the threads it can with something
|
||||
descriptive. For example, inputs show up as `<inputname`, filter workers show up
|
||||
as `|worker`, and outputs show up as `>outputworker`. Where possible, other
|
||||
threads are also labeled to help you identify their purpose should you wonder
|
||||
why they are consuming resources!
|
||||
The Logstash defaults are chosen to provide fast, safe performance for most users. To increase performance, increase the number of pipeline workers or the batch size, taking into account the following suggestions:
|
||||
|
||||
Measure each change to make sure it increases, rather than decreases, performance.
|
||||
Ensure that you leave enough memory available to cope with a sudden increase in event size. For example, an application that generates exceptions that are represented as large blobs of text.
|
||||
The number of workers may be set higher than the number of CPU cores since outputs often spend idle time in I/O wait conditions.
|
||||
|
||||
Threads in Java have names and you can use the `jstack`, `top`, and the VisualVM graphical tools to figure out which resources a given thread uses.
|
||||
|
||||
On Linux platforms, Logstash labels all the threads it can with something descriptive. For example, inputs show up as `[base]<inputname`, filter/output workers show up as `[base]>workerN`, where N is an integer. Where possible, other threads are also labeled to help you identify their purpose.
|
||||
|
||||
[float]
|
||||
==== Profiling the Heap
|
||||
|
||||
When tuning Logstash you may have to adjust the heap size. You can use the https://visualvm.java.net/[VisualVM] tool to profile the heap. The *Monitor* pane in particular is useful for checking whether your heap allocation is sufficient for the current workload. The screenshots below show sample *Monitor* panes. The first pane examines a Logstash instance configured with too many inflight events. The second pane examines a Logstash instance configured with an appropriate amount of inflight events. Note that the specific batch sizes used here are most likely not applicable to your specific workload, as the memory demands of Logstash vary in large part based on the type of messages you are sending.
|
||||
|
||||
image::static/images/pipeline_overload.png[]
|
||||
|
||||
image::static/images/pipeline_correct_load.png[]
|
||||
|
||||
In the first example we see that the CPU isn’t being used very efficiently. In fact, the JVM is often times having to stop the VM for “full GCs”. Full garbage collections are a common symptom of excessive memory pressure. This is visible in the spiky pattern on the CPU chart. In the more efficiently configured example, the GC graph pattern is more smooth, and the CPU is used in a more uniform manner. You can also see that there is ample headroom between the allocated heap size, and the maximum allowed, giving the JVM GC a lot of room to work with.
|
||||
|
||||
Examining the in-depth GC statistics with a tool similar to the excellent https://visualvm.java.net/plugins.html[VisualGC] plugin shows that the over-allocated VM spends very little time in the efficient Eden GC, compared to the time spent in the more resource-intensive Old Gen “Full” GCs.
|
||||
|
||||
NOTE: As long as the GC pattern is acceptable, heap sizes that occasionally increase to the maximum are acceptable. Such heap size spikes happen in response to a burst of large events passing through the pipeline. In general practice, maintain a gap between the used amount of heap memory and the maximum.
|
||||
This document is not a comprehensive guide to JVM GC tuning. Read the official http://www.oracle.com/webfolder/technetwork/tutorials/obe/java/gc01/index.html[Oracle guide] for more information on the topic. We also recommend reading http://www.semicomplete.com/blog/geekery/debugging-java-performance.html[Debugging Java Performance].
|
||||
|
|
Loading…
Add table
Add a link
Reference in a new issue