Add some javadocs comments to the classes

Removes never used class org.logstash.execution.queue.LegacyMemoryQueueWriter

(cherry picked from commit 9a63ec2705)
This commit is contained in:
andsel 2020-12-04 09:30:38 +01:00 committed by J.A.R.V.I.S. - an Elastic git bot
parent a9e4ba6073
commit bc56c2068e
80 changed files with 316 additions and 48 deletions

View file

@ -43,7 +43,9 @@ import java.io.IOException;
import java.nio.ByteBuffer;
import org.logstash.ackedqueue.Queueable;
/**
* Dead letter queue item
*/
public class DLQEntry implements Cloneable, Queueable {
private final Event event;

View file

@ -43,6 +43,9 @@ import java.util.Map;
import static org.logstash.ObjectMappers.CBOR_MAPPER;
import static org.logstash.ObjectMappers.JSON_MAPPER;
/**
* Event implementation, is the Logstash's event implementation
* */
public final class Event implements Cloneable, Queueable, co.elastic.logstash.api.Event {
private boolean cancelled;

View file

@ -29,6 +29,9 @@ import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import org.jruby.RubyString;
/**
* Represents a reference to another field of the event {@link Event}
* */
public final class FieldReference {
/**
* A custom unchecked {@link RuntimeException} that can be thrown by parsing methods when

View file

@ -125,6 +125,9 @@ public final class RubyJavaIntegration {
}
}
/**
* Helper class to provide some typical Ruby methods to Java Collection class when used from Ruby
* */
public static final class JavaCollectionOverride {
private static final Collection<IRubyObject> NIL_COLLECTION =
@ -223,6 +226,9 @@ public final class RubyJavaIntegration {
}
}
/**
* Helper class to provide "is_a?" method to Java Map class when use dfrom Ruby
* */
public static final class JavaMapOverride {
private JavaMapOverride() {

View file

@ -25,7 +25,9 @@ import java.util.LinkedHashSet;
import java.util.List;
import java.util.Map;
/**
* Static utility methods that provide merge methods for List and Map.
*/
public class Util {
private Util() {}

View file

@ -28,6 +28,9 @@ import org.logstash.ext.JrubyEventExtLibrary.RubyEvent;
import static org.logstash.RubyUtil.RUBY;
/**
* Persistent queue batch of events collection
* */
public final class AckedBatch {
private Batch batch;

View file

@ -31,6 +31,9 @@ import java.util.Collection;
import static org.logstash.RubyUtil.RUBY;
/**
* Persistent queue collection of events implementation
* */
public final class AckedReadBatch implements QueueBatch {
private AckedBatch ackedBatch;

View file

@ -26,6 +26,9 @@ import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.atomic.AtomicBoolean;
/**
* Container for a set of events from queue to be processed by filters/outputs.
* */
public class Batch implements Closeable {
private final List<Queueable> elements;

View file

@ -20,6 +20,11 @@
package org.logstash.ackedqueue;
/**
* Persistent queue chekpoint. There is one checkpoint per queue, and simply it's a picture of some
* management data like first not acked sequence number.
*
* */
public class Checkpoint {
// Checkpoint file structure see FileCheckpointIO

View file

@ -28,6 +28,10 @@ import org.codehaus.commons.nullanalysis.NotNull;
import org.logstash.ackedqueue.io.CheckpointIO;
import org.logstash.ackedqueue.io.PageIO;
/**
* Represents persistent queue Page metadata, like the page number, the minimum sequence number contained in the page,
* the status of the page (writeable or not).
* */
public final class Page implements Closeable {
protected final int pageNum;
protected long minSeqNum; // TODO: see if we can make it final?

View file

@ -30,6 +30,9 @@ import java.util.Comparator;
import java.util.stream.StreamSupport;
import org.logstash.ackedqueue.io.FileCheckpointIO;
/**
* Utility application to detect corrupted persistent queues.
* */
public final class PqCheck {
private static final String DEFAULT_PQ_DIR = "data/queue/main";

View file

@ -46,6 +46,9 @@ import org.logstash.ackedqueue.io.MmapPageIOV2;
import org.logstash.ackedqueue.io.PageIO;
import org.logstash.common.FsUtil;
/**
* Persistent queue implementation.
* */
public final class Queue implements Closeable {
private long seqNum;

View file

@ -36,6 +36,9 @@ import org.logstash.ackedqueue.ext.JRubyWrappedAckedQueueExt;
import org.logstash.execution.AbstractWrappedQueueExt;
import org.logstash.ext.JrubyWrappedSynchronousQueueExt;
/**
* Persistent queue factory JRuby extension.
* */
@JRubyClass(name = "QueueFactory")
public final class QueueFactoryExt extends RubyBasicObject {

View file

@ -20,6 +20,9 @@
package org.logstash.ackedqueue;
/**
* Runtime exception specialization for persistent queue related runtime errors.
* */
public class QueueRuntimeException extends RuntimeException {
private static final long serialVersionUID = 1L;

View file

@ -41,6 +41,9 @@ import org.logstash.ackedqueue.io.MmapPageIOV1;
import org.logstash.ackedqueue.io.MmapPageIOV2;
import org.logstash.ackedqueue.io.PageIO;
/**
* Exposes the {@link #upgradeQueueDirectoryToV2} method upgrade old v1 queue disk format to the new v2
* */
public final class QueueUpgrade {
private static final Logger LOGGER = LogManager.getLogger(QueueUpgrade.class);

View file

@ -22,6 +22,9 @@ package org.logstash.ackedqueue;
import java.io.IOException;
/**
* Subclasses that are stored in queues must implement this to provide serializer and deserializer.
* */
public interface Queueable {
byte[] serialize() throws IOException;

View file

@ -23,6 +23,9 @@ package org.logstash.ackedqueue;
import java.util.List;
import org.logstash.ackedqueue.io.LongVector;
/**
* Carries sequence numbers and items read from queue.
* */
public class SequencedList<E> {
private final List<E> elements;
private final LongVector seqNums;

View file

@ -20,6 +20,9 @@
package org.logstash.ackedqueue;
/**
* Persistent queue settings definition.
* */
public interface Settings {
Class<? extends Queueable> getElementClass();
@ -38,6 +41,9 @@ public interface Settings {
boolean getCheckpointRetry();
/**
* Persistent queue Setting's fluent builder definition
* */
interface Builder {
Builder elementClass(Class<? extends Queueable> elementClass);

View file

@ -20,6 +20,9 @@
package org.logstash.ackedqueue;
/**
* Persistent queue settings implementation.
* */
public class SettingsImpl implements Settings {
private String dirForFiles;
private Class<? extends Queueable> elementClass;
@ -94,6 +97,9 @@ public class SettingsImpl implements Settings {
return this.checkpointRetry;
}
/**
* Default implementation for Setting's Builder
* */
private static final class BuilderImpl implements Builder {
/**

View file

@ -38,6 +38,9 @@ import org.logstash.ackedqueue.Batch;
import org.logstash.ackedqueue.Queue;
import org.logstash.ackedqueue.SettingsImpl;
/**
* JRuby extension to wrap a persistent queue istance.
* */
@JRubyClass(name = "AckedQueue")
public final class JRubyAckedQueueExt extends RubyObject {

View file

@ -39,6 +39,9 @@ import org.logstash.ext.JrubyAckedReadClientExt;
import org.logstash.ext.JrubyAckedWriteClientExt;
import org.logstash.ext.JrubyEventExtLibrary;
/**
* JRuby extension
* */
@JRubyClass(name = "WrappedAckedQueue")
public final class JRubyWrappedAckedQueueExt extends AbstractWrappedQueueExt {

View file

@ -23,6 +23,9 @@ package org.logstash.ackedqueue.io;
import org.logstash.ackedqueue.Checkpoint;
import java.io.IOException;
/**
* Internal API to access checkpoint stored in files.
* */
public interface CheckpointIO {
// @return Checkpoint the written checkpoint object

View file

@ -33,6 +33,10 @@ import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.logstash.ackedqueue.Checkpoint;
/**
* File implementation for {@link CheckpointIO}
* */
public class FileCheckpointIO implements CheckpointIO {
// Checkpoint file structure
//

View file

@ -20,6 +20,12 @@
package org.logstash.ackedqueue.io;
/**
* Internal class used in persistent queue implementation.
*
* It'a vector that stores primitives long (no autoboxing) expanding, by copy, the underling array when
* more space is needed.
* */
public final class LongVector {
private int count;

View file

@ -35,6 +35,9 @@ import org.apache.logging.log4j.Logger;
import org.logstash.LogstashJavaCompat;
import org.logstash.ackedqueue.SequencedList;
/**
* Internal API, v2 mmap implementation of {@link PageIO}
* */
public final class MmapPageIOV2 implements PageIO {
public static final byte VERSION_TWO = (byte) 2;
@ -392,6 +395,9 @@ public final class MmapPageIOV2 implements PageIO {
}
}
/**
* Invalid Page structure exception
* */
public static final class PageIOInvalidElementException extends IOException {
private static final long serialVersionUID = 1L;
@ -401,6 +407,9 @@ public final class MmapPageIOV2 implements PageIO {
}
}
/**
* Invalid page version exception.
* */
public static final class PageIOInvalidVersionException extends IOException {
private static final long serialVersionUID = 1L;

View file

@ -25,6 +25,9 @@ import org.logstash.ackedqueue.SequencedList;
import java.io.Closeable;
import java.io.IOException;
/**
* Internal API to access pages stored in files.
* */
public interface PageIO extends Closeable {
// the concrete class should be constructed with the pageNum, capacity and dirPath attributes

View file

@ -65,6 +65,9 @@ import org.logstash.instrument.metrics.NullMetricExt;
import org.logstash.secret.store.SecretStore;
import org.logstash.secret.store.SecretStoreExt;
/**
* JRuby extension to provide ancestor class for Ruby's Pipeline and JavaPipeline classes.
* */
@JRubyClass(name = "AbstractPipeline")
public class AbstractPipelineExt extends RubyBasicObject {

View file

@ -29,6 +29,9 @@ import org.jruby.runtime.ThreadContext;
import org.jruby.runtime.builtin.IRubyObject;
import org.logstash.ext.JRubyAbstractQueueWriteClientExt;
/**
* Interface definition for JRuby extensions for in memory and persistent queue
* */
@JRubyClass(name = "AbstractWrappedQueue")
public abstract class AbstractWrappedQueueExt extends RubyBasicObject {

View file

@ -33,6 +33,9 @@ import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
/**
* JRuby extension, used by agent to collect the results of running pipeliene actions (Create, Update, Delete)
* */
@JRubyClass(name = "ConvergeResult")
public class ConvergeResultExt extends RubyObject {
@ -102,6 +105,9 @@ public class ConvergeResultExt extends RubyObject {
}
/**
* Base class for all action results classes (Failed / Success)
* */
@JRubyClass(name = "ActionResult")
public static abstract class ActionResultExt extends RubyBasicObject {
@ -153,6 +159,9 @@ public class ConvergeResultExt extends RubyObject {
protected abstract boolean getSuccessFul();
}
/**
* Failed result of running an action.
* */
@JRubyClass(name = "FailedAction")
public static final class FailedActionExt extends ActionResultExt {
@ -209,6 +218,9 @@ public class ConvergeResultExt extends RubyObject {
}
}
/**
* Successful result of running an action.
* */
@JRubyClass(name = "SuccessfulAction")
public static final class SuccessfulActionExt extends ActionResultExt {

View file

@ -30,6 +30,9 @@ import org.jruby.anno.JRubyMethod;
import org.jruby.runtime.ThreadContext;
import org.jruby.runtime.builtin.IRubyObject;
/**
* JRuby extension that provides a class to keep track of listener and dispatch events to those.
* */
@JRubyClass(name = "EventDispatcher")
public final class EventDispatcherExt extends RubyBasicObject {

View file

@ -31,6 +31,10 @@ import org.jruby.runtime.builtin.IRubyObject;
import org.logstash.RubyUtil;
import org.logstash.common.AbstractDeadLetterQueueWriterExt;
/**
* JRuby extension to provide execution context to the plugins,
* instantiated by {@link org.logstash.plugins.factory.ExecutionContextFactoryExt}
* */
@JRubyClass(name = "ExecutionContext")
public final class ExecutionContextExt extends RubyObject {

View file

@ -44,6 +44,9 @@ import java.security.NoSuchAlgorithmException;
import java.util.Collection;
import java.util.stream.Stream;
/**
* JRuby extension used as parent for Ruby's JavaPipelines
* */
@JRubyClass(name = "JavaBasePipeline")
public final class JavaBasePipelineExt extends AbstractPipelineExt {

View file

@ -26,6 +26,9 @@ import java.util.Collection;
import static org.logstash.RubyUtil.RUBY;
/**
* In memory queue collection of events implementation
* */
public final class MemoryReadBatch implements QueueBatch {
private final Collection<RubyEvent> events;

View file

@ -28,6 +28,9 @@ import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
@SuppressWarnings("try")
/**
* Used by Ruby's JavaPipeline
* */
public final class PeriodicFlush implements AutoCloseable {
private static final Logger LOGGER = LogManager.getLogger(PeriodicFlush.class);

View file

@ -37,6 +37,9 @@ import org.logstash.config.ir.compiler.AbstractOutputDelegatorExt;
import java.util.Collection;
/**
* JRuby extension
* */
@JRubyClass(name = "PipelineReporter")
public final class PipelineReporterExt extends RubyBasicObject {

View file

@ -24,6 +24,9 @@ import org.logstash.ext.JrubyEventExtLibrary.RubyEvent;
import java.io.IOException;
import java.util.Collection;
/**
* Defines collection of queue events to be processed.
* */
public interface QueueBatch {
int filteredSize();
RubyArray<RubyEvent> to_a();

View file

@ -22,6 +22,9 @@ package org.logstash.execution;
import java.io.IOException;
/**
* Defines the methods to interact with queues from the the reader perspective
* */
public interface QueueReadClient {
QueueBatch readBatch() throws InterruptedException;
QueueBatch newBatch();

View file

@ -39,6 +39,9 @@ import java.io.IOException;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.TimeUnit;
/**
* Common code shared by Persistent and In-Memory queues clients implementation
* */
@JRubyClass(name = "QueueReadClientBase")
public abstract class QueueReadClientBase extends RubyObject implements QueueReadClient {

View file

@ -37,6 +37,9 @@ import org.jruby.runtime.ThreadContext;
import org.jruby.runtime.builtin.IRubyObject;
import org.logstash.RubyUtil;
/**
* JRuby extension, used by pipelines to execute the shutdown flow of a pipeline.
* */
@JRubyClass(name = "ShutdownWatcher")
public final class ShutdownWatcherExt extends RubyBasicObject {

View file

@ -25,6 +25,10 @@ import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.logstash.config.ir.CompiledPipeline;
/**
* Pipeline execution worker, it's responsible to execute filters and output plugins for each {@link QueueBatch} that
* pull out from queue.
* */
public final class WorkerLoop implements Runnable {
private static final Logger LOGGER = LogManager.getLogger(WorkerLoop.class);

View file

@ -1,46 +0,0 @@
/*
* Licensed to Elasticsearch B.V. under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch B.V. 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.execution.queue;
import java.util.Map;
import java.util.concurrent.BlockingQueue;
import org.logstash.Event;
import org.logstash.RubyUtil;
import org.logstash.ext.JrubyEventExtLibrary;
public final class LegacyMemoryQueueWriter implements QueueWriter {
private final BlockingQueue<JrubyEventExtLibrary.RubyEvent> queue;
public LegacyMemoryQueueWriter(final BlockingQueue<JrubyEventExtLibrary.RubyEvent> queue) {
this.queue = queue;
}
@Override
public void push(final Map<String, Object> event) {
try {
queue.put(JrubyEventExtLibrary.RubyEvent.newRubyEvent(RubyUtil.RUBY, new Event(event)));
} catch (final InterruptedException ex) {
throw new IllegalStateException(ex);
}
}
}

View file

@ -35,6 +35,9 @@ import org.logstash.execution.QueueReadClientBase;
import java.io.IOException;
/**
* JRuby extension to provide an implementation of queue client for Persistent queue
* */
@JRubyClass(name = "AckedReadClient", parent = "QueueReadClientBase")
public final class JrubyAckedReadClientExt extends QueueReadClientBase implements QueueReadClient {

View file

@ -31,6 +31,9 @@ import org.logstash.execution.MemoryReadBatch;
import org.logstash.execution.QueueBatch;
import org.logstash.execution.QueueReadClientBase;
/**
* JRuby extension to provide an implementation of queue client for InMemory queue
* */
@JRubyClass(name = "MemoryReadClient", parent = "QueueReadClientBase")
public final class JrubyMemoryReadClientExt extends QueueReadClientBase {

View file

@ -32,6 +32,9 @@ import org.jruby.runtime.builtin.IRubyObject;
import org.logstash.execution.AbstractWrappedQueueExt;
import org.logstash.execution.QueueReadClientBase;
/**
* JRuby extension to wrap in memory queue
* */
@JRubyClass(name = "WrappedSynchronousQueue")
public final class JrubyWrappedSynchronousQueueExt extends AbstractWrappedQueueExt {

View file

@ -28,6 +28,9 @@ import org.jruby.anno.JRubyMethod;
import org.jruby.runtime.ThreadContext;
import org.jruby.runtime.builtin.IRubyObject;
/**
* JRuby extension to model the metrics collection
* */
@JRubyClass(name = "AbstractMetric")
public abstract class AbstractMetricExt extends RubyObject {

View file

@ -48,6 +48,9 @@ import org.apache.logging.log4j.message.Message;
import java.util.List;
/**
* Customized Logstash log event
* */
@JsonSerialize(using = CustomLogEventSerializer.class)
public class CustomLogEvent extends Log4jLogEvent {

View file

@ -28,6 +28,9 @@ import com.fasterxml.jackson.databind.SerializerProvider;
import java.io.IOException;
import java.util.Map;
/**
* Json serializer for logging messages, use in json appender.
* */
public class CustomLogEventSerializer extends JsonSerializer<CustomLogEvent> {
@Override
public void serialize(CustomLogEvent event, JsonGenerator generator, SerializerProvider provider) throws IOException {

View file

@ -29,6 +29,9 @@ import org.jruby.anno.JRubyMethod;
import org.jruby.runtime.ThreadContext;
import org.jruby.runtime.builtin.IRubyObject;
/**
* JRuby extension to provide deprecation logger functionality to Ruby classes
* */
@JRubyClass(name = "DeprecationLogger")
public class DeprecationLoggerExt extends RubyObject {

View file

@ -33,6 +33,9 @@ import java.util.Locale;
import static org.logstash.log.SlowLoggerExt.toLong;
/**
* JRuby extension, it's part of log4j wrapping for JRuby.
* */
@JRubyModule(name = "Loggable")
public final class LoggableExt {

View file

@ -39,6 +39,10 @@ import org.jruby.runtime.builtin.IRubyObject;
import java.io.File;
import java.net.URI;
/**
* JRuby extension, it's part of log4j wrapping for JRuby.
* Wrapper log4j Logger as Ruby like class
* */
@JRubyClass(name = "Logger")
public class LoggerExt extends RubyObject {

View file

@ -33,6 +33,10 @@ import java.io.IOException;
import java.io.InputStream;
import java.util.Properties;
/**
* Logstash configuration factory customization used to remove the routing appender from log4j configuration
* if "separate_logs" settings is false.
* */
@Plugin(name = "LogstashConfigurationFactory", category = ConfigurationFactory.CATEGORY)
@Order(9)
public class LogstashConfigurationFactory extends ConfigurationFactory {

View file

@ -29,6 +29,9 @@ import org.apache.logging.log4j.message.Message;
import java.util.List;
/**
* Logstash implementation of factory for Log4j LogEvent
* */
public class LogstashLogEventFactory implements LogEventFactory {
@Override
public LogEvent createEvent(String loggerName, Marker marker, String fqcn, Level level, Message data, List<Property> properties, Throwable t) {

View file

@ -28,6 +28,9 @@ import org.apache.logging.log4j.message.SimpleMessage;
import java.util.Map;
/**
* Used in Log4j configuration.
* */
public final class LogstashMessageFactory implements MessageFactory {
public static final LogstashMessageFactory INSTANCE = new LogstashMessageFactory();

View file

@ -34,6 +34,9 @@ import org.jruby.runtime.ThreadContext;
import org.jruby.runtime.builtin.IRubyObject;
import org.logstash.RubyUtil;
/**
* JRuby extension to provide slow logger functionality to Ruby classes
* */
@JRubyClass(name = "SlowLogger")
public class SlowLoggerExt extends RubyObject {

View file

@ -26,6 +26,9 @@ import org.apache.logging.log4j.message.Message;
import java.util.HashMap;
import java.util.Map;
/**
* Logging message extension class
* */
@JsonSerialize(using = CustomLogEventSerializer.class)
public class StructuredMessage implements Message {

View file

@ -28,6 +28,10 @@ import java.nio.charset.StandardCharsets;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
/**
* Expand the configuration variables used in pipeline configuration, bringing them from secret store or from the
* environment.
* */
public class ConfigVariableExpander implements AutoCloseable {
private static String SUBSTITUTION_PLACEHOLDER_REGEX = "\\$\\{(?<name>[a-zA-Z_.][a-zA-Z0-9_.]*)(:(?<default>[^}]*))?}";

View file

@ -29,6 +29,9 @@ import org.logstash.log.DefaultDeprecationLogger;
import java.io.Serializable;
import java.util.Map;
/**
* Default implementation of Logstash Plugin's {@link Context}
* */
public class ContextImpl implements Context {
private DeadLetterQueueWriter dlqWriter;

View file

@ -25,6 +25,9 @@ import org.jruby.runtime.ThreadContext;
import org.logstash.instrument.metrics.AbstractNamespacedMetricExt;
import org.logstash.instrument.metrics.counter.LongCounter;
/**
* Default implementation of {@link CounterMetric}
* */
public class CounterMetricImpl implements CounterMetric {
private LongCounter longCounter;

View file

@ -34,6 +34,9 @@ import java.util.List;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.CopyOnWriteArrayList;
/**
* JRuby extension class that collect all the emitters and hooks
* */
@JRubyClass(name = "HooksRegistry")
public final class HooksRegistryExt extends RubyObject {

View file

@ -106,6 +106,9 @@ public final class PluginLookup {
}
}
/**
* Descriptor of a plugin implementation. Defines the implementation language and the plugin class
* */
public interface PluginClass {
PluginLookup.PluginLanguage language();
@ -116,10 +119,16 @@ public final class PluginLookup {
}
}
/**
* Enum all the implementation languages used by plugins
* */
public enum PluginLanguage {
JAVA, RUBY
}
/**
* Enum all the plugins types used inside Logstash
* */
public enum PluginType {
INPUT("input", "inputs", Input.class),
FILTER("filter", "filters", Filter.class),

View file

@ -33,6 +33,9 @@ import java.util.Collection;
import java.util.List;
import java.util.stream.Collectors;
/**
* Plugin's utility methods
* */
public class PluginUtil {
private PluginUtil() { /* utility methods */ }

View file

@ -36,6 +36,9 @@ import java.util.Map;
import java.util.UUID;
import java.util.function.Consumer;
/**
* Java implementation of the "dots" codec
* */
@LogstashPlugin(name = "jdots")
public class Dots implements Codec {

View file

@ -46,6 +46,9 @@ import java.util.function.Consumer;
import static org.logstash.ObjectMappers.JSON_MAPPER;
/**
* Java implementation of the "line" codec
* */
@LogstashPlugin(name = "java_line")
public class Line implements Codec {

View file

@ -19,6 +19,9 @@ import org.logstash.plugins.ContextImpl;
import org.logstash.plugins.NamespacedMetricImpl;
import org.logstash.plugins.PluginLookup;
/**
* JRuby extension, factory class for plugins execution contexts
* */
@JRubyClass(name = "ExecutionContextFactory")
public final class ExecutionContextFactoryExt extends RubyBasicObject {

View file

@ -25,10 +25,16 @@ import org.logstash.plugins.PluginLookup;
import java.util.*;
import java.util.concurrent.ConcurrentHashMap;
/**
* JRuby extension to implement the factory that create plugin instances
* */
@JRubyClass(name = "PluginFactory")
public final class PluginFactoryExt extends RubyBasicObject
implements RubyIntegration.PluginFactory {
/**
* Definition of plugin resolver, maps plugin type and name to the plugin's class.
* */
@FunctionalInterface
public interface PluginResolver {
PluginLookup.PluginClass resolve(PluginLookup.PluginType type, String name);

View file

@ -13,6 +13,9 @@ import org.logstash.instrument.metrics.NullMetricExt;
import java.util.Arrays;
/**
* JRuby extension to implement a factory class for Plugin's metrics
* */
@JRubyClass(name = "PluginMetricsFactory")
public final class PluginMetricsFactoryExt extends RubyBasicObject {

View file

@ -33,6 +33,9 @@ import java.util.Arrays;
import java.util.Collection;
import java.util.UUID;
/**
* Java implementation of the "uuid" filter
* */
@LogstashPlugin(name = "java_uuid")
public class Uuid implements Filter {

View file

@ -42,6 +42,9 @@ import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import java.util.function.Consumer;
/**
* Java implementation of the "generator" input
* */
@LogstashPlugin(name = "java_generator")
public class Generator implements Input {

View file

@ -43,6 +43,9 @@ import java.util.Map;
import java.util.concurrent.CountDownLatch;
import java.util.function.Consumer;
/**
* Java implementation of the "stdin" input
* */
@LogstashPlugin(name = "java_stdin")
public class Stdin implements Input, Consumer<Map<String, Object>> {

View file

@ -30,6 +30,9 @@ import co.elastic.logstash.api.PluginHelper;
import java.util.Collection;
/**
* Java implementation of "sink" output plugin
* */
@LogstashPlugin(name = "sink")
public class Sink implements Output {

View file

@ -35,6 +35,9 @@ import java.util.Collection;
import java.util.Collections;
import java.util.concurrent.CountDownLatch;
/**
* Java implementation of the "stdout" output plugin
* */
@LogstashPlugin(name = "java_stdout")
public class Stdout implements Output {

View file

@ -24,6 +24,9 @@ import org.logstash.ext.JrubyEventExtLibrary;
import java.util.stream.Stream;
/**
* Represents the in endpoint of a pipeline to pipeline communication.
* */
public interface PipelineInput {
/**
* Accepts an event. It might be rejected if the input is stopping.

View file

@ -20,5 +20,8 @@
package org.logstash.plugins.pipeline;
/**
* Represents the out endpoint of a pipeline to pipeline communication.
* */
public interface PipelineOutput {
}

View file

@ -37,6 +37,9 @@ public class SecretStoreException extends RuntimeException {
super(message);
}
/**
* Error in retrive a secret from a keystore
* */
static public class RetrievalException extends SecretStoreException {
private static final long serialVersionUID = 1L;
public RetrievalException(SecretIdentifier secretIdentifier, Throwable cause) {
@ -44,6 +47,9 @@ public class SecretStoreException extends RuntimeException {
}
}
/**
* Error in listing secrets in key store
* */
static public class ListException extends SecretStoreException {
private static final long serialVersionUID = 1L;
public ListException(Throwable cause) {
@ -51,6 +57,9 @@ public class SecretStoreException extends RuntimeException {
}
}
/**
* Error in creating Logstash key store
* */
static public class CreateException extends SecretStoreException {
private static final long serialVersionUID = 1L;
public CreateException(String message, Throwable cause) {
@ -62,6 +71,9 @@ public class SecretStoreException extends RuntimeException {
}
}
/**
* Error in loading a key store, probably because it's not a Logstash keystore
* */
static public class LoadException extends SecretStoreException {
private static final long serialVersionUID = 1L;
public LoadException(String message, Throwable cause) {
@ -73,6 +85,9 @@ public class SecretStoreException extends RuntimeException {
}
}
/**
* Error in persisting a secret into a keystore
* */
static public class PersistException extends SecretStoreException {
private static final long serialVersionUID = 1L;
public PersistException(SecretIdentifier secretIdentifier, Throwable cause) {
@ -80,6 +95,9 @@ public class SecretStoreException extends RuntimeException {
}
}
/**
* Error in purge a secret from a keystore
* */
static public class PurgeException extends SecretStoreException {
private static final long serialVersionUID = 1L;
public PurgeException(SecretIdentifier secretIdentifier, Throwable cause) {
@ -87,6 +105,9 @@ public class SecretStoreException extends RuntimeException {
}
}
/**
* Generic error
* */
static public class UnknownException extends SecretStoreException {
private static final long serialVersionUID = 1L;
public UnknownException(String message, Throwable cause) {
@ -94,6 +115,9 @@ public class SecretStoreException extends RuntimeException {
}
}
/**
* Problem with "keystore.classname", can't be instantiated or class can't be found
* */
static public class ImplementationNotFoundException extends SecretStoreException {
private static final long serialVersionUID = 1L;
public ImplementationNotFoundException(String message, Throwable throwable) {
@ -101,6 +125,9 @@ public class SecretStoreException extends RuntimeException {
}
}
/**
* Problem with the value of "keystore.classname", class is invalid or not accessible
* */
static public class ImplementationInvalidException extends SecretStoreException {
private static final long serialVersionUID = 1L;
public ImplementationInvalidException(String message, Throwable throwable) {
@ -108,6 +135,9 @@ public class SecretStoreException extends RuntimeException {
}
}
/**
* Launched when there is problem accessing a keystore, for example the user provided empty password
* */
static public class AccessException extends SecretStoreException {
private static final long serialVersionUID = 1L;
public AccessException(String message, Throwable throwable) {
@ -119,6 +149,9 @@ public class SecretStoreException extends RuntimeException {
}
}
/**
* Launched when a keystore already exists
* */
static public class AlreadyExistsException extends SecretStoreException {
private static final long serialVersionUID = 1L;
public AlreadyExistsException(String message) {
@ -126,6 +159,9 @@ public class SecretStoreException extends RuntimeException {
}
}
/**
* Never used
* */
static public class InvalidConfigurationException extends SecretStoreException {
private static final long serialVersionUID = 1L;
public InvalidConfigurationException(String message) {

View file

@ -24,6 +24,9 @@ import org.jruby.RubyHash;
import org.logstash.RubyUtil;
import org.logstash.secret.SecretIdentifier;
/**
* Collect static methods to access Java keystore
* */
public class SecretStoreExt {
private static final SecretStoreFactory SECRET_STORE_FACTORY = SecretStoreFactory.fromEnvironment();

View file

@ -22,6 +22,9 @@ package org.logstash.util;
import co.elastic.logstash.api.Password;
import org.logstash.RubyUtil;
/**
* Represents and decode credentials to access Elastic cloud instance.
* */
public class CloudSettingAuth {
private String original;

View file

@ -38,6 +38,9 @@ import java.util.Base64;
* The third segment is the kibana host identifier, e.g. 'identifier'
* The 'cloud.id' value decoded into the various fields.
*/
/**
* Represents and decode an Elastic cloudid of an instance.
* */
public class CloudSettingId {
private static class HostAndPort {

View file

@ -29,6 +29,10 @@ import java.util.Map;
import java.util.TreeMap;
import java.util.stream.Collectors;
/**
* Wrapper of Logstash modules settings, with ability to replace password fields with
* the obfuscator {@link Password} implementation.
* */
public final class ModulesSettingArray extends ArrayList<Map<String, Object>> {
private static final long serialVersionUID = 4094949366274116593L;

View file

@ -23,6 +23,9 @@ import org.logstash.RubyUtil;
import java.util.Objects;
import java.util.concurrent.TimeUnit;
/**
* Express a period of time, expressing in numerical quantity and time unit, for example "3 seconds"
* */
public class TimeValue {
public static TimeValue fromValue(Object value) {

View file

@ -26,6 +26,9 @@ import org.jruby.anno.JRubyModule;
import org.jruby.runtime.ThreadContext;
import org.jruby.runtime.builtin.IRubyObject;
/**
* Utility methods class
* */
@JRubyModule(name = "Util") // LogStash::Util
public class UtilExt {