mirror of
https://github.com/elastic/logstash.git
synced 2025-04-24 14:47:19 -04:00
Introduce DeadLetterQueue to the Execution Context (#6894)
* Introduce a DeadLetterQueueFactory DeadLetterQueueFactory is a static class that keeps a static collection of DeadLetterQueueWriteManagers per pipeline that has plugins requesting to use it. * DeadLetterQueue was added as a first-class field in the execution context that input/filter/output plugins can leverage
This commit is contained in:
parent
69f5e46b0b
commit
458910bcb9
23 changed files with 460 additions and 68 deletions
|
@ -131,6 +131,16 @@
|
|||
#
|
||||
# queue.checkpoint.interval: 1000
|
||||
#
|
||||
# ------------ Dead-Letter Queue Settings --------------
|
||||
# Flag to turn on dead-letter queue.
|
||||
#
|
||||
# dead_letter_queue.enable: false
|
||||
#
|
||||
# If using dead_letter_queue.enable: true, the directory path where the data files will be stored.
|
||||
# Default is path.data/dead_letter_queue
|
||||
#
|
||||
# path.dead_letter_queue:
|
||||
#
|
||||
# ------------ Metrics Settings --------------
|
||||
#
|
||||
# Bind address for the metrics REST endpoint
|
||||
|
|
8
docs/static/settings-file.asciidoc
vendored
8
docs/static/settings-file.asciidoc
vendored
|
@ -134,6 +134,14 @@ The `logstash.yml` file includes the following settings:
|
|||
| The interval in milliseconds when a checkpoint is forced on the head page when persistent queues are enabled (`queue.type: persisted`). Specify `queue.checkpoint.interval: 0` for no periodic checkpoint.
|
||||
| 1000
|
||||
|
||||
| `dead_letter_queue.enable`
|
||||
| Flag to instruct Logstash to enable the DLQ feature supported by plugins.
|
||||
| `false`
|
||||
|
||||
| `path.dead_letter_queue`
|
||||
| The directory path where the data files will be stored for the dead-letter queue.
|
||||
| `path.data/dead_letter_queue`
|
||||
|
||||
| `http.host`
|
||||
| The bind address for the metrics REST endpoint.
|
||||
| `"127.0.0.1"`
|
||||
|
|
|
@ -50,6 +50,7 @@ module LogStash
|
|||
Setting::Numeric.new("queue.checkpoint.acks", 1024), # 0 is unlimited
|
||||
Setting::Numeric.new("queue.checkpoint.writes", 1024), # 0 is unlimited
|
||||
Setting::Numeric.new("queue.checkpoint.interval", 1000), # 0 is no time-based checkpointing
|
||||
Setting::Boolean.new("dead_letter_queue.enable", false),
|
||||
Setting::TimeValue.new("slowlog.threshold.warn", "-1"),
|
||||
Setting::TimeValue.new("slowlog.threshold.info", "-1"),
|
||||
Setting::TimeValue.new("slowlog.threshold.debug", "-1"),
|
||||
|
@ -59,13 +60,21 @@ module LogStash
|
|||
# Compute the default queue path based on `path.data`
|
||||
default_queue_file_path = ::File.join(SETTINGS.get("path.data"), "queue")
|
||||
SETTINGS.register Setting::WritableDirectory.new("path.queue", default_queue_file_path)
|
||||
|
||||
# Compute the default dead_letter_queue path based on `path.data`
|
||||
default_dlq_file_path = ::File.join(SETTINGS.get("path.data"), "dead_letter_queue")
|
||||
SETTINGS.register Setting::WritableDirectory.new("path.dead_letter_queue", default_dlq_file_path)
|
||||
|
||||
SETTINGS.on_post_process do |settings|
|
||||
# If the data path is overridden but the queue path isn't recompute the queue path
|
||||
# We need to do this at this stage because of the weird execution order
|
||||
# our monkey-patched Clamp follows
|
||||
if settings.set?("path.data") && !settings.set?("path.queue")
|
||||
settings.set_value("path.queue", ::File.join(settings.get("path.data"), "queue"))
|
||||
if settings.set?("path.data")
|
||||
if !settings.set?("path.queue")
|
||||
settings.set_value("path.queue", ::File.join(settings.get("path.data"), "queue"))
|
||||
end
|
||||
if !settings.set?("path.dead_letter_queue")
|
||||
settings.set_value("path.dead_letter_queue", ::File.join(settings.get("path.data"), "dead_letter_queue"))
|
||||
end
|
||||
end
|
||||
end
|
||||
|
||||
|
|
|
@ -1,13 +1,17 @@
|
|||
# encoding: utf-8
|
||||
require "logstash/util/dead_letter_queue_manager"
|
||||
module LogStash
|
||||
class ExecutionContext
|
||||
attr_reader :pipeline, :agent
|
||||
attr_reader :pipeline, :agent, :dlq_writer
|
||||
|
||||
def initialize(pipeline, agent)
|
||||
def initialize(pipeline, agent, plugin_id, plugin_type, dlq_writer)
|
||||
@pipeline = pipeline
|
||||
@agent = agent
|
||||
@plugin_id = plugin_id
|
||||
@plugin_type = plugin_type
|
||||
@dlq_writer = LogStash::Util::PluginDeadLetterQueueWriter.new(dlq_writer, @plugin_id, @plugin_type)
|
||||
end
|
||||
|
||||
|
||||
def pipeline_id
|
||||
@pipeline.pipeline_id
|
||||
end
|
||||
|
|
|
@ -105,6 +105,8 @@ class LogStash::Inputs::Base < LogStash::Plugin
|
|||
super
|
||||
# There is no easy way to propage an instance variable into the codec, because the codec
|
||||
# are created at the class level
|
||||
# TODO(talevy): Codecs should have their own execution_context, for now they will inherit their
|
||||
# parent plugin's
|
||||
@codec.execution_context = context
|
||||
context
|
||||
end
|
||||
|
|
|
@ -109,6 +109,8 @@ class LogStash::Outputs::Base < LogStash::Plugin
|
|||
super
|
||||
# There is no easy way to propage an instance variable into the codec, because the codec
|
||||
# are created at the class level
|
||||
# TODO(talevy): Codecs should have their own execution_context, for now they will inherit their
|
||||
# parent plugin's
|
||||
@codec.execution_context = context
|
||||
context
|
||||
end
|
||||
|
|
|
@ -18,12 +18,16 @@ require "logstash/instrument/null_metric"
|
|||
require "logstash/instrument/namespaced_null_metric"
|
||||
require "logstash/instrument/collector"
|
||||
require "logstash/instrument/wrapped_write_client"
|
||||
require "logstash/util/dead_letter_queue_manager"
|
||||
require "logstash/output_delegator"
|
||||
require "logstash/filter_delegator"
|
||||
require "logstash/queue_factory"
|
||||
require "logstash/compiler"
|
||||
require "logstash/execution_context"
|
||||
|
||||
java_import org.logstash.common.DeadLetterQueueFactory
|
||||
java_import org.logstash.common.io.DeadLetterQueueWriter
|
||||
|
||||
module LogStash; class BasePipeline
|
||||
include LogStash::Util::Loggable
|
||||
|
||||
|
@ -49,7 +53,13 @@ module LogStash; class BasePipeline
|
|||
@inputs = nil
|
||||
@filters = nil
|
||||
@outputs = nil
|
||||
@execution_context = LogStash::ExecutionContext.new(self, agent)
|
||||
@agent = agent
|
||||
|
||||
if settings.get_value("dead_letter_queue.enable")
|
||||
@dlq_writer = DeadLetterQueueFactory.getWriter(pipeline_id, settings.get_value("path.dead_letter_queue"))
|
||||
else
|
||||
@dlq_writer = LogStash::Util::DummyDeadLetterQueueWriter.new
|
||||
end
|
||||
|
||||
grammar = LogStashConfigParser.new
|
||||
parsed_config = grammar.parse(config_str)
|
||||
|
@ -100,16 +110,18 @@ module LogStash; class BasePipeline
|
|||
|
||||
klass = Plugin.lookup(plugin_type, name)
|
||||
|
||||
execution_context = ExecutionContext.new(self, @agent, id, klass.config_name, @dlq_writer)
|
||||
|
||||
if plugin_type == "output"
|
||||
OutputDelegator.new(@logger, klass, type_scoped_metric, @execution_context, OutputDelegatorStrategyRegistry.instance, args)
|
||||
OutputDelegator.new(@logger, klass, type_scoped_metric, execution_context, OutputDelegatorStrategyRegistry.instance, args)
|
||||
elsif plugin_type == "filter"
|
||||
FilterDelegator.new(@logger, klass, type_scoped_metric, @execution_context, args)
|
||||
FilterDelegator.new(@logger, klass, type_scoped_metric, execution_context, args)
|
||||
else # input
|
||||
input_plugin = klass.new(args)
|
||||
scoped_metric = type_scoped_metric.namespace(id.to_sym)
|
||||
scoped_metric.gauge(:name, input_plugin.config_name)
|
||||
input_plugin.metric = scoped_metric
|
||||
input_plugin.execution_context = @execution_context
|
||||
input_plugin.execution_context = execution_context
|
||||
input_plugin
|
||||
end
|
||||
end
|
||||
|
@ -180,6 +192,7 @@ module LogStash; class Pipeline < BasePipeline
|
|||
)
|
||||
@drain_queue = @settings.get_value("queue.drain")
|
||||
|
||||
|
||||
@events_filtered = Concurrent::AtomicFixnum.new(0)
|
||||
@events_consumed = Concurrent::AtomicFixnum.new(0)
|
||||
|
||||
|
@ -191,8 +204,6 @@ module LogStash; class Pipeline < BasePipeline
|
|||
@force_shutdown = Concurrent::AtomicBoolean.new(false)
|
||||
end # def initialize
|
||||
|
||||
|
||||
|
||||
def ready?
|
||||
@ready.value
|
||||
end
|
||||
|
@ -305,6 +316,7 @@ module LogStash; class Pipeline < BasePipeline
|
|||
def close
|
||||
@filter_queue_client.close
|
||||
@queue.close
|
||||
@dlq_writer.close
|
||||
end
|
||||
|
||||
def transition_to_running
|
||||
|
|
|
@ -3,6 +3,7 @@ require "logstash/namespace"
|
|||
require "logstash/logging"
|
||||
require "logstash/config/mixin"
|
||||
require "logstash/instrument/null_metric"
|
||||
require "logstash/util/dead_letter_queue_manager"
|
||||
require "concurrent"
|
||||
require "securerandom"
|
||||
|
||||
|
|
61
logstash-core/lib/logstash/util/dead_letter_queue_manager.rb
Normal file
61
logstash-core/lib/logstash/util/dead_letter_queue_manager.rb
Normal file
|
@ -0,0 +1,61 @@
|
|||
require 'logstash/environment'
|
||||
|
||||
module LogStash; module Util
|
||||
class PluginDeadLetterQueueWriter
|
||||
|
||||
attr_reader :plugin_id, :plugin_type, :inner_writer
|
||||
|
||||
def initialize(inner_writer, plugin_id, plugin_type)
|
||||
@plugin_id = plugin_id
|
||||
@plugin_type = plugin_type
|
||||
@inner_writer = inner_writer
|
||||
end
|
||||
|
||||
def write(logstash_event, reason)
|
||||
if @inner_writer && @inner_writer.is_open
|
||||
@inner_writer.writeEntry(logstash_event.to_java, @plugin_type, @plugin_id, reason)
|
||||
end
|
||||
end
|
||||
|
||||
def close
|
||||
if @inner_writer && @inner_writer.is_open
|
||||
@inner_writer.close
|
||||
end
|
||||
end
|
||||
end
|
||||
|
||||
class DummyDeadLetterQueueWriter
|
||||
# class uses to represent a writer when dead_letter_queue is disabled
|
||||
def initialize
|
||||
end
|
||||
|
||||
def write(logstash_event, reason)
|
||||
# noop
|
||||
end
|
||||
|
||||
def is_open
|
||||
false
|
||||
end
|
||||
|
||||
def close
|
||||
# noop
|
||||
end
|
||||
end
|
||||
|
||||
class DeadLetterQueueFactory
|
||||
java_import org.logstash.common.DeadLetterQueueFactory
|
||||
|
||||
def self.get(pipeline_id)
|
||||
if LogStash::SETTINGS.get("dead_letter_queue.enable")
|
||||
return DeadLetterQueueWriter.new(
|
||||
DeadLetterQueueFactory.getWriter(pipeline_id, LogStash::SETTINGS.get("path.dead_letter_queue")))
|
||||
else
|
||||
return DeadLetterQueueWriter.new(nil)
|
||||
end
|
||||
end
|
||||
|
||||
def self.close(pipeline_id)
|
||||
DeadLetterQueueFactory.close(pipeline_id)
|
||||
end
|
||||
end
|
||||
end end
|
|
@ -26,18 +26,6 @@ describe LogStash::Agent do
|
|||
expect(converge_result).to be_a_successful_converge
|
||||
end
|
||||
|
||||
|
||||
describe "passing the agent to the pipeline" do
|
||||
let(:source_loader) { TestSourceLoader.new(pipeline_config) }
|
||||
let(:pipeline_config) { mock_pipeline_config(:main, "input { generator { count => 10 } } output { null {} }") }
|
||||
|
||||
before { subject.execute }
|
||||
|
||||
it "execute the pipeline and stop execution" do
|
||||
expect(subject.get_pipeline(:main).execution_context.agent).to eq(subject)
|
||||
end
|
||||
end
|
||||
|
||||
context "Agent execute options" do
|
||||
let(:source_loader) do
|
||||
TestSourceLoader.new(finite_pipeline_config)
|
||||
|
|
|
@ -1,28 +1,38 @@
|
|||
# encoding: utf-8
|
||||
require "spec_helper"
|
||||
require "logstash/util/dead_letter_queue_manager"
|
||||
require "logstash/execution_context"
|
||||
|
||||
describe LogStash::ExecutionContext do
|
||||
let(:pipeline) { double("pipeline") }
|
||||
let(:pipeline_id) { :main }
|
||||
let(:agent) { double("agent") }
|
||||
|
||||
let(:plugin_id) { "plugin_id" }
|
||||
let(:plugin_type) { "plugin_type" }
|
||||
let(:dlq_writer) { LogStash::Util::DummyDeadLetterQueueWriter.new }
|
||||
|
||||
before do
|
||||
allow(pipeline).to receive(:agent).and_return(agent)
|
||||
allow(pipeline).to receive(:pipeline_id).and_return(pipeline_id)
|
||||
end
|
||||
|
||||
subject { described_class.new(pipeline, agent) }
|
||||
subject { described_class.new(pipeline, agent, plugin_id, plugin_type, dlq_writer) }
|
||||
|
||||
it "returns the `pipeline_id`" do
|
||||
expect(subject.pipeline_id).to eq(pipeline_id)
|
||||
end
|
||||
|
||||
|
||||
it "returns the pipeline" do
|
||||
expect(subject.pipeline).to eq(pipeline)
|
||||
end
|
||||
|
||||
|
||||
it "returns the agent" do
|
||||
expect(subject.agent).to eq(agent)
|
||||
end
|
||||
|
||||
it "returns the plugin-specific dlq writer" do
|
||||
expect(subject.dlq_writer.plugin_type).to eq(plugin_type)
|
||||
expect(subject.dlq_writer.plugin_id).to eq(plugin_id)
|
||||
expect(subject.dlq_writer.inner_writer).to eq(dlq_writer)
|
||||
end
|
||||
end
|
||||
|
|
|
@ -66,7 +66,7 @@ describe "LogStash::Inputs::Base#decorate" do
|
|||
|
||||
context "execution context" do
|
||||
include_context "execution_context"
|
||||
|
||||
|
||||
let(:klass) { LogStash::Inputs::NOOP }
|
||||
|
||||
subject(:instance) { klass.new({}) }
|
||||
|
|
|
@ -10,7 +10,7 @@ describe LogStash::OutputDelegator do
|
|||
let(:plugin_args) { {"id" => "foo", "arg1" => "val1"} }
|
||||
let(:collector) { [] }
|
||||
let(:metric) { LogStash::Instrument::NamespacedNullMetric.new(collector, :null) }
|
||||
|
||||
|
||||
include_context "execution_context"
|
||||
|
||||
subject { described_class.new(logger, out_klass, metric, execution_context, ::LogStash::OutputDelegatorStrategyRegistry.instance, plugin_args) }
|
||||
|
|
|
@ -82,7 +82,7 @@ describe "LogStash::Outputs::Base#new" do
|
|||
|
||||
context "execution context" do
|
||||
include_context "execution_context"
|
||||
|
||||
|
||||
let(:klass) { LogStash::Outputs::NOOPSingle }
|
||||
|
||||
subject(:instance) { klass.new(params.dup) }
|
||||
|
|
107
logstash-core/spec/logstash/pipeline_dlq_commit_spec.rb
Normal file
107
logstash-core/spec/logstash/pipeline_dlq_commit_spec.rb
Normal file
|
@ -0,0 +1,107 @@
|
|||
# encoding: utf-8
|
||||
require "tmpdir"
|
||||
require "spec_helper"
|
||||
require "logstash/codecs/plain"
|
||||
require_relative "../support/mocks_classes"
|
||||
|
||||
class SingleGeneratorInput < LogStash::Inputs::Base
|
||||
config_name "singlegenerator"
|
||||
milestone 2
|
||||
|
||||
def register
|
||||
end
|
||||
|
||||
def run(queue)
|
||||
queue << LogStash::Event.new
|
||||
end
|
||||
|
||||
def close
|
||||
end
|
||||
end
|
||||
|
||||
class DLQCommittingFilter < LogStash::Filters::Base
|
||||
config_name "dlq_commit"
|
||||
milestone 2
|
||||
|
||||
def register()
|
||||
end
|
||||
|
||||
def filter(event)
|
||||
execution_context.dlq_writer.write(event, "my reason")
|
||||
end
|
||||
|
||||
def threadsafe?() true; end
|
||||
|
||||
def close() end
|
||||
end
|
||||
|
||||
describe LogStash::Pipeline do
|
||||
let(:pipeline_settings_obj) { LogStash::SETTINGS }
|
||||
let(:pipeline_settings) do
|
||||
{
|
||||
"pipeline.workers" => 2,
|
||||
"pipeline.id" => pipeline_id,
|
||||
"dead_letter_queue.enable" => enable_dlq,
|
||||
"path.dead_letter_queue" => Dir.mktmpdir
|
||||
}
|
||||
end
|
||||
let(:metric) { LogStash::Instrument::Metric.new(LogStash::Instrument::Collector.new) }
|
||||
let(:test_config) {
|
||||
<<-eos
|
||||
input { singlegenerator { id => input_id } }
|
||||
|
||||
filter { dlq_commit { id => filter_id } }
|
||||
|
||||
output { dummyoutput { id => output_id } }
|
||||
eos
|
||||
}
|
||||
|
||||
subject { LogStash::Pipeline.new(test_config, pipeline_settings_obj, metric) }
|
||||
|
||||
before(:each) do
|
||||
pipeline_settings.each {|k, v| pipeline_settings_obj.set(k, v) }
|
||||
allow(LogStash::Plugin).to receive(:lookup).with("input", "singlegenerator").and_return(SingleGeneratorInput)
|
||||
allow(LogStash::Plugin).to receive(:lookup).with("codec", "plain").and_return(LogStash::Codecs::Plain)
|
||||
allow(LogStash::Plugin).to receive(:lookup).with("filter", "dlq_commit").and_return(DLQCommittingFilter)
|
||||
allow(LogStash::Plugin).to receive(:lookup).with("output", "dummyoutput").and_return(::LogStash::Outputs::DummyOutput)
|
||||
end
|
||||
|
||||
after(:each) do
|
||||
FileUtils.remove_entry pipeline_settings["path.dead_letter_queue"]
|
||||
end
|
||||
|
||||
context "dlq is enabled" do
|
||||
let(:enable_dlq) { true }
|
||||
let(:pipeline_id) { "test-dlq" }
|
||||
|
||||
it "retrieves proper pipeline-level DLQ writer" do
|
||||
expect(LogStash::Util::PluginDeadLetterQueueWriter).to receive(:new).with(anything, "input_id", "singlegenerator").and_call_original
|
||||
expect(LogStash::Util::PluginDeadLetterQueueWriter).to receive(:new).with(anything, "filter_id", "dlq_commit").and_call_original
|
||||
expect(LogStash::Util::PluginDeadLetterQueueWriter).to receive(:new).with(anything, "output_id", "dummyoutput").and_call_original
|
||||
expect_any_instance_of(org.logstash.common.io.DeadLetterQueueWriter).to receive(:close).and_call_original
|
||||
subject.run
|
||||
dlq_path = java.nio.file.Paths.get(pipeline_settings_obj.get("path.dead_letter_queue"), pipeline_id)
|
||||
dlq_reader = org.logstash.common.io.DeadLetterQueueReader.new(dlq_path)
|
||||
entry = dlq_reader.pollEntry(40)
|
||||
expect(entry).to_not be_nil
|
||||
expect(entry.reason).to eq("my reason")
|
||||
end
|
||||
end
|
||||
|
||||
context "dlq is disabled" do
|
||||
let(:enable_dlq) { false }
|
||||
let(:pipeline_id) { "test-without-dlq" }
|
||||
|
||||
it "does not write to the DLQ" do
|
||||
expect(LogStash::Util::PluginDeadLetterQueueWriter).to receive(:new).with(anything, "input_id", "singlegenerator").and_call_original
|
||||
expect(LogStash::Util::PluginDeadLetterQueueWriter).to receive(:new).with(anything, "filter_id", "dlq_commit").and_call_original
|
||||
expect(LogStash::Util::PluginDeadLetterQueueWriter).to receive(:new).with(anything, "output_id", "dummyoutput").and_call_original
|
||||
expect(LogStash::Util::DummyDeadLetterQueueWriter).to receive(:new).and_call_original
|
||||
expect_any_instance_of(LogStash::Util::DummyDeadLetterQueueWriter).to receive(:close).and_call_original
|
||||
subject.run
|
||||
dlq_path = java.nio.file.Paths.get(pipeline_settings_obj.get("path.dead_letter_queue"), pipeline_id)
|
||||
expect(java.nio.file.Files.exists(dlq_path)).to eq(false)
|
||||
end
|
||||
end
|
||||
|
||||
end
|
|
@ -172,38 +172,56 @@ describe LogStash::Runner do
|
|||
allow(pipeline).to receive(:run).and_return(task)
|
||||
allow(pipeline).to receive(:shutdown)
|
||||
end
|
||||
|
||||
|
||||
context "when :path.data is defined by the user" do
|
||||
let(:test_data_path) { "/tmp/ls-test-data" }
|
||||
let(:test_queue_path) { test_data_path + "/" + "queue" }
|
||||
|
||||
let(:test_dlq_path) { test_data_path + "/" + "dead_letter_queue" }
|
||||
|
||||
it "should set data paths" do
|
||||
expect(LogStash::Agent).to receive(:new) do |settings|
|
||||
expect(settings.get("path.data")).to eq(test_data_path)
|
||||
expect(settings.get("path.queue")).to eq(test_queue_path)
|
||||
expect(settings.get("path.dead_letter_queue")).to eq(test_dlq_path)
|
||||
end
|
||||
|
||||
|
||||
args = ["--path.data", test_data_path, "-e", pipeline_string]
|
||||
subject.run("bin/logstash", args)
|
||||
end
|
||||
|
||||
|
||||
context "and path.queue is manually set" do
|
||||
let(:queue_override_path) { "/tmp/queue-override_path" }
|
||||
|
||||
|
||||
it "should set data paths" do
|
||||
expect(LogStash::Agent).to receive(:new) do |settings|
|
||||
expect(settings.get("path.data")).to eq(test_data_path)
|
||||
expect(settings.get("path.queue")).to eq(queue_override_path)
|
||||
end
|
||||
|
||||
|
||||
LogStash::SETTINGS.set("path.queue", queue_override_path)
|
||||
|
||||
|
||||
args = ["--path.data", test_data_path, "-e", pipeline_string]
|
||||
subject.run("bin/logstash", args)
|
||||
end
|
||||
end
|
||||
|
||||
context "and path.dead_letter_queue is manually set" do
|
||||
let(:queue_override_path) { "/tmp/queue-override_path" }
|
||||
|
||||
it "should set data paths" do
|
||||
expect(LogStash::Agent).to receive(:new) do |settings|
|
||||
expect(settings.get("path.data")).to eq(test_data_path)
|
||||
expect(settings.get("path.dead_letter_queue")).to eq(queue_override_path)
|
||||
end
|
||||
|
||||
LogStash::SETTINGS.set("path.dead_letter_queue", queue_override_path)
|
||||
|
||||
args = ["--path.data", test_data_path, "-e", pipeline_string]
|
||||
subject.run("bin/logstash", args)
|
||||
end
|
||||
end
|
||||
end
|
||||
|
||||
|
||||
context "when :http.host is defined by the user" do
|
||||
it "should pass the value to the webserver" do
|
||||
expect(LogStash::Agent).to receive(:new) do |settings|
|
||||
|
|
|
@ -2,10 +2,13 @@ shared_context "execution_context" do
|
|||
let(:pipeline) { double("pipeline") }
|
||||
let(:pipeline_id) { :main }
|
||||
let(:agent) { double("agent") }
|
||||
let(:plugin_id) { :plugin_id }
|
||||
let(:plugin_type) { :plugin_type }
|
||||
let(:dlq_writer) { double("dlq_writer") }
|
||||
let(:execution_context) do
|
||||
::LogStash::ExecutionContext.new(pipeline, agent)
|
||||
::LogStash::ExecutionContext.new(pipeline, agent, plugin_id, plugin_type, dlq_writer)
|
||||
end
|
||||
|
||||
|
||||
before do
|
||||
allow(pipeline).to receive(:pipeline_id).and_return(pipeline_id)
|
||||
allow(pipeline).to receive(:agent).and_return(agent)
|
||||
|
|
|
@ -0,0 +1,66 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
package org.logstash.common;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.logstash.common.io.DeadLetterQueueWriter;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.file.Paths;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
||||
/**
|
||||
* This class manages static collection of {@link DeadLetterQueueWriter} instances that
|
||||
* can be created and retrieved by a specific String-valued key.
|
||||
*/
|
||||
public class DeadLetterQueueFactory {
|
||||
|
||||
private static final Logger logger = LogManager.getLogger(DeadLetterQueueFactory.class);
|
||||
private static final ConcurrentHashMap<String, DeadLetterQueueWriter> REGISTRY = new ConcurrentHashMap<>();
|
||||
private static final long MAX_SEGMENT_SIZE_BYTES = 10 * 1024 * 1024;
|
||||
|
||||
/**
|
||||
* This class is only meant to be used statically, and therefore
|
||||
* the constructor is private.
|
||||
*/
|
||||
private DeadLetterQueueFactory() {
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieves an existing {@link DeadLetterQueueWriter} associated with {@param id}, or
|
||||
* opens a new one to be returned. It is the retrievers responsibility to close these newly
|
||||
* created writers.
|
||||
*
|
||||
* @param id The identifier context for this dlq manager
|
||||
* @param dlqPath The path to use for the queue's backing data directory. contains sub-directories
|
||||
* for each {@param id}
|
||||
* @return The write manager for the specific id's dead-letter-queue context
|
||||
*/
|
||||
public static DeadLetterQueueWriter getWriter(String id, String dlqPath) {
|
||||
return REGISTRY.computeIfAbsent(id, k -> {
|
||||
try {
|
||||
return new DeadLetterQueueWriter(Paths.get(dlqPath, k), MAX_SEGMENT_SIZE_BYTES, Long.MAX_VALUE);
|
||||
} catch (IOException e) {
|
||||
logger.error("unable to create dead letter queue writer", e);
|
||||
}
|
||||
return null;
|
||||
});
|
||||
}
|
||||
}
|
|
@ -37,17 +37,17 @@ import java.util.stream.Collectors;
|
|||
|
||||
import static java.nio.file.StandardWatchEventKinds.ENTRY_CREATE;
|
||||
import static java.nio.file.StandardWatchEventKinds.ENTRY_DELETE;
|
||||
import static org.logstash.common.io.DeadLetterQueueWriteManager.getSegmentPaths;
|
||||
import static org.logstash.common.io.DeadLetterQueueWriter.getSegmentPaths;
|
||||
|
||||
public class DeadLetterQueueReadManager {
|
||||
private static final Logger logger = LogManager.getLogger(DeadLetterQueueReadManager.class);
|
||||
public class DeadLetterQueueReader {
|
||||
private static final Logger logger = LogManager.getLogger(DeadLetterQueueReader.class);
|
||||
|
||||
private RecordIOReader currentReader;
|
||||
private final Path queuePath;
|
||||
private final ConcurrentSkipListSet<Path> segments;
|
||||
private final WatchService watchService;
|
||||
|
||||
public DeadLetterQueueReadManager(Path queuePath) throws Exception {
|
||||
public DeadLetterQueueReader(Path queuePath) throws Exception {
|
||||
this.queuePath = queuePath;
|
||||
this.watchService = FileSystems.getDefault().newWatchService();
|
||||
this.queuePath.register(watchService, ENTRY_CREATE, ENTRY_DELETE);
|
|
@ -21,6 +21,8 @@ package org.logstash.common.io;
|
|||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.logstash.DLQEntry;
|
||||
import org.logstash.Event;
|
||||
import org.logstash.Timestamp;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.channels.FileChannel;
|
||||
|
@ -28,14 +30,16 @@ import java.nio.channels.FileLock;
|
|||
import java.nio.channels.OverlappingFileLockException;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.nio.file.Paths;
|
||||
import java.nio.file.StandardOpenOption;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static org.logstash.common.io.RecordIOWriter.RECORD_HEADER_SIZE;
|
||||
|
||||
public class DeadLetterQueueWriteManager {
|
||||
public class DeadLetterQueueWriter {
|
||||
|
||||
private static final Logger logger = LogManager.getLogger(DeadLetterQueueWriteManager.class);
|
||||
private static final Logger logger = LogManager.getLogger(DeadLetterQueueWriter.class);
|
||||
private static final long MAX_SEGMENT_SIZE_BYTES = 10 * 1024 * 1024;
|
||||
|
||||
static final String SEGMENT_FILE_PATTERN = "%d.log";
|
||||
static final String LOCK_FILE = ".lock";
|
||||
|
@ -46,6 +50,8 @@ public class DeadLetterQueueWriteManager {
|
|||
private RecordIOWriter currentWriter;
|
||||
private long currentQueueSize;
|
||||
private int currentSegmentIndex;
|
||||
private Timestamp lastEntryTimestamp;
|
||||
private boolean open;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -53,7 +59,9 @@ public class DeadLetterQueueWriteManager {
|
|||
* @param maxSegmentSize
|
||||
* @throws IOException
|
||||
*/
|
||||
public DeadLetterQueueWriteManager(Path queuePath, long maxSegmentSize, long maxQueueSize) throws IOException {
|
||||
public DeadLetterQueueWriter(Path queuePath, long maxSegmentSize, long maxQueueSize) throws IOException {
|
||||
// ensure path exists, create it otherwise.
|
||||
Files.createDirectories(queuePath);
|
||||
// check that only one instance of the writer is open in this configured path
|
||||
Path lockFilePath = queuePath.resolve(LOCK_FILE);
|
||||
boolean isNewlyCreated = lockFilePath.toFile().createNewFile();
|
||||
|
@ -77,6 +85,18 @@ public class DeadLetterQueueWriteManager {
|
|||
.mapToInt(Integer::parseInt)
|
||||
.max().orElse(0);
|
||||
this.currentWriter = nextWriter();
|
||||
this.lastEntryTimestamp = Timestamp.now();
|
||||
this.open = true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructor for Writer that uses defaults
|
||||
*
|
||||
* @param queuePath the path to the dead letter queue segments directory
|
||||
* @throws IOException
|
||||
*/
|
||||
public DeadLetterQueueWriter(String queuePath) throws IOException {
|
||||
this(Paths.get(queuePath), MAX_SEGMENT_SIZE_BYTES, Long.MAX_VALUE);
|
||||
}
|
||||
|
||||
private long getStartupQueueSize() throws IOException {
|
||||
|
@ -99,11 +119,25 @@ public class DeadLetterQueueWriteManager {
|
|||
return Files.list(path).filter((p) -> p.toString().endsWith(".log"));
|
||||
}
|
||||
|
||||
public synchronized void writeEntry(DLQEntry event) throws IOException {
|
||||
byte[] record = event.serialize();
|
||||
public synchronized void writeEntry(DLQEntry entry) throws IOException {
|
||||
innerWriteEntry(entry);
|
||||
}
|
||||
|
||||
public synchronized void writeEntry(Event event, String pluginName, String pluginId, String reason) throws IOException {
|
||||
Timestamp entryTimestamp = Timestamp.now();
|
||||
if (entryTimestamp.getTime().isBefore(lastEntryTimestamp.getTime())) {
|
||||
entryTimestamp = lastEntryTimestamp;
|
||||
}
|
||||
DLQEntry entry = new DLQEntry(event, pluginName, pluginId, reason);
|
||||
innerWriteEntry(entry);
|
||||
lastEntryTimestamp = entryTimestamp;
|
||||
}
|
||||
|
||||
private void innerWriteEntry(DLQEntry entry) throws IOException {
|
||||
byte[] record = entry.serialize();
|
||||
int eventPayloadSize = RECORD_HEADER_SIZE + record.length;
|
||||
if (currentQueueSize + eventPayloadSize > maxQueueSize) {
|
||||
logger.error("cannot write event to DLQ, no space available");
|
||||
logger.error("cannot write event to DLQ: reached maxQueueSize of " + maxQueueSize);
|
||||
return;
|
||||
} else if (currentWriter.getPosition() + eventPayloadSize > maxSegmentSize) {
|
||||
currentWriter.close();
|
||||
|
@ -112,11 +146,17 @@ public class DeadLetterQueueWriteManager {
|
|||
currentQueueSize += currentWriter.writeEvent(record);
|
||||
}
|
||||
|
||||
|
||||
public synchronized void close() throws IOException {
|
||||
this.lock.release();
|
||||
if (currentWriter != null) {
|
||||
currentWriter.close();
|
||||
}
|
||||
Files.deleteIfExists(queuePath.resolve(LOCK_FILE));
|
||||
open = false;
|
||||
}
|
||||
|
||||
public boolean isOpen() {
|
||||
return open;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,54 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.logstash.common;
|
||||
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
import org.logstash.common.io.DeadLetterQueueWriter;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.file.Path;
|
||||
|
||||
import static junit.framework.TestCase.assertSame;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
public class DeadLetterQueueFactoryTest {
|
||||
private Path dir;
|
||||
|
||||
@Rule
|
||||
public TemporaryFolder temporaryFolder = new TemporaryFolder();
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
dir = temporaryFolder.newFolder().toPath();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test() throws IOException {
|
||||
Path pipelineA = dir.resolve("pipelineA");
|
||||
DeadLetterQueueWriter writer = DeadLetterQueueFactory.getWriter("pipelineA", pipelineA.toString());
|
||||
assertTrue(writer.isOpen());
|
||||
DeadLetterQueueWriter writer2 = DeadLetterQueueFactory.getWriter("pipelineA", pipelineA.toString());
|
||||
assertSame(writer, writer2);
|
||||
writer.close();
|
||||
}
|
||||
}
|
|
@ -29,9 +29,6 @@ import org.logstash.Event;
|
|||
import org.logstash.Timestamp;
|
||||
import org.logstash.ackedqueue.StringElement;
|
||||
|
||||
import java.io.BufferedWriter;
|
||||
import java.io.IOException;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.util.Collections;
|
||||
|
||||
|
@ -40,14 +37,14 @@ import static org.hamcrest.CoreMatchers.is;
|
|||
import static org.hamcrest.CoreMatchers.nullValue;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
|
||||
public class DeadLetterQueueReadManagerTest {
|
||||
public class DeadLetterQueueReaderTest {
|
||||
private Path dir;
|
||||
|
||||
@Rule
|
||||
public TemporaryFolder temporaryFolder = new TemporaryFolder();
|
||||
|
||||
private static String segmentFileName(int i) {
|
||||
return String.format(DeadLetterQueueWriteManager.SEGMENT_FILE_PATTERN, i);
|
||||
return String.format(DeadLetterQueueWriter.SEGMENT_FILE_PATTERN, i);
|
||||
}
|
||||
|
||||
@Before
|
||||
|
@ -70,7 +67,7 @@ public class DeadLetterQueueReadManagerTest {
|
|||
}
|
||||
}
|
||||
|
||||
DeadLetterQueueReadManager manager = new DeadLetterQueueReadManager(dir);
|
||||
DeadLetterQueueReader manager = new DeadLetterQueueReader(dir);
|
||||
|
||||
for (int i = 0; i < 50; i++) {
|
||||
String first = StringElement.deserialize(manager.pollEntryBytes()).toString();
|
||||
|
@ -116,7 +113,7 @@ public class DeadLetterQueueReadManagerTest {
|
|||
|
||||
@Test
|
||||
public void testSeek() throws Exception {
|
||||
DeadLetterQueueWriteManager writeManager = new DeadLetterQueueWriteManager(dir, 10000000, 10000000);
|
||||
DeadLetterQueueWriter writeManager = new DeadLetterQueueWriter(dir, 10000000, 10000000);
|
||||
Event event = new Event(Collections.emptyMap());
|
||||
Timestamp target = null;
|
||||
long currentEpoch = System.currentTimeMillis();
|
||||
|
@ -130,7 +127,7 @@ public class DeadLetterQueueReadManagerTest {
|
|||
}
|
||||
writeManager.close();
|
||||
|
||||
DeadLetterQueueReadManager readManager = new DeadLetterQueueReadManager(dir);
|
||||
DeadLetterQueueReader readManager = new DeadLetterQueueReader(dir);
|
||||
readManager.seekToNextEvent(target);
|
||||
DLQEntry entry = readManager.pollEntry(100);
|
||||
assertThat(entry.getEntryTime().toIso8601(), equalTo(target.toIso8601()));
|
||||
|
@ -139,7 +136,7 @@ public class DeadLetterQueueReadManagerTest {
|
|||
|
||||
@Test
|
||||
public void testInvalidDirectory() throws Exception {
|
||||
DeadLetterQueueReadManager readManager = new DeadLetterQueueReadManager(dir);
|
||||
assertThat(readManager.pollEntry(100), is(nullValue()));
|
||||
DeadLetterQueueReader reader = new DeadLetterQueueReader(dir);
|
||||
assertThat(reader.pollEntry(100), is(nullValue()));
|
||||
}
|
||||
}
|
||||
}
|
|
@ -36,7 +36,7 @@ import static junit.framework.TestCase.assertFalse;
|
|||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
public class DeadLetterQueueWriteManagerTest {
|
||||
public class DeadLetterQueueWriterTest {
|
||||
private Path dir;
|
||||
|
||||
@Rule
|
||||
|
@ -50,7 +50,7 @@ public class DeadLetterQueueWriteManagerTest {
|
|||
@Test
|
||||
public void testLockFileManagement() throws Exception {
|
||||
Path lockFile = dir.resolve(".lock");
|
||||
DeadLetterQueueWriteManager writer = new DeadLetterQueueWriteManager(dir, 1000, 1000000);
|
||||
DeadLetterQueueWriter writer = new DeadLetterQueueWriter(dir, 1000, 1000000);
|
||||
assertTrue(Files.exists(lockFile));
|
||||
writer.close();
|
||||
assertFalse(Files.exists(lockFile));
|
||||
|
@ -58,9 +58,9 @@ public class DeadLetterQueueWriteManagerTest {
|
|||
|
||||
@Test
|
||||
public void testFileLocking() throws Exception {
|
||||
DeadLetterQueueWriteManager writer = new DeadLetterQueueWriteManager(dir, 1000, 1000000);
|
||||
DeadLetterQueueWriter writer = new DeadLetterQueueWriter(dir, 1000, 1000000);
|
||||
try {
|
||||
new DeadLetterQueueWriteManager(dir, 1000, 100000);
|
||||
new DeadLetterQueueWriter(dir, 1000, 100000);
|
||||
fail();
|
||||
} catch (RuntimeException e) {
|
||||
} finally {
|
||||
|
@ -72,7 +72,7 @@ public class DeadLetterQueueWriteManagerTest {
|
|||
public void testUncleanCloseOfPreviousWriter() throws Exception {
|
||||
Path lockFilePath = dir.resolve(".lock");
|
||||
boolean created = lockFilePath.toFile().createNewFile();
|
||||
DeadLetterQueueWriteManager writer = new DeadLetterQueueWriteManager(dir, 1000, 1000000);
|
||||
DeadLetterQueueWriter writer = new DeadLetterQueueWriter(dir, 1000, 1000000);
|
||||
|
||||
FileChannel channel = FileChannel.open(lockFilePath, StandardOpenOption.WRITE);
|
||||
try {
|
||||
|
@ -87,7 +87,7 @@ public class DeadLetterQueueWriteManagerTest {
|
|||
|
||||
@Test
|
||||
public void testWrite() throws Exception {
|
||||
DeadLetterQueueWriteManager writer = new DeadLetterQueueWriteManager(dir, 1000, 1000000);
|
||||
DeadLetterQueueWriter writer = new DeadLetterQueueWriter(dir, 1000, 1000000);
|
||||
DLQEntry entry = new DLQEntry(new Event(), "type", "id", "reason");
|
||||
writer.writeEntry(entry);
|
||||
writer.close();
|
Loading…
Add table
Add a link
Reference in a new issue