Skip to content

Commit 53cc2d6

Browse files
committed
Zero Buffer implementation work
Signed-off-by: Souvik Bose <souvik04in@gmail.com>
1 parent 9f17acb commit 53cc2d6

13 files changed

Lines changed: 603 additions & 192 deletions

File tree

data-prepper-api/src/main/java/org/opensearch/dataprepper/model/buffer/Buffer.java

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -122,4 +122,8 @@ default boolean isWrittenOffHeapOnly() {
122122
*/
123123
default void shutdown() {
124124
}
125+
126+
// Zero Buffer functionality
127+
default boolean isZeroBuffer() { return false; }
128+
125129
}

data-prepper-api/src/test/java/org/opensearch/dataprepper/model/buffer/BufferTest.java

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -65,4 +65,10 @@ void testWriteBytes() {
6565

6666
}
6767

68+
@Test
69+
void isZeroBuffer() {
70+
assertThat(createObjectUnderTest().isZeroBuffer(), equalTo(false));
71+
}
72+
73+
6874
}

data-prepper-core/src/main/java/org/opensearch/dataprepper/parser/PipelineTransformer.java

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -29,6 +29,7 @@
2929
import org.opensearch.dataprepper.pipeline.parser.model.SinkContextPluginSetting;
3030
import org.opensearch.dataprepper.pipeline.router.Router;
3131
import org.opensearch.dataprepper.pipeline.router.RouterFactory;
32+
import org.opensearch.dataprepper.pipeline.zerobuffer.AbstractZeroBuffer;
3233
import org.opensearch.dataprepper.sourcecoordination.SourceCoordinatorFactory;
3334
import org.slf4j.Logger;
3435
import org.slf4j.LoggerFactory;
@@ -156,6 +157,11 @@ private void buildPipelineFromConfiguration(
156157
eventFactory, acknowledgementSetManager, sourceCoordinatorFactory, processorThreads, readBatchDelay,
157158
dataPrepperConfiguration.getProcessorShutdownTimeout(), dataPrepperConfiguration.getSinkShutdownTimeout(),
158159
getPeerForwarderDrainTimeout(dataPrepperConfiguration));
160+
161+
if (buffer.isZeroBuffer() && pipelineDefinedBuffer instanceof AbstractZeroBuffer) {
162+
((AbstractZeroBuffer<?>) pipelineDefinedBuffer).setPipeline(pipeline);
163+
}
164+
159165
pipelineMap.put(pipelineName, pipeline);
160166
} catch (Exception ex) {
161167
//If pipeline construction errors out, we will skip that pipeline and proceed

data-prepper-core/src/main/java/org/opensearch/dataprepper/pipeline/Pipeline.java

Lines changed: 90 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -7,9 +7,14 @@
77

88
import com.google.common.base.Preconditions;
99
import org.opensearch.dataprepper.acknowledgements.InactiveAcknowledgementSetManager;
10+
import org.opensearch.dataprepper.model.CheckpointState;
1011
import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager;
1112
import org.opensearch.dataprepper.model.buffer.Buffer;
13+
import org.opensearch.dataprepper.model.event.DefaultEventHandle;
14+
import org.opensearch.dataprepper.model.event.Event;
1215
import org.opensearch.dataprepper.model.event.EventFactory;
16+
import org.opensearch.dataprepper.model.event.EventHandle;
17+
import org.opensearch.dataprepper.model.event.InternalEventHandle;
1318
import org.opensearch.dataprepper.model.processor.Processor;
1419
import org.opensearch.dataprepper.model.record.Record;
1520
import org.opensearch.dataprepper.model.sink.Sink;
@@ -21,8 +26,11 @@
2126
import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourcePartition;
2227
import org.opensearch.dataprepper.model.source.coordinator.enhanced.UsesEnhancedSourceCoordination;
2328
import org.opensearch.dataprepper.parser.DataFlowComponent;
29+
import org.opensearch.dataprepper.pipeline.common.FutureHelper;
30+
import org.opensearch.dataprepper.pipeline.common.FutureHelperResult;
2431
import org.opensearch.dataprepper.pipeline.common.PipelineThreadFactory;
2532
import org.opensearch.dataprepper.pipeline.common.PipelineThreadPoolExecutor;
33+
import org.opensearch.dataprepper.pipeline.exceptions.InvalidEventHandleException;
2634
import org.opensearch.dataprepper.pipeline.router.Router;
2735
import org.opensearch.dataprepper.pipeline.router.RouterCopyRecordStrategy;
2836
import org.opensearch.dataprepper.pipeline.router.RouterGetRecordStrategy;
@@ -37,7 +45,9 @@
3745
import java.util.Collections;
3846
import java.util.LinkedList;
3947
import java.util.List;
48+
import java.util.Map;
4049
import java.util.Objects;
50+
import java.util.Set;
4151
import java.util.concurrent.ExecutorService;
4252
import java.util.concurrent.Future;
4353
import java.util.concurrent.TimeUnit;
@@ -54,7 +64,9 @@
5464
@SuppressWarnings({"rawtypes", "unchecked"})
5565
public class Pipeline {
5666
private static final Logger LOG = LoggerFactory.getLogger(Pipeline.class);
67+
private final boolean acknowledgementsEnabled;
5768
private volatile AtomicBoolean stopRequested;
69+
private boolean isEmptyRecordsLogged = false;
5870

5971
private final String name;
6072
private final Source source;
@@ -137,6 +149,7 @@ public Pipeline(
137149
new PipelineThreadFactory(format("%s-sink-worker", name)), this);
138150

139151
stopRequested = new AtomicBoolean(false);
152+
this.acknowledgementsEnabled = source.areAcknowledgementsEnabled() || buffer.areAcknowledgementsEnabled();
140153
}
141154

142155
AcknowledgementSetManager getAcknowledgementSetManager() {
@@ -310,6 +323,83 @@ public void removeShutdownObserver(final PipelineObserver pipelineObserver) {
310323
observers.remove(pipelineObserver);
311324
}
312325

326+
public void runProcessorsAndSinks(final List<Processor> processors) {
327+
final Map.Entry<Collection, CheckpointState> readResult = buffer.read(getReadBatchTimeoutInMillis());
328+
Collection records = readResult.getKey();
329+
final CheckpointState checkpointState = readResult.getValue();
330+
//TODO Hacky way to avoid logging continuously - Will be removed as part of metrics implementation
331+
if (records.isEmpty()) {
332+
if(!isEmptyRecordsLogged) {
333+
LOG.debug(" {} Worker: No records received from buffer", getName());
334+
isEmptyRecordsLogged = true;
335+
}
336+
} else {
337+
LOG.debug(" {} Worker: Processing {} records from buffer", getName(), records.size());
338+
}
339+
340+
//Should Empty list from buffer should be sent to the processors? For now sending as the Stateful processors expects it.
341+
for (final Processor processor : processors) {
342+
343+
List<Event> inputEvents = null;
344+
if (acknowledgementsEnabled) {
345+
inputEvents = ((List<Record<Event>>) records).stream().map(Record::getData).collect(Collectors.toList());
346+
}
347+
348+
try {
349+
records = processor.execute(records);
350+
if (inputEvents != null) {
351+
processAcknowledgements(inputEvents, records);
352+
}
353+
} catch (final Exception e) {
354+
LOG.error("A processor threw an exception. This batch of Events will be dropped, and their EventHandles will be released: ", e);
355+
if (inputEvents != null) {
356+
processAcknowledgements(inputEvents, Collections.emptyList());
357+
}
358+
359+
records = Collections.emptyList();
360+
break;
361+
}
362+
}
363+
364+
postToSink(records);
365+
// Checkpoint the current batch read from the buffer after being processed by processors and sinks.
366+
buffer.checkpoint(checkpointState);
367+
368+
}
369+
370+
public void executeAllProcessorsAndSinks() {
371+
List<Processor> processors = this.processorSets.stream().flatMap(Collection::stream).collect(Collectors.toList());
372+
runProcessorsAndSinks(processors);
373+
}
374+
375+
private void processAcknowledgements(List<Event> inputEvents, Collection<Record<Event>> outputRecords) {
376+
Set<Event> outputEventsSet = outputRecords.stream().map(Record::getData).collect(Collectors.toSet());
377+
// For each event in the input events list that is not present in the output events, send positive acknowledgement, if acknowledgements are enabled for it
378+
inputEvents.forEach(event -> {
379+
EventHandle eventHandle = event.getEventHandle();
380+
if (eventHandle != null && eventHandle instanceof DefaultEventHandle) {
381+
InternalEventHandle internalEventHandle = (InternalEventHandle)(DefaultEventHandle)eventHandle;
382+
if (internalEventHandle.getAcknowledgementSet() != null && !outputEventsSet.contains(event)) {
383+
eventHandle.release(true);
384+
}
385+
} else if (eventHandle != null) {
386+
throw new InvalidEventHandleException("Unexpected EventHandle");
387+
}
388+
});
389+
}
390+
391+
/**
392+
* TODO Add isolator pattern - Fail if one of the Sink fails [isolator Pattern]
393+
* Uses the pipeline method to publish to sinks, waits for each of the sink result to be true before attempting to
394+
* process more records from buffer.
395+
*/
396+
private boolean postToSink(final Collection<Record> records) {
397+
LOG.debug("Pipeline Worker: Submitting {} processed records to sinks", records.size());
398+
final List<Future<Void>> sinkFutures = publishToSinks(records);
399+
final FutureHelperResult<Void> futureResults = FutureHelper.awaitFuturesIndefinitely(sinkFutures);
400+
return futureResults.getFailedReasons().isEmpty();
401+
}
402+
313403
private void shutdownExecutorService(final ExecutorService executorService, final long timeoutForTerminationInMillis, final String workerName) {
314404
LOG.info("Pipeline [{}] - Shutting down {} process workers.", name, workerName);
315405

data-prepper-core/src/main/java/org/opensearch/dataprepper/pipeline/ProcessWorker.java

Lines changed: 6 additions & 82 deletions
Original file line numberDiff line numberDiff line change
@@ -7,26 +7,13 @@
77

88
import io.micrometer.core.instrument.Counter;
99
import org.opensearch.dataprepper.metrics.PluginMetrics;
10-
import org.opensearch.dataprepper.model.CheckpointState;
1110
import org.opensearch.dataprepper.model.buffer.Buffer;
12-
import org.opensearch.dataprepper.model.event.DefaultEventHandle;
13-
import org.opensearch.dataprepper.model.event.Event;
14-
import org.opensearch.dataprepper.model.event.EventHandle;
15-
import org.opensearch.dataprepper.model.event.InternalEventHandle;
1611
import org.opensearch.dataprepper.model.processor.Processor;
17-
import org.opensearch.dataprepper.model.record.Record;
18-
import org.opensearch.dataprepper.pipeline.common.FutureHelper;
19-
import org.opensearch.dataprepper.pipeline.common.FutureHelperResult;
12+
import org.opensearch.dataprepper.pipeline.exceptions.InvalidEventHandleException;
2013
import org.slf4j.Logger;
2114
import org.slf4j.LoggerFactory;
2215

23-
import java.util.Collection;
24-
import java.util.Collections;
2516
import java.util.List;
26-
import java.util.Map;
27-
import java.util.Set;
28-
import java.util.concurrent.Future;
29-
import java.util.stream.Collectors;
3017

3118
@SuppressWarnings({"rawtypes", "unchecked"})
3219
public class ProcessWorker implements Runnable {
@@ -36,7 +23,6 @@ public class ProcessWorker implements Runnable {
3623
private final Buffer readBuffer;
3724
private final List<Processor> processors;
3825
private final Pipeline pipeline;
39-
private boolean isEmptyRecordsLogged = false;
4026
private PluginMetrics pluginMetrics;
4127
private final Counter invalidEventHandlesCounter;
4228
private boolean acknowledgementsEnabled;
@@ -93,80 +79,18 @@ public void run() {
9379
}
9480
}
9581

96-
private void processAcknowledgements(List<Event> inputEvents, Collection<Record<Event>> outputRecords) {
97-
Set<Event> outputEventsSet = outputRecords.stream().map(Record::getData).collect(Collectors.toSet());
98-
// For each event in the input events list that is not present in the output events, send positive acknowledgement, if acknowledgements are enabled for it
99-
inputEvents.forEach(event -> {
100-
EventHandle eventHandle = event.getEventHandle();
101-
if (eventHandle != null && eventHandle instanceof DefaultEventHandle) {
102-
InternalEventHandle internalEventHandle = (InternalEventHandle)(DefaultEventHandle)eventHandle;
103-
if (internalEventHandle.getAcknowledgementSet() != null && !outputEventsSet.contains(event)) {
104-
eventHandle.release(true);
105-
}
106-
} else if (eventHandle != null) {
107-
invalidEventHandlesCounter.increment();
108-
throw new RuntimeException("Unexpected EventHandle");
109-
}
110-
});
111-
}
112-
11382
private void doRun() {
114-
final Map.Entry<Collection, CheckpointState> readResult = readBuffer.read(pipeline.getReadBatchTimeoutInMillis());
115-
Collection records = readResult.getKey();
116-
final CheckpointState checkpointState = readResult.getValue();
117-
//TODO Hacky way to avoid logging continuously - Will be removed as part of metrics implementation
118-
if (records.isEmpty()) {
119-
if(!isEmptyRecordsLogged) {
120-
LOG.debug(" {} Worker: No records received from buffer", pipeline.getName());
121-
isEmptyRecordsLogged = true;
122-
}
123-
} else {
124-
LOG.debug(" {} Worker: Processing {} records from buffer", pipeline.getName(), records.size());
125-
}
126-
//Should Empty list from buffer should be sent to the processors? For now sending as the Stateful processors expects it.
127-
for (final Processor processor : processors) {
128-
129-
List<Event> inputEvents = null;
130-
if (acknowledgementsEnabled) {
131-
inputEvents = ((List<Record<Event>>) records).stream().map(Record::getData).collect(Collectors.toList());
132-
}
133-
134-
try {
135-
records = processor.execute(records);
136-
if (inputEvents != null) {
137-
processAcknowledgements(inputEvents, records);
138-
}
139-
} catch (final Exception e) {
140-
LOG.error("A processor threw an exception. This batch of Events will be dropped, and their EventHandles will be released: ", e);
141-
if (inputEvents != null) {
142-
processAcknowledgements(inputEvents, Collections.emptyList());
143-
}
144-
145-
records = Collections.emptyList();
146-
break;
147-
}
83+
try {
84+
pipeline.runProcessorsAndSinks(processors);
85+
} catch (InvalidEventHandleException ex) {
86+
invalidEventHandlesCounter.increment();
87+
throw ex;
14888
}
149-
150-
postToSink(records);
151-
// Checkpoint the current batch read from the buffer after being processed by processors and sinks.
152-
readBuffer.checkpoint(checkpointState);
15389
}
15490

15591
private boolean areComponentsReadyForShutdown() {
15692
return readBuffer.isEmpty() && processors.stream()
15793
.map(Processor::isReadyForShutdown)
15894
.allMatch(result -> result == true);
15995
}
160-
161-
/**
162-
* TODO Add isolator pattern - Fail if one of the Sink fails [isolator Pattern]
163-
* Uses the pipeline method to publish to sinks, waits for each of the sink result to be true before attempting to
164-
* process more records from buffer.
165-
*/
166-
private boolean postToSink(final Collection<Record> records) {
167-
LOG.debug("Pipeline Worker: Submitting {} processed records to sinks", records.size());
168-
final List<Future<Void>> sinkFutures = pipeline.publishToSinks(records);
169-
final FutureHelperResult<Void> futureResults = FutureHelper.awaitFuturesIndefinitely(sinkFutures);
170-
return futureResults.getFailedReasons().size() == 0;
171-
}
17296
}
Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,7 @@
1+
package org.opensearch.dataprepper.pipeline.exceptions;
2+
3+
public class InvalidEventHandleException extends RuntimeException {
4+
public InvalidEventHandleException(final String message) {
5+
super(message);
6+
}
7+
}
Lines changed: 22 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,22 @@
1+
package org.opensearch.dataprepper.pipeline.zerobuffer;
2+
3+
import org.opensearch.dataprepper.model.buffer.Buffer;
4+
import org.opensearch.dataprepper.model.record.Record;
5+
import org.opensearch.dataprepper.pipeline.Pipeline;
6+
7+
public abstract class AbstractZeroBuffer <T extends Record<?>> implements Buffer<T> {
8+
private Pipeline pipeline;
9+
10+
public void setPipeline(Pipeline pipeline) {
11+
this.pipeline = pipeline;
12+
}
13+
14+
public Pipeline getPipeline() {
15+
return pipeline;
16+
}
17+
18+
@Override
19+
public boolean isZeroBuffer() {
20+
return true;
21+
}
22+
}

0 commit comments

Comments
 (0)