mirror of
https://github.com/elastic/logstash.git
synced 2025-04-24 06:37:19 -04:00
JAVAFICATION: Further dry up ruby and java pipeline and move more of their logic to Java
Fixes #9687
This commit is contained in:
parent
0a72df874d
commit
060a8e3bd2
7 changed files with 333 additions and 275 deletions
|
@ -13,21 +13,20 @@ java_import org.logstash.common.io.DeadLetterQueueWriter
|
|||
java_import org.logstash.config.ir.CompiledPipeline
|
||||
java_import org.logstash.config.ir.ConfigCompiler
|
||||
|
||||
module LogStash; class JavaBasePipeline < LogstashPipeline
|
||||
module LogStash; class JavaBasePipeline < AbstractPipeline
|
||||
include LogStash::Util::Loggable
|
||||
|
||||
attr_reader :inputs, :filters, :outputs
|
||||
|
||||
def initialize(pipeline_config, namespaced_metric = nil, agent = nil)
|
||||
super pipeline_config, namespaced_metric
|
||||
@logger = self.logger
|
||||
@dlq_writer = dlq_writer
|
||||
super pipeline_config, namespaced_metric, @logger, @queue
|
||||
@lir_execution = CompiledPipeline.new(
|
||||
lir,
|
||||
LogStash::Plugins::PluginFactory.new(
|
||||
# use NullMetric if called in the BasePipeline context otherwise use the @metric value
|
||||
lir, LogStash::Plugins::PluginMetricFactory.new(pipeline_id, metric),
|
||||
LogStash::Plugins::ExecutionContextFactory.new(agent, self, @dlq_writer),
|
||||
LogStash::Plugins::ExecutionContextFactory.new(agent, self, dlq_writer),
|
||||
JavaFilterDelegator
|
||||
)
|
||||
)
|
||||
|
@ -43,10 +42,6 @@ module LogStash; class JavaBasePipeline < LogstashPipeline
|
|||
configured_as_reloadable? && reloadable_plugins?
|
||||
end
|
||||
|
||||
def configured_as_reloadable?
|
||||
settings.get("pipeline.reloadable")
|
||||
end
|
||||
|
||||
def reloadable_plugins?
|
||||
non_reloadable_plugins.empty?
|
||||
end
|
||||
|
@ -67,26 +62,22 @@ module LogStash; class JavaPipeline < JavaBasePipeline
|
|||
:worker_threads,
|
||||
:events_consumed,
|
||||
:events_filtered,
|
||||
:reporter,
|
||||
:started_at,
|
||||
:thread,
|
||||
:filter_queue_client,
|
||||
:input_queue_client,
|
||||
:queue
|
||||
:input_queue_client
|
||||
|
||||
MAX_INFLIGHT_WARN_THRESHOLD = 10_000
|
||||
|
||||
def initialize(pipeline_config, namespaced_metric = nil, agent = nil)
|
||||
super
|
||||
@reporter = PipelineReporter.new(@logger, self)
|
||||
@worker_threads = []
|
||||
|
||||
begin
|
||||
@queue = LogStash::QueueFactory.create(settings)
|
||||
@queue = LogStash::QueueFactory.create(pipeline_config.settings)
|
||||
rescue => e
|
||||
@logger.error("Logstash failed to create queue", default_logging_keys("exception" => e.message, "backtrace" => e.backtrace))
|
||||
raise e
|
||||
end
|
||||
super
|
||||
@worker_threads = []
|
||||
|
||||
@input_queue_client = @queue.write_client
|
||||
@filter_queue_client = @queue.read_client
|
||||
|
@ -244,10 +235,6 @@ module LogStash; class JavaPipeline < JavaBasePipeline
|
|||
@running.false?
|
||||
end
|
||||
|
||||
def system?
|
||||
settings.get_value("pipeline.system")
|
||||
end
|
||||
|
||||
# register_plugins calls #register_plugin on the plugins list and upon exception will call Plugin#do_close on all registered plugins
|
||||
# @param plugins [Array[Plugin]] the list of plugins to register
|
||||
def register_plugins(plugins)
|
||||
|
@ -280,7 +267,7 @@ module LogStash; class JavaPipeline < JavaBasePipeline
|
|||
config_metric.gauge(:config_reload_automatic, settings.get("config.reload.automatic"))
|
||||
config_metric.gauge(:config_reload_interval, settings.get("config.reload.interval"))
|
||||
config_metric.gauge(:dead_letter_queue_enabled, dlq_enabled?)
|
||||
config_metric.gauge(:dead_letter_queue_path, @dlq_writer.get_path.to_absolute_path.to_s) if dlq_enabled?
|
||||
config_metric.gauge(:dead_letter_queue_path, dlq_writer.get_path.to_absolute_path.to_s) if dlq_enabled?
|
||||
|
||||
|
||||
@logger.info("Starting pipeline", default_logging_keys(
|
||||
|
@ -466,37 +453,6 @@ module LogStash; class JavaPipeline < JavaBasePipeline
|
|||
.each {|t| t.delete("status") }
|
||||
end
|
||||
|
||||
def collect_dlq_stats
|
||||
if dlq_enabled?
|
||||
dlq_metric = metric.namespace([:stats, :pipelines, pipeline_id.to_s.to_sym, :dlq])
|
||||
dlq_metric.gauge(:queue_size_in_bytes, @dlq_writer.get_current_queue_size)
|
||||
end
|
||||
end
|
||||
|
||||
def collect_stats
|
||||
pipeline_metric = metric.namespace([:stats, :pipelines, pipeline_id.to_s.to_sym, :queue])
|
||||
pipeline_metric.gauge(:type, settings.get("queue.type"))
|
||||
if @queue.is_a?(LogStash::WrappedAckedQueue) && @queue.queue.is_a?(LogStash::AckedQueue)
|
||||
queue = @queue.queue
|
||||
dir_path = queue.dir_path
|
||||
file_store = Files.get_file_store(Paths.get(dir_path))
|
||||
|
||||
pipeline_metric.namespace([:capacity]).tap do |n|
|
||||
n.gauge(:page_capacity_in_bytes, queue.page_capacity)
|
||||
n.gauge(:max_queue_size_in_bytes, queue.max_size_in_bytes)
|
||||
n.gauge(:max_unread_events, queue.max_unread_events)
|
||||
n.gauge(:queue_size_in_bytes, queue.persisted_size_in_bytes)
|
||||
end
|
||||
pipeline_metric.namespace([:data]).tap do |n|
|
||||
n.gauge(:free_space_in_bytes, file_store.get_unallocated_space)
|
||||
n.gauge(:storage_type, file_store.type)
|
||||
n.gauge(:path, dir_path)
|
||||
end
|
||||
|
||||
pipeline_metric.gauge(:events, queue.unread_count)
|
||||
end
|
||||
end
|
||||
|
||||
def clear_pipeline_metrics
|
||||
# TODO(ph): I think the metric should also proxy that call correctly to the collector
|
||||
# this will simplify everything since the null metric would simply just do a noop
|
||||
|
|
|
@ -17,14 +17,14 @@ java_import org.logstash.common.SourceWithMetadata
|
|||
java_import org.logstash.common.io.DeadLetterQueueWriter
|
||||
java_import org.logstash.config.ir.ConfigCompiler
|
||||
|
||||
module LogStash; class BasePipeline < LogstashPipeline
|
||||
module LogStash; class BasePipeline < AbstractPipeline
|
||||
include LogStash::Util::Loggable
|
||||
|
||||
attr_reader :inputs, :filters, :outputs
|
||||
|
||||
def initialize(pipeline_config, namespaced_metric = nil, agent = nil)
|
||||
super pipeline_config, namespaced_metric
|
||||
@logger = self.logger
|
||||
super pipeline_config, namespaced_metric, @logger, @queue
|
||||
@mutex = Mutex.new
|
||||
|
||||
@inputs = nil
|
||||
|
@ -32,12 +32,10 @@ module LogStash; class BasePipeline < LogstashPipeline
|
|||
@outputs = nil
|
||||
@agent = agent
|
||||
|
||||
@dlq_writer = dlq_writer
|
||||
|
||||
@plugin_factory = LogStash::Plugins::PluginFactory.new(
|
||||
# use NullMetric if called in the BasePipeline context otherwise use the @metric value
|
||||
lir, LogStash::Plugins::PluginMetricFactory.new(pipeline_id, metric),
|
||||
LogStash::Plugins::ExecutionContextFactory.new(@agent, self, @dlq_writer),
|
||||
LogStash::Plugins::ExecutionContextFactory.new(@agent, self, dlq_writer),
|
||||
FilterDelegator
|
||||
)
|
||||
grammar = LogStashConfigParser.new
|
||||
|
@ -64,10 +62,6 @@ module LogStash; class BasePipeline < LogstashPipeline
|
|||
configured_as_reloadable? && reloadable_plugins?
|
||||
end
|
||||
|
||||
def configured_as_reloadable?
|
||||
settings.get("pipeline.reloadable")
|
||||
end
|
||||
|
||||
def reloadable_plugins?
|
||||
non_reloadable_plugins.empty?
|
||||
end
|
||||
|
@ -93,27 +87,23 @@ module LogStash; class Pipeline < BasePipeline
|
|||
:worker_threads,
|
||||
:events_consumed,
|
||||
:events_filtered,
|
||||
:reporter,
|
||||
:started_at,
|
||||
:thread,
|
||||
:filter_queue_client,
|
||||
:input_queue_client,
|
||||
:queue
|
||||
:input_queue_client
|
||||
|
||||
MAX_INFLIGHT_WARN_THRESHOLD = 10_000
|
||||
|
||||
def initialize(pipeline_config, namespaced_metric = nil, agent = nil)
|
||||
super
|
||||
|
||||
@reporter = PipelineReporter.new(@logger, self)
|
||||
@worker_threads = []
|
||||
|
||||
begin
|
||||
@queue = LogStash::QueueFactory.create(settings)
|
||||
@queue = LogStash::QueueFactory.create(pipeline_config.settings)
|
||||
rescue => e
|
||||
@logger.error("Logstash failed to create queue", default_logging_keys("exception" => e.message, "backtrace" => e.backtrace))
|
||||
raise e
|
||||
end
|
||||
super
|
||||
|
||||
@worker_threads = []
|
||||
|
||||
@input_queue_client = @queue.write_client
|
||||
@filter_queue_client = @queue.read_client
|
||||
|
@ -271,10 +261,6 @@ module LogStash; class Pipeline < BasePipeline
|
|||
@running.false?
|
||||
end
|
||||
|
||||
def system?
|
||||
settings.get_value("pipeline.system")
|
||||
end
|
||||
|
||||
# register_plugin simply calls the plugin #register method and catches & logs any error
|
||||
# @param plugin [Plugin] the plugin to register
|
||||
# @return [Plugin] the registered plugin
|
||||
|
@ -315,7 +301,7 @@ module LogStash; class Pipeline < BasePipeline
|
|||
config_metric.gauge(:config_reload_automatic, settings.get("config.reload.automatic"))
|
||||
config_metric.gauge(:config_reload_interval, settings.get("config.reload.interval"))
|
||||
config_metric.gauge(:dead_letter_queue_enabled, dlq_enabled?)
|
||||
config_metric.gauge(:dead_letter_queue_path, @dlq_writer.get_path.to_absolute_path.to_s) if dlq_enabled?
|
||||
config_metric.gauge(:dead_letter_queue_path, dlq_writer.get_path.to_absolute_path.to_s) if dlq_enabled?
|
||||
|
||||
if max_inflight > MAX_INFLIGHT_WARN_THRESHOLD
|
||||
@logger.warn("CAUTION: Recommended inflight events max exceeded! Logstash will run with up to #{max_inflight} events in memory in your current configuration. If your message sizes are large this may cause instability with the default heap size. Please consider setting a non-standard heap size, changing the batch size (currently #{batch_size}), or changing the number of pipeline workers (currently #{pipeline_workers})", default_logging_keys)
|
||||
|
@ -601,37 +587,6 @@ module LogStash; class Pipeline < BasePipeline
|
|||
.each {|t| t.delete("status") }
|
||||
end
|
||||
|
||||
def collect_dlq_stats
|
||||
if dlq_enabled?
|
||||
dlq_metric = metric.namespace([:stats, :pipelines, pipeline_id.to_s.to_sym, :dlq])
|
||||
dlq_metric.gauge(:queue_size_in_bytes, @dlq_writer.get_current_queue_size)
|
||||
end
|
||||
end
|
||||
|
||||
def collect_stats
|
||||
pipeline_metric = metric.namespace([:stats, :pipelines, pipeline_id.to_s.to_sym, :queue])
|
||||
pipeline_metric.gauge(:type, settings.get("queue.type"))
|
||||
if @queue.is_a?(LogStash::WrappedAckedQueue) && @queue.queue.is_a?(LogStash::AckedQueue)
|
||||
queue = @queue.queue
|
||||
dir_path = queue.dir_path
|
||||
file_store = Files.get_file_store(Paths.get(dir_path))
|
||||
|
||||
pipeline_metric.namespace([:capacity]).tap do |n|
|
||||
n.gauge(:page_capacity_in_bytes, queue.page_capacity)
|
||||
n.gauge(:max_queue_size_in_bytes, queue.max_size_in_bytes)
|
||||
n.gauge(:max_unread_events, queue.max_unread_events)
|
||||
n.gauge(:queue_size_in_bytes, queue.persisted_size_in_bytes)
|
||||
end
|
||||
pipeline_metric.namespace([:data]).tap do |n|
|
||||
n.gauge(:free_space_in_bytes, file_store.get_unallocated_space)
|
||||
n.gauge(:storage_type, file_store.type)
|
||||
n.gauge(:path, dir_path)
|
||||
end
|
||||
|
||||
pipeline_metric.gauge(:events, queue.unread_count)
|
||||
end
|
||||
end
|
||||
|
||||
def clear_pipeline_metrics
|
||||
# TODO(ph): I think the metric should also proxy that call correctly to the collector
|
||||
# this will simplify everything since the null metric would simply just do a noop
|
||||
|
|
|
@ -19,7 +19,7 @@ import org.logstash.config.ir.compiler.OutputStrategyExt;
|
|||
import org.logstash.execution.AbstractWrappedQueueExt;
|
||||
import org.logstash.execution.EventDispatcherExt;
|
||||
import org.logstash.execution.ExecutionContextExt;
|
||||
import org.logstash.execution.LogstashPipelineExt;
|
||||
import org.logstash.execution.AbstractPipelineExt;
|
||||
import org.logstash.execution.PipelineReporterExt;
|
||||
import org.logstash.execution.QueueReadClientBase;
|
||||
import org.logstash.execution.ShutdownWatcherExt;
|
||||
|
@ -390,7 +390,7 @@ public final class RubyUtil {
|
|||
LOGGABLE_MODULE = UTIL_MODULE.defineModuleUnder("Loggable");
|
||||
LOGGABLE_MODULE.defineAnnotatedMethods(LoggableExt.class);
|
||||
LOGSTASH_PIPELINE_CLASS =
|
||||
setupLogstashClass(LogstashPipelineExt::new, LogstashPipelineExt.class);
|
||||
setupLogstashClass(AbstractPipelineExt::new, AbstractPipelineExt.class);
|
||||
final RubyModule json = LOGSTASH_MODULE.defineOrGetModuleUnder("Json");
|
||||
final RubyClass stdErr = RUBY.getStandardError();
|
||||
LOGSTASH_ERROR = LOGSTASH_MODULE.defineClassUnder(
|
||||
|
|
|
@ -6,6 +6,7 @@ import org.jruby.RubyBoolean;
|
|||
import org.jruby.RubyClass;
|
||||
import org.jruby.RubyFixnum;
|
||||
import org.jruby.RubyObject;
|
||||
import org.jruby.RubyString;
|
||||
import org.jruby.anno.JRubyClass;
|
||||
import org.jruby.anno.JRubyMethod;
|
||||
import org.jruby.javasupport.JavaObject;
|
||||
|
@ -68,7 +69,7 @@ public final class JRubyAckedQueueExt extends RubyObject {
|
|||
}
|
||||
|
||||
@JRubyMethod(name = "dir_path")
|
||||
public IRubyObject ruby_dir_path(ThreadContext context) {
|
||||
public RubyString ruby_dir_path(ThreadContext context) {
|
||||
return context.runtime.newString(queue.getDirPath());
|
||||
}
|
||||
|
||||
|
|
|
@ -44,7 +44,7 @@ public final class JRubyWrappedAckedQueueExt extends AbstractWrappedQueueExt {
|
|||
}
|
||||
|
||||
@JRubyMethod(name = "queue")
|
||||
public IRubyObject rubyGetQueue(ThreadContext context) {
|
||||
public JRubyAckedQueueExt rubyGetQueue() {
|
||||
return queue;
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,311 @@
|
|||
package org.logstash.execution;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.file.FileStore;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Paths;
|
||||
import java.security.MessageDigest;
|
||||
import java.security.NoSuchAlgorithmException;
|
||||
import java.util.Arrays;
|
||||
import java.util.UUID;
|
||||
import org.apache.commons.codec.binary.Hex;
|
||||
import org.jruby.Ruby;
|
||||
import org.jruby.RubyArray;
|
||||
import org.jruby.RubyBasicObject;
|
||||
import org.jruby.RubyClass;
|
||||
import org.jruby.RubyString;
|
||||
import org.jruby.RubySymbol;
|
||||
import org.jruby.anno.JRubyClass;
|
||||
import org.jruby.anno.JRubyMethod;
|
||||
import org.jruby.javasupport.JavaUtil;
|
||||
import org.jruby.runtime.ThreadContext;
|
||||
import org.jruby.runtime.builtin.IRubyObject;
|
||||
import org.logstash.RubyUtil;
|
||||
import org.logstash.ackedqueue.ext.JRubyAckedQueueExt;
|
||||
import org.logstash.ackedqueue.ext.JRubyWrappedAckedQueueExt;
|
||||
import org.logstash.common.DeadLetterQueueFactory;
|
||||
import org.logstash.common.IncompleteSourceWithMetadataException;
|
||||
import org.logstash.config.ir.ConfigCompiler;
|
||||
import org.logstash.config.ir.PipelineIR;
|
||||
import org.logstash.instrument.metrics.AbstractMetricExt;
|
||||
import org.logstash.instrument.metrics.AbstractNamespacedMetricExt;
|
||||
import org.logstash.instrument.metrics.NullMetricExt;
|
||||
|
||||
@JRubyClass(name = "AbstractPipeline")
|
||||
public final class AbstractPipelineExt extends RubyBasicObject {
|
||||
|
||||
private static final RubyArray CAPACITY_NAMESPACE =
|
||||
RubyArray.newArray(RubyUtil.RUBY, RubyUtil.RUBY.newSymbol("capacity"));
|
||||
|
||||
private static final RubyArray DATA_NAMESPACE =
|
||||
RubyArray.newArray(RubyUtil.RUBY, RubyUtil.RUBY.newSymbol("data"));
|
||||
|
||||
private static final RubySymbol PAGE_CAPACITY_IN_BYTES =
|
||||
RubyUtil.RUBY.newSymbol("page_capacity_in_bytes");
|
||||
|
||||
private static final RubySymbol MAX_QUEUE_SIZE_IN_BYTES =
|
||||
RubyUtil.RUBY.newSymbol("max_queue_size_in_bytes");
|
||||
|
||||
private static final RubySymbol MAX_QUEUE_UNREAD_EVENTS =
|
||||
RubyUtil.RUBY.newSymbol("max_unread_events");
|
||||
|
||||
private static final RubySymbol QUEUE_SIZE_IN_BYTES =
|
||||
RubyUtil.RUBY.newSymbol("queue_size_in_bytes");
|
||||
|
||||
private static final RubySymbol FREE_SPACE_IN_BYTES =
|
||||
RubyUtil.RUBY.newSymbol("free_space_in_bytes");
|
||||
|
||||
private static final RubySymbol STORAGE_TYPE = RubyUtil.RUBY.newSymbol("storage_type");
|
||||
|
||||
private static final RubySymbol PATH = RubyUtil.RUBY.newSymbol("path");
|
||||
|
||||
private static final RubySymbol STATS_KEY = RubyUtil.RUBY.newSymbol("stats");
|
||||
|
||||
private static final RubySymbol PIPELINES_KEY = RubyUtil.RUBY.newSymbol("pipelines");
|
||||
|
||||
private static final RubySymbol EVENTS_KEY = RubyUtil.RUBY.newSymbol("events");
|
||||
|
||||
private static final RubySymbol TYPE_KEY = RubyUtil.RUBY.newSymbol("type");
|
||||
|
||||
private static final RubySymbol QUEUE_KEY = RubyUtil.RUBY.newSymbol("queue");
|
||||
|
||||
private static final RubySymbol DLQ_KEY = RubyUtil.RUBY.newSymbol("dlq");
|
||||
|
||||
private static final RubySymbol DLQ_SIZE_KEY =
|
||||
RubyUtil.RUBY.newSymbol("queue_size_in_bytes");
|
||||
|
||||
private final RubyString ephemeralId = RubyUtil.RUBY.newString(UUID.randomUUID().toString());
|
||||
|
||||
private AbstractNamespacedMetricExt dlqMetric;
|
||||
|
||||
private RubyString configString;
|
||||
|
||||
private RubyString configHash;
|
||||
|
||||
private IRubyObject settings;
|
||||
|
||||
private IRubyObject pipelineSettings;
|
||||
|
||||
private IRubyObject pipelineId;
|
||||
|
||||
private AbstractMetricExt metric;
|
||||
|
||||
private PipelineIR lir;
|
||||
|
||||
private IRubyObject dlqWriter;
|
||||
|
||||
private PipelineReporterExt reporter;
|
||||
|
||||
private IRubyObject queue;
|
||||
|
||||
public AbstractPipelineExt(final Ruby runtime, final RubyClass metaClass) {
|
||||
super(runtime, metaClass);
|
||||
}
|
||||
|
||||
@JRubyMethod(required = 4)
|
||||
public AbstractPipelineExt initialize(final ThreadContext context, final IRubyObject[] args)
|
||||
throws NoSuchAlgorithmException, IncompleteSourceWithMetadataException {
|
||||
final IRubyObject namespacedMetric = args[1];
|
||||
queue = args[3];
|
||||
reporter = new PipelineReporterExt(
|
||||
context.runtime, RubyUtil.PIPELINE_REPORTER_CLASS).initialize(context, args[2], this
|
||||
);
|
||||
pipelineSettings = args[0];
|
||||
configString = (RubyString) pipelineSettings.callMethod(context, "config_string");
|
||||
configHash = context.runtime.newString(
|
||||
Hex.encodeHexString(
|
||||
MessageDigest.getInstance("SHA1").digest(configString.getBytes())
|
||||
)
|
||||
);
|
||||
settings = pipelineSettings.callMethod(context, "settings");
|
||||
final IRubyObject id = getSetting(context, "pipeline.id");
|
||||
if (id.isNil()) {
|
||||
pipelineId = id();
|
||||
} else {
|
||||
pipelineId = id;
|
||||
}
|
||||
if (namespacedMetric.isNil()) {
|
||||
metric = new NullMetricExt(context.runtime, RubyUtil.NULL_METRIC_CLASS).initialize(
|
||||
context, new IRubyObject[0]
|
||||
);
|
||||
} else {
|
||||
final AbstractMetricExt java = (AbstractMetricExt) namespacedMetric;
|
||||
if (getSetting(context, "metric.collect").isTrue()) {
|
||||
metric = java;
|
||||
} else {
|
||||
metric = new NullMetricExt(context.runtime, RubyUtil.NULL_METRIC_CLASS).initialize(
|
||||
context, new IRubyObject[]{java.collector(context)}
|
||||
);
|
||||
}
|
||||
}
|
||||
lir = ConfigCompiler.configToPipelineIR(
|
||||
configString.asJavaString(),
|
||||
getSetting(context, "config.support_escapes").isTrue()
|
||||
);
|
||||
return this;
|
||||
}
|
||||
|
||||
@JRubyMethod(name = "config_str")
|
||||
public RubyString configStr() {
|
||||
return configString;
|
||||
}
|
||||
|
||||
@JRubyMethod(name = "config_hash")
|
||||
public RubyString configHash() {
|
||||
return configHash;
|
||||
}
|
||||
|
||||
@JRubyMethod(name = "ephemeral_id")
|
||||
public RubyString ephemeralId() {
|
||||
return ephemeralId;
|
||||
}
|
||||
|
||||
@JRubyMethod
|
||||
public IRubyObject settings() {
|
||||
return settings;
|
||||
}
|
||||
|
||||
@JRubyMethod(name = "pipeline_config")
|
||||
public IRubyObject pipelineConfig() {
|
||||
return pipelineSettings;
|
||||
}
|
||||
|
||||
@JRubyMethod(name = "pipeline_id")
|
||||
public IRubyObject pipelineId() {
|
||||
return pipelineId;
|
||||
}
|
||||
|
||||
@JRubyMethod
|
||||
public AbstractMetricExt metric() {
|
||||
return metric;
|
||||
}
|
||||
|
||||
@JRubyMethod
|
||||
public IRubyObject lir(final ThreadContext context) {
|
||||
return JavaUtil.convertJavaToUsableRubyObject(context.runtime, lir);
|
||||
}
|
||||
|
||||
@JRubyMethod(name = "dlq_writer")
|
||||
public IRubyObject dlqWriter(final ThreadContext context) {
|
||||
if (dlqWriter == null) {
|
||||
if (dlqEnabled(context).isTrue()) {
|
||||
dlqWriter = JavaUtil.convertJavaToUsableRubyObject(
|
||||
context.runtime,
|
||||
DeadLetterQueueFactory.getWriter(
|
||||
pipelineId.asJavaString(),
|
||||
getSetting(context, "path.dead_letter_queue").asJavaString(),
|
||||
getSetting(context, "dead_letter_queue.max_bytes").convertToInteger()
|
||||
.getLongValue()
|
||||
)
|
||||
);
|
||||
} else {
|
||||
dlqWriter = RubyUtil.DUMMY_DLQ_WRITER_CLASS.callMethod(context, "new");
|
||||
}
|
||||
}
|
||||
return dlqWriter;
|
||||
}
|
||||
|
||||
@JRubyMethod(name = "dlq_enabled?")
|
||||
public IRubyObject dlqEnabled(final ThreadContext context) {
|
||||
return getSetting(context, "dead_letter_queue.enable");
|
||||
}
|
||||
|
||||
@JRubyMethod(name = "close_dlq_writer")
|
||||
public IRubyObject closeDlqWriter(final ThreadContext context) {
|
||||
dlqWriter.callMethod(context, "close");
|
||||
if (dlqEnabled(context).isTrue()) {
|
||||
DeadLetterQueueFactory.release(pipelineId.asJavaString());
|
||||
}
|
||||
return context.nil;
|
||||
}
|
||||
|
||||
@JRubyMethod
|
||||
public PipelineReporterExt reporter() {
|
||||
return reporter;
|
||||
}
|
||||
|
||||
@JRubyMethod(name = "collect_dlq_stats")
|
||||
public IRubyObject collectDlqStats(final ThreadContext context) {
|
||||
if (dlqEnabled(context).isTrue()) {
|
||||
getDlqMetric(context).gauge(
|
||||
context, DLQ_SIZE_KEY,
|
||||
dlqWriter(context).callMethod(context, "get_current_queue_size")
|
||||
);
|
||||
}
|
||||
return context.nil;
|
||||
}
|
||||
|
||||
@JRubyMethod(name = "system?")
|
||||
public IRubyObject isSystem(final ThreadContext context) {
|
||||
return getSetting(context, "pipeline.system");
|
||||
}
|
||||
|
||||
@JRubyMethod(name = "configured_as_reloadable?")
|
||||
public IRubyObject isConfiguredReloadable(final ThreadContext context) {
|
||||
return getSetting(context, "pipeline.reloadable");
|
||||
}
|
||||
|
||||
@JRubyMethod(name = "collect_stats")
|
||||
public IRubyObject collectStats(final ThreadContext context) throws IOException {
|
||||
final AbstractNamespacedMetricExt pipelineMetric = metric.namespace(
|
||||
context,
|
||||
RubyArray.newArray(
|
||||
context.runtime,
|
||||
Arrays.asList(STATS_KEY, PIPELINES_KEY, pipelineId.asString().intern(), QUEUE_KEY)
|
||||
)
|
||||
);
|
||||
pipelineMetric.gauge(context, TYPE_KEY, getSetting(context, "queue.type"));
|
||||
if (queue instanceof JRubyWrappedAckedQueueExt) {
|
||||
final JRubyAckedQueueExt inner = ((JRubyWrappedAckedQueueExt) queue).rubyGetQueue();
|
||||
final RubyString dirPath = inner.ruby_dir_path(context);
|
||||
final AbstractNamespacedMetricExt capacityMetrics =
|
||||
pipelineMetric.namespace(context, CAPACITY_NAMESPACE);
|
||||
capacityMetrics.gauge(
|
||||
context, PAGE_CAPACITY_IN_BYTES, inner.ruby_page_capacity(context)
|
||||
);
|
||||
capacityMetrics.gauge(
|
||||
context, MAX_QUEUE_SIZE_IN_BYTES, inner.ruby_max_size_in_bytes(context)
|
||||
);
|
||||
capacityMetrics.gauge(
|
||||
context, MAX_QUEUE_UNREAD_EVENTS, inner.ruby_max_unread_events(context)
|
||||
);
|
||||
capacityMetrics.gauge(
|
||||
context, QUEUE_SIZE_IN_BYTES, inner.ruby_persisted_size_in_bytes(context)
|
||||
);
|
||||
final AbstractNamespacedMetricExt dataMetrics =
|
||||
pipelineMetric.namespace(context, DATA_NAMESPACE);
|
||||
final FileStore fileStore = Files.getFileStore(Paths.get(dirPath.asJavaString()));
|
||||
dataMetrics.gauge(
|
||||
context, FREE_SPACE_IN_BYTES,
|
||||
context.runtime.newFixnum(fileStore.getUnallocatedSpace())
|
||||
);
|
||||
dataMetrics.gauge(context, STORAGE_TYPE, context.runtime.newString(fileStore.type()));
|
||||
dataMetrics.gauge(context, PATH, dirPath);
|
||||
pipelineMetric.gauge(context, EVENTS_KEY, inner.ruby_unread_count(context));
|
||||
}
|
||||
return context.nil;
|
||||
}
|
||||
|
||||
@JRubyMethod
|
||||
public IRubyObject queue() {
|
||||
return queue;
|
||||
}
|
||||
|
||||
private IRubyObject getSetting(final ThreadContext context, final String name) {
|
||||
return settings.callMethod(context, "get_value", context.runtime.newString(name));
|
||||
}
|
||||
|
||||
private AbstractNamespacedMetricExt getDlqMetric(final ThreadContext context) {
|
||||
if (dlqMetric == null) {
|
||||
dlqMetric = metric.namespace(
|
||||
context, RubyArray.newArray(
|
||||
context.runtime,
|
||||
Arrays.asList(
|
||||
STATS_KEY, PIPELINES_KEY, pipelineId.asString().intern(), DLQ_KEY
|
||||
)
|
||||
)
|
||||
);
|
||||
}
|
||||
return dlqMetric;
|
||||
}
|
||||
}
|
|
@ -1,165 +0,0 @@
|
|||
package org.logstash.execution;
|
||||
|
||||
import java.security.MessageDigest;
|
||||
import java.security.NoSuchAlgorithmException;
|
||||
import java.util.UUID;
|
||||
import org.apache.commons.codec.binary.Hex;
|
||||
import org.jruby.Ruby;
|
||||
import org.jruby.RubyBasicObject;
|
||||
import org.jruby.RubyClass;
|
||||
import org.jruby.RubyString;
|
||||
import org.jruby.anno.JRubyClass;
|
||||
import org.jruby.anno.JRubyMethod;
|
||||
import org.jruby.javasupport.JavaUtil;
|
||||
import org.jruby.runtime.ThreadContext;
|
||||
import org.jruby.runtime.builtin.IRubyObject;
|
||||
import org.logstash.RubyUtil;
|
||||
import org.logstash.common.DeadLetterQueueFactory;
|
||||
import org.logstash.common.IncompleteSourceWithMetadataException;
|
||||
import org.logstash.config.ir.ConfigCompiler;
|
||||
import org.logstash.config.ir.PipelineIR;
|
||||
import org.logstash.instrument.metrics.AbstractMetricExt;
|
||||
import org.logstash.instrument.metrics.NullMetricExt;
|
||||
|
||||
@JRubyClass(name = "LogstashPipeline")
|
||||
public final class LogstashPipelineExt extends RubyBasicObject {
|
||||
|
||||
private final RubyString ephemeralId = RubyUtil.RUBY.newString(UUID.randomUUID().toString());
|
||||
|
||||
private RubyString configString;
|
||||
|
||||
private RubyString configHash;
|
||||
|
||||
private IRubyObject settings;
|
||||
|
||||
private IRubyObject pipelineSettings;
|
||||
|
||||
private IRubyObject pipelineId;
|
||||
|
||||
private AbstractMetricExt metric;
|
||||
|
||||
private PipelineIR lir;
|
||||
|
||||
private IRubyObject dlqWriter;
|
||||
|
||||
public LogstashPipelineExt(final Ruby runtime, final RubyClass metaClass) {
|
||||
super(runtime, metaClass);
|
||||
}
|
||||
|
||||
@JRubyMethod
|
||||
public LogstashPipelineExt initialize(final ThreadContext context,
|
||||
final IRubyObject pipelineSettings, final IRubyObject namespacedMetric)
|
||||
throws NoSuchAlgorithmException, IncompleteSourceWithMetadataException {
|
||||
this.pipelineSettings = pipelineSettings;
|
||||
configString = (RubyString) pipelineSettings.callMethod(context, "config_string");
|
||||
configHash = context.runtime.newString(
|
||||
Hex.encodeHexString(
|
||||
MessageDigest.getInstance("SHA1").digest(configString.getBytes())
|
||||
)
|
||||
);
|
||||
this.settings = pipelineSettings.callMethod(context, "settings");
|
||||
final IRubyObject id = getSetting(context, "pipeline.id");
|
||||
if (id.isNil()) {
|
||||
pipelineId = id();
|
||||
} else {
|
||||
pipelineId = id;
|
||||
}
|
||||
if (namespacedMetric.isNil()) {
|
||||
metric = new NullMetricExt(context.runtime, RubyUtil.NULL_METRIC_CLASS).initialize(
|
||||
context, new IRubyObject[0]
|
||||
);
|
||||
} else {
|
||||
final AbstractMetricExt java = (AbstractMetricExt) namespacedMetric;
|
||||
if (getSetting(context, "metric.collect").isTrue()) {
|
||||
metric = java;
|
||||
} else {
|
||||
metric = new NullMetricExt(context.runtime, RubyUtil.NULL_METRIC_CLASS).initialize(
|
||||
context, new IRubyObject[]{java.collector(context)}
|
||||
);
|
||||
}
|
||||
}
|
||||
lir = ConfigCompiler.configToPipelineIR(
|
||||
configString.asJavaString(),
|
||||
getSetting(context, "config.support_escapes").isTrue()
|
||||
);
|
||||
return this;
|
||||
}
|
||||
|
||||
@JRubyMethod(name = "config_str")
|
||||
public RubyString configStr() {
|
||||
return configString;
|
||||
}
|
||||
|
||||
@JRubyMethod(name = "config_hash")
|
||||
public RubyString configHash() {
|
||||
return configHash;
|
||||
}
|
||||
|
||||
@JRubyMethod(name = "ephemeral_id")
|
||||
public RubyString ephemeralId() {
|
||||
return ephemeralId;
|
||||
}
|
||||
|
||||
@JRubyMethod
|
||||
public IRubyObject settings() {
|
||||
return settings;
|
||||
}
|
||||
|
||||
@JRubyMethod(name = "pipeline_config")
|
||||
public IRubyObject pipelineConfig() {
|
||||
return pipelineSettings;
|
||||
}
|
||||
|
||||
@JRubyMethod(name = "pipeline_id")
|
||||
public IRubyObject pipelineId() {
|
||||
return pipelineId;
|
||||
}
|
||||
|
||||
@JRubyMethod
|
||||
public AbstractMetricExt metric() {
|
||||
return metric;
|
||||
}
|
||||
|
||||
@JRubyMethod
|
||||
public IRubyObject lir(final ThreadContext context) {
|
||||
return JavaUtil.convertJavaToUsableRubyObject(context.runtime, lir);
|
||||
}
|
||||
|
||||
@JRubyMethod(name = "dlq_writer")
|
||||
public IRubyObject dlqWriter(final ThreadContext context) {
|
||||
if (dlqWriter == null) {
|
||||
if (dlqEnabled(context).isTrue()) {
|
||||
dlqWriter = JavaUtil.convertJavaToUsableRubyObject(
|
||||
context.runtime,
|
||||
DeadLetterQueueFactory.getWriter(
|
||||
pipelineId.asJavaString(),
|
||||
getSetting(context, "path.dead_letter_queue").asJavaString(),
|
||||
getSetting(context, "dead_letter_queue.max_bytes").convertToInteger()
|
||||
.getLongValue()
|
||||
)
|
||||
);
|
||||
} else {
|
||||
dlqWriter = RubyUtil.DUMMY_DLQ_WRITER_CLASS.callMethod(context, "new");
|
||||
}
|
||||
}
|
||||
return dlqWriter;
|
||||
}
|
||||
|
||||
@JRubyMethod(name = "dlq_enabled?")
|
||||
public IRubyObject dlqEnabled(final ThreadContext context) {
|
||||
return getSetting(context, "dead_letter_queue.enable");
|
||||
}
|
||||
|
||||
@JRubyMethod(name = "close_dlq_writer")
|
||||
public IRubyObject closeDlqWriter(final ThreadContext context) {
|
||||
dlqWriter.callMethod(context, "close");
|
||||
if (dlqEnabled(context).isTrue()) {
|
||||
DeadLetterQueueFactory.release(pipelineId.asJavaString());
|
||||
}
|
||||
return context.nil;
|
||||
}
|
||||
|
||||
private IRubyObject getSetting(final ThreadContext context, final String name) {
|
||||
return settings.callMethod(context, "get_value", context.runtime.newString(name));
|
||||
}
|
||||
}
|
Loading…
Add table
Add a link
Reference in a new issue