mirror of
https://github.com/elastic/logstash.git
synced 2025-06-28 09:46:03 -04:00
parent
fbb167751e
commit
cccd044c92
8 changed files with 62 additions and 52 deletions
|
@ -14,8 +14,7 @@ module LogStash; class JavaPipeline < JavaBasePipeline
|
||||||
:events_consumed,
|
:events_consumed,
|
||||||
:events_filtered,
|
:events_filtered,
|
||||||
:started_at,
|
:started_at,
|
||||||
:thread,
|
:thread
|
||||||
:filter_queue_client
|
|
||||||
|
|
||||||
MAX_INFLIGHT_WARN_THRESHOLD = 10_000
|
MAX_INFLIGHT_WARN_THRESHOLD = 10_000
|
||||||
|
|
||||||
|
@ -24,13 +23,6 @@ module LogStash; class JavaPipeline < JavaBasePipeline
|
||||||
super pipeline_config, namespaced_metric, @logger, agent
|
super pipeline_config, namespaced_metric, @logger, agent
|
||||||
@worker_threads = []
|
@worker_threads = []
|
||||||
|
|
||||||
@filter_queue_client = queue.read_client
|
|
||||||
# Note that @inflight_batches as a central mechanism for tracking inflight
|
|
||||||
# batches will fail if we have multiple read clients here.
|
|
||||||
@filter_queue_client.set_events_metric(metric.namespace([:stats, :events]))
|
|
||||||
@filter_queue_client.set_pipeline_metric(
|
|
||||||
metric.namespace([:stats, :pipelines, pipeline_id.to_s.to_sym, :events])
|
|
||||||
)
|
|
||||||
@drain_queue = settings.get_value("queue.drain") || settings.get("queue.type") == "memory"
|
@drain_queue = settings.get_value("queue.drain") || settings.get("queue.type") == "memory"
|
||||||
|
|
||||||
@events_filtered = java.util.concurrent.atomic.LongAdder.new
|
@events_filtered = java.util.concurrent.atomic.LongAdder.new
|
||||||
|
@ -157,12 +149,6 @@ module LogStash; class JavaPipeline < JavaBasePipeline
|
||||||
return 0
|
return 0
|
||||||
end # def run
|
end # def run
|
||||||
|
|
||||||
def close
|
|
||||||
@filter_queue_client.close
|
|
||||||
queue.close
|
|
||||||
close_dlq_writer
|
|
||||||
end
|
|
||||||
|
|
||||||
def transition_to_running
|
def transition_to_running
|
||||||
@running.make_true
|
@running.make_true
|
||||||
end
|
end
|
||||||
|
@ -223,12 +209,12 @@ module LogStash; class JavaPipeline < JavaBasePipeline
|
||||||
@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)
|
@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)
|
||||||
end
|
end
|
||||||
|
|
||||||
@filter_queue_client.set_batch_dimensions(batch_size, batch_delay)
|
filter_queue_client.set_batch_dimensions(batch_size, batch_delay)
|
||||||
|
|
||||||
pipeline_workers.times do |t|
|
pipeline_workers.times do |t|
|
||||||
thread = Thread.new do
|
thread = Thread.new do
|
||||||
org.logstash.execution.WorkerLoop.new(
|
org.logstash.execution.WorkerLoop.new(
|
||||||
lir_execution, @filter_queue_client, @events_filtered, @events_consumed,
|
lir_execution, filter_queue_client, @events_filtered, @events_consumed,
|
||||||
@flushRequested, @flushing, @shutdownRequested, @drain_queue).run
|
@flushRequested, @flushing, @shutdownRequested, @drain_queue).run
|
||||||
end
|
end
|
||||||
thread.name="[#{pipeline_id}]>worker#{t}"
|
thread.name="[#{pipeline_id}]>worker#{t}"
|
||||||
|
|
|
@ -82,8 +82,7 @@ module LogStash; class Pipeline < BasePipeline
|
||||||
:events_consumed,
|
:events_consumed,
|
||||||
:events_filtered,
|
:events_filtered,
|
||||||
:started_at,
|
:started_at,
|
||||||
:thread,
|
:thread
|
||||||
:filter_queue_client
|
|
||||||
|
|
||||||
MAX_INFLIGHT_WARN_THRESHOLD = 10_000
|
MAX_INFLIGHT_WARN_THRESHOLD = 10_000
|
||||||
|
|
||||||
|
@ -92,14 +91,8 @@ module LogStash; class Pipeline < BasePipeline
|
||||||
|
|
||||||
@worker_threads = []
|
@worker_threads = []
|
||||||
|
|
||||||
@filter_queue_client = queue.read_client
|
|
||||||
@signal_queue = java.util.concurrent.LinkedBlockingQueue.new
|
@signal_queue = java.util.concurrent.LinkedBlockingQueue.new
|
||||||
# Note that @inflight_batches as a central mechanism for tracking inflight
|
|
||||||
# batches will fail if we have multiple read clients here.
|
|
||||||
@filter_queue_client.set_events_metric(metric.namespace([:stats, :events]))
|
|
||||||
@filter_queue_client.set_pipeline_metric(
|
|
||||||
metric.namespace([:stats, :pipelines, pipeline_id.to_s.to_sym, :events])
|
|
||||||
)
|
|
||||||
@drain_queue = settings.get_value("queue.drain") || settings.get("queue.type") == "memory"
|
@drain_queue = settings.get_value("queue.drain") || settings.get("queue.type") == "memory"
|
||||||
|
|
||||||
|
|
||||||
|
@ -225,12 +218,6 @@ module LogStash; class Pipeline < BasePipeline
|
||||||
return 0
|
return 0
|
||||||
end # def run
|
end # def run
|
||||||
|
|
||||||
def close
|
|
||||||
@filter_queue_client.close
|
|
||||||
queue.close
|
|
||||||
close_dlq_writer
|
|
||||||
end
|
|
||||||
|
|
||||||
def transition_to_running
|
def transition_to_running
|
||||||
@running.make_true
|
@running.make_true
|
||||||
end
|
end
|
||||||
|
@ -320,12 +307,12 @@ module LogStash; class Pipeline < BasePipeline
|
||||||
# Main body of what a worker thread does
|
# Main body of what a worker thread does
|
||||||
# Repeatedly takes batches off the queue, filters, then outputs them
|
# Repeatedly takes batches off the queue, filters, then outputs them
|
||||||
def worker_loop(batch_size, batch_delay)
|
def worker_loop(batch_size, batch_delay)
|
||||||
@filter_queue_client.set_batch_dimensions(batch_size, batch_delay)
|
filter_queue_client.set_batch_dimensions(batch_size, batch_delay)
|
||||||
output_events_map = Hash.new { |h, k| h[k] = [] }
|
output_events_map = Hash.new { |h, k| h[k] = [] }
|
||||||
while true
|
while true
|
||||||
signal = @signal_queue.poll || NO_SIGNAL
|
signal = @signal_queue.poll || NO_SIGNAL
|
||||||
|
|
||||||
batch = @filter_queue_client.read_batch.to_java # metrics are started in read_batch
|
batch = filter_queue_client.read_batch.to_java # metrics are started in read_batch
|
||||||
batch_size = batch.filteredSize
|
batch_size = batch.filteredSize
|
||||||
if batch_size > 0
|
if batch_size > 0
|
||||||
@events_consumed.add(batch_size)
|
@events_consumed.add(batch_size)
|
||||||
|
@ -334,7 +321,7 @@ module LogStash; class Pipeline < BasePipeline
|
||||||
flush_filters_to_batch(batch, :final => false) if signal.flush?
|
flush_filters_to_batch(batch, :final => false) if signal.flush?
|
||||||
if batch.filteredSize > 0
|
if batch.filteredSize > 0
|
||||||
output_batch(batch, output_events_map)
|
output_batch(batch, output_events_map)
|
||||||
@filter_queue_client.close_batch(batch)
|
filter_queue_client.close_batch(batch)
|
||||||
end
|
end
|
||||||
# keep break at end of loop, after the read_batch operation, some pipeline specs rely on this "final read_batch" before shutdown.
|
# keep break at end of loop, after the read_batch operation, some pipeline specs rely on this "final read_batch" before shutdown.
|
||||||
break if (@worker_shutdown.get && !draining_queue?)
|
break if (@worker_shutdown.get && !draining_queue?)
|
||||||
|
@ -342,11 +329,11 @@ module LogStash; class Pipeline < BasePipeline
|
||||||
|
|
||||||
# we are shutting down, queue is drained if it was required, now perform a final flush.
|
# we are shutting down, queue is drained if it was required, now perform a final flush.
|
||||||
# for this we need to create a new empty batch to contain the final flushed events
|
# for this we need to create a new empty batch to contain the final flushed events
|
||||||
batch = @filter_queue_client.to_java.newBatch
|
batch = filter_queue_client.to_java.newBatch
|
||||||
@filter_queue_client.start_metrics(batch) # explicitly call start_metrics since we dont do a read_batch here
|
filter_queue_client.start_metrics(batch) # explicitly call start_metrics since we dont do a read_batch here
|
||||||
flush_filters_to_batch(batch, :final => true)
|
flush_filters_to_batch(batch, :final => true)
|
||||||
output_batch(batch, output_events_map)
|
output_batch(batch, output_events_map)
|
||||||
@filter_queue_client.close_batch(batch)
|
filter_queue_client.close_batch(batch)
|
||||||
end
|
end
|
||||||
|
|
||||||
def filter_batch(batch)
|
def filter_batch(batch)
|
||||||
|
@ -354,7 +341,7 @@ module LogStash; class Pipeline < BasePipeline
|
||||||
#these are both original and generated events
|
#these are both original and generated events
|
||||||
batch.merge(e) unless e.cancelled?
|
batch.merge(e) unless e.cancelled?
|
||||||
end
|
end
|
||||||
@filter_queue_client.add_filtered_metrics(batch.filtered_size)
|
filter_queue_client.add_filtered_metrics(batch.filtered_size)
|
||||||
@events_filtered.add(batch.filteredSize)
|
@events_filtered.add(batch.filteredSize)
|
||||||
rescue Exception => e
|
rescue Exception => e
|
||||||
# Plugins authors should manage their own exceptions in the plugin code
|
# Plugins authors should manage their own exceptions in the plugin code
|
||||||
|
@ -386,7 +373,7 @@ module LogStash; class Pipeline < BasePipeline
|
||||||
events.clear
|
events.clear
|
||||||
end
|
end
|
||||||
|
|
||||||
@filter_queue_client.add_output_metrics(batch.filtered_size)
|
filter_queue_client.add_output_metrics(batch.filtered_size)
|
||||||
end
|
end
|
||||||
|
|
||||||
def wait_inputs
|
def wait_inputs
|
||||||
|
@ -615,6 +602,6 @@ module LogStash; class Pipeline < BasePipeline
|
||||||
end
|
end
|
||||||
|
|
||||||
def draining_queue?
|
def draining_queue?
|
||||||
@drain_queue ? !@filter_queue_client.empty? : false
|
@drain_queue ? !filter_queue_client.empty? : false
|
||||||
end
|
end
|
||||||
end; end
|
end; end
|
||||||
|
|
|
@ -13,6 +13,7 @@ import org.jruby.runtime.ThreadContext;
|
||||||
import org.jruby.runtime.builtin.IRubyObject;
|
import org.jruby.runtime.builtin.IRubyObject;
|
||||||
import org.logstash.RubyUtil;
|
import org.logstash.RubyUtil;
|
||||||
import org.logstash.execution.AbstractWrappedQueueExt;
|
import org.logstash.execution.AbstractWrappedQueueExt;
|
||||||
|
import org.logstash.execution.QueueReadClientBase;
|
||||||
import org.logstash.ext.JRubyAbstractQueueWriteClientExt;
|
import org.logstash.ext.JRubyAbstractQueueWriteClientExt;
|
||||||
import org.logstash.ext.JrubyAckedReadClientExt;
|
import org.logstash.ext.JrubyAckedReadClientExt;
|
||||||
import org.logstash.ext.JrubyAckedWriteClientExt;
|
import org.logstash.ext.JrubyAckedWriteClientExt;
|
||||||
|
@ -77,7 +78,7 @@ public final class JRubyWrappedAckedQueueExt extends AbstractWrappedQueueExt {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected IRubyObject getReadClient() {
|
protected QueueReadClientBase getReadClient() {
|
||||||
return JrubyAckedReadClientExt.create(queue);
|
return JrubyAckedReadClientExt.create(queue);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -68,14 +68,16 @@ public class AbstractPipelineExt extends RubyBasicObject {
|
||||||
|
|
||||||
private static final RubySymbol PATH = RubyUtil.RUBY.newSymbol("path");
|
private static final RubySymbol PATH = RubyUtil.RUBY.newSymbol("path");
|
||||||
|
|
||||||
private static final RubySymbol STATS_KEY = RubyUtil.RUBY.newSymbol("stats");
|
|
||||||
|
|
||||||
private static final RubySymbol TYPE_KEY = RubyUtil.RUBY.newSymbol("type");
|
private static final RubySymbol TYPE_KEY = RubyUtil.RUBY.newSymbol("type");
|
||||||
|
|
||||||
private static final RubySymbol QUEUE_KEY = RubyUtil.RUBY.newSymbol("queue");
|
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_KEY = RubyUtil.RUBY.newSymbol("dlq");
|
||||||
|
|
||||||
|
private static final RubyArray EVENTS_METRIC_NAMESPACE = RubyArray.newArray(
|
||||||
|
RubyUtil.RUBY, new IRubyObject[]{MetricKeys.STATS_KEY, MetricKeys.EVENTS_KEY}
|
||||||
|
);
|
||||||
|
|
||||||
protected PipelineIR lir;
|
protected PipelineIR lir;
|
||||||
|
|
||||||
private final RubyString ephemeralId = RubyUtil.RUBY.newString(UUID.randomUUID().toString());
|
private final RubyString ephemeralId = RubyUtil.RUBY.newString(UUID.randomUUID().toString());
|
||||||
|
@ -102,6 +104,8 @@ public class AbstractPipelineExt extends RubyBasicObject {
|
||||||
|
|
||||||
private JRubyAbstractQueueWriteClientExt inputQueueClient;
|
private JRubyAbstractQueueWriteClientExt inputQueueClient;
|
||||||
|
|
||||||
|
private QueueReadClientBase filterQueueClient;
|
||||||
|
|
||||||
public AbstractPipelineExt(final Ruby runtime, final RubyClass metaClass) {
|
public AbstractPipelineExt(final Ruby runtime, final RubyClass metaClass) {
|
||||||
super(runtime, metaClass);
|
super(runtime, metaClass);
|
||||||
}
|
}
|
||||||
|
@ -129,6 +133,7 @@ public class AbstractPipelineExt extends RubyBasicObject {
|
||||||
throw new IllegalStateException(ex);
|
throw new IllegalStateException(ex);
|
||||||
}
|
}
|
||||||
inputQueueClient = queue.writeClient(context);
|
inputQueueClient = queue.writeClient(context);
|
||||||
|
filterQueueClient = queue.readClient();
|
||||||
final IRubyObject id = getSetting(context, "pipeline.id");
|
final IRubyObject id = getSetting(context, "pipeline.id");
|
||||||
if (id.isNil()) {
|
if (id.isNil()) {
|
||||||
pipelineId = id();
|
pipelineId = id();
|
||||||
|
@ -153,9 +158,27 @@ public class AbstractPipelineExt extends RubyBasicObject {
|
||||||
configString.asJavaString(),
|
configString.asJavaString(),
|
||||||
getSetting(context, "config.support_escapes").isTrue()
|
getSetting(context, "config.support_escapes").isTrue()
|
||||||
);
|
);
|
||||||
|
filterQueueClient.setEventsMetric(metric.namespace(context, EVENTS_METRIC_NAMESPACE));
|
||||||
|
filterQueueClient.setPipelineMetric(
|
||||||
|
metric.namespace(
|
||||||
|
context,
|
||||||
|
RubyArray.newArray(
|
||||||
|
context.runtime,
|
||||||
|
new IRubyObject[]{
|
||||||
|
MetricKeys.STATS_KEY, MetricKeys.PIPELINES_KEY,
|
||||||
|
pipelineId.convertToString().intern19(), MetricKeys.EVENTS_KEY
|
||||||
|
}
|
||||||
|
)
|
||||||
|
)
|
||||||
|
);
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@JRubyMethod(name = "filter_queue_client")
|
||||||
|
public final QueueReadClientBase filterQueueClient() {
|
||||||
|
return filterQueueClient;
|
||||||
|
}
|
||||||
|
|
||||||
@JRubyMethod(name = "config_str")
|
@JRubyMethod(name = "config_str")
|
||||||
public final RubyString configStr() {
|
public final RubyString configStr() {
|
||||||
return configString;
|
return configString;
|
||||||
|
@ -262,7 +285,7 @@ public class AbstractPipelineExt extends RubyBasicObject {
|
||||||
context,
|
context,
|
||||||
RubyArray.newArray(
|
RubyArray.newArray(
|
||||||
context.runtime,
|
context.runtime,
|
||||||
Arrays.asList(STATS_KEY, MetricKeys.PIPELINES_KEY, pipelineId.asString().intern(), QUEUE_KEY)
|
Arrays.asList(MetricKeys.STATS_KEY, MetricKeys.PIPELINES_KEY, pipelineId.asString().intern(), QUEUE_KEY)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
pipelineMetric.gauge(context, TYPE_KEY, getSetting(context, "queue.type"));
|
pipelineMetric.gauge(context, TYPE_KEY, getSetting(context, "queue.type"));
|
||||||
|
@ -307,6 +330,14 @@ public class AbstractPipelineExt extends RubyBasicObject {
|
||||||
return queue;
|
return queue;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@JRubyMethod
|
||||||
|
public final IRubyObject close(final ThreadContext context) throws IOException {
|
||||||
|
filterQueueClient.close();
|
||||||
|
queue.close(context);
|
||||||
|
closeDlqWriter(context);
|
||||||
|
return context.nil;
|
||||||
|
}
|
||||||
|
|
||||||
@JRubyMethod(name = "wrapped_write_client", visibility = Visibility.PROTECTED)
|
@JRubyMethod(name = "wrapped_write_client", visibility = Visibility.PROTECTED)
|
||||||
public final JRubyWrappedWriteClientExt wrappedWriteClient(final ThreadContext context,
|
public final JRubyWrappedWriteClientExt wrappedWriteClient(final ThreadContext context,
|
||||||
final IRubyObject pluginId) {
|
final IRubyObject pluginId) {
|
||||||
|
@ -324,7 +355,8 @@ public class AbstractPipelineExt extends RubyBasicObject {
|
||||||
context, RubyArray.newArray(
|
context, RubyArray.newArray(
|
||||||
context.runtime,
|
context.runtime,
|
||||||
Arrays.asList(
|
Arrays.asList(
|
||||||
STATS_KEY, MetricKeys.PIPELINES_KEY, pipelineId.asString().intern(), DLQ_KEY
|
MetricKeys.STATS_KEY, MetricKeys.PIPELINES_KEY,
|
||||||
|
pipelineId.asString().intern(), DLQ_KEY
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
|
@ -22,7 +22,7 @@ public abstract class AbstractWrappedQueueExt extends RubyBasicObject {
|
||||||
}
|
}
|
||||||
|
|
||||||
@JRubyMethod(name = "read_client")
|
@JRubyMethod(name = "read_client")
|
||||||
public final IRubyObject readClient() {
|
public final QueueReadClientBase readClient() {
|
||||||
return getReadClient();
|
return getReadClient();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -35,5 +35,5 @@ public abstract class AbstractWrappedQueueExt extends RubyBasicObject {
|
||||||
|
|
||||||
protected abstract JRubyAbstractQueueWriteClientExt getWriteClient(ThreadContext context);
|
protected abstract JRubyAbstractQueueWriteClientExt getWriteClient(ThreadContext context);
|
||||||
|
|
||||||
protected abstract IRubyObject getReadClient();
|
protected abstract QueueReadClientBase getReadClient();
|
||||||
}
|
}
|
||||||
|
|
|
@ -10,6 +10,7 @@ import org.jruby.anno.JRubyMethod;
|
||||||
import org.jruby.runtime.ThreadContext;
|
import org.jruby.runtime.ThreadContext;
|
||||||
import org.jruby.runtime.builtin.IRubyObject;
|
import org.jruby.runtime.builtin.IRubyObject;
|
||||||
import org.logstash.execution.AbstractWrappedQueueExt;
|
import org.logstash.execution.AbstractWrappedQueueExt;
|
||||||
|
import org.logstash.execution.QueueReadClientBase;
|
||||||
|
|
||||||
@JRubyClass(name = "WrappedSynchronousQueue")
|
@JRubyClass(name = "WrappedSynchronousQueue")
|
||||||
public final class JrubyWrappedSynchronousQueueExt extends AbstractWrappedQueueExt {
|
public final class JrubyWrappedSynchronousQueueExt extends AbstractWrappedQueueExt {
|
||||||
|
@ -35,7 +36,7 @@ public final class JrubyWrappedSynchronousQueueExt extends AbstractWrappedQueueE
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected IRubyObject getReadClient() {
|
protected QueueReadClientBase getReadClient() {
|
||||||
// batch size and timeout are currently hard-coded to 125 and 50ms as values observed
|
// batch size and timeout are currently hard-coded to 125 and 50ms as values observed
|
||||||
// to be reasonable tradeoffs between latency and throughput per PR #8707
|
// to be reasonable tradeoffs between latency and throughput per PR #8707
|
||||||
return JrubyMemoryReadClientExt.create(queue, 125, 50);
|
return JrubyMemoryReadClientExt.create(queue, 125, 50);
|
||||||
|
|
|
@ -23,4 +23,6 @@ public final class MetricKeys {
|
||||||
RubyUtil.RUBY.newSymbol("duration_in_millis");
|
RubyUtil.RUBY.newSymbol("duration_in_millis");
|
||||||
|
|
||||||
public static final RubySymbol FILTERED_KEY = RubyUtil.RUBY.newSymbol("filtered");
|
public static final RubySymbol FILTERED_KEY = RubyUtil.RUBY.newSymbol("filtered");
|
||||||
|
|
||||||
|
public static final RubySymbol STATS_KEY = RubyUtil.RUBY.newSymbol("stats");
|
||||||
}
|
}
|
||||||
|
|
|
@ -279,8 +279,6 @@ public final class PluginFactoryExt {
|
||||||
@JRubyClass(name = "PluginMetricFactory")
|
@JRubyClass(name = "PluginMetricFactory")
|
||||||
public static final class Metrics extends RubyBasicObject {
|
public static final class Metrics extends RubyBasicObject {
|
||||||
|
|
||||||
private static final RubySymbol STATS = RubyUtil.RUBY.newSymbol("stats");
|
|
||||||
|
|
||||||
private static final RubySymbol PLUGINS = RubyUtil.RUBY.newSymbol("plugins");
|
private static final RubySymbol PLUGINS = RubyUtil.RUBY.newSymbol("plugins");
|
||||||
|
|
||||||
private RubySymbol pipelineId;
|
private RubySymbol pipelineId;
|
||||||
|
@ -308,7 +306,10 @@ public final class PluginFactoryExt {
|
||||||
return metric.namespace(
|
return metric.namespace(
|
||||||
context,
|
context,
|
||||||
RubyArray.newArray(
|
RubyArray.newArray(
|
||||||
context.runtime, Arrays.asList(STATS, MetricKeys.PIPELINES_KEY, pipelineId, PLUGINS)
|
context.runtime,
|
||||||
|
Arrays.asList(
|
||||||
|
MetricKeys.STATS_KEY, MetricKeys.PIPELINES_KEY, pipelineId, PLUGINS
|
||||||
|
)
|
||||||
)
|
)
|
||||||
).namespace(
|
).namespace(
|
||||||
context, RubyUtil.RUBY.newSymbol(String.format("%ss", pluginType.asJavaString()))
|
context, RubyUtil.RUBY.newSymbol(String.format("%ss", pluginType.asJavaString()))
|
||||||
|
|
Loading…
Add table
Add a link
Reference in a new issue