Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

feature: #242 Explicit terminal and retry exceptions for cleaner logging and poison pills #291

Closed
wants to merge 23 commits into from
Closed
Show file tree
Hide file tree
Changes from 22 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,38 @@
package io.confluent.parallelconsumer;

/*-
* Copyright (C) 2020-2022 Confluent, Inc.
*/

import lombok.ToString;
import lombok.experimental.Delegate;

import java.util.Arrays;
import java.util.List;
import java.util.stream.Collectors;

@ToString
public class Offsets {

@Delegate
private final List<Long> rawOffsets;

public Offsets(List<Long> records) {
this.rawOffsets = records;
}

public static Offsets from(List<RecordContext<?, ?>> records) {
return of(records.stream()
.map(RecordContext::offset)
.collect(Collectors.toUnmodifiableList()));
}

// due to type erasure, can't use method overloading
public static Offsets of(List<Long> rawOffsetsIn) {
return new Offsets(rawOffsetsIn);
}

public static Offsets of(long... rawOffsetsIn) {
return new Offsets(Arrays.stream(rawOffsetsIn).boxed().collect(Collectors.toList()));
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
package io.confluent.parallelconsumer;

/*-
* Copyright (C) 2020-2022 Confluent, Inc.
*/

import lombok.ToString;

/**
* A user's processing function can throw this exception, which signals to PC that processing of the message has failed,
* and that it should be retired at a later time.
* <p>
* The advantage of throwing this exception explicitly, is that PC will not log an ERROR. If any other type of exception
* is thrown by the user's function, that will be logged as an error (but will still be retried later).
* <p>
* So in short, if this exception is thrown, nothing will be logged (except at DEBUG level), any other exception will be
* logged as an error.
*/
@ToString
public class PCRetriableException extends PCUserException {

public PCRetriableException(String message) {
super(message);
}

public PCRetriableException(String message, Throwable cause) {
super(message, cause);
}

public PCRetriableException(Throwable cause) {
super(cause);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,29 @@
package io.confluent.parallelconsumer;

/*-
* Copyright (C) 2020-2022 Confluent, Inc.
*/

/**
* A user's processing function can throw this exception, which signals to PC that processing of the message has failed,
* and that it should be retired at a later time.
* <p>
* The advantage of throwing this exception explicitly, is that PC will not log an ERROR. If any other type of exception
* is thrown by the user's function, that will be logged as an error (but will still be retried later).
* <p>
* So in short, if this exception is thrown, nothing will be logged (except at DEBUG level), any other exception will be
* logged as an error.
*/
Comment on lines +7 to +16
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

update java doc

public class PCTerminalException extends PCUserException {
public PCTerminalException(String message) {
super(message);
}

public PCTerminalException(String message, Throwable cause) {
super(message, cause);
}

public PCTerminalException(Throwable cause) {
super(cause);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,22 @@
package io.confluent.parallelconsumer;

/*-
* Copyright (C) 2020-2022 Confluent, Inc.
*/

/**
* todo
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

docs

*/
public class PCUserException extends RuntimeException {
public PCUserException(String message) {
super(message);
}

public PCUserException(String message, Throwable cause) {
super(message, cause);
}

public PCUserException(Throwable cause) {
super(cause);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -223,6 +223,14 @@ public boolean isUsingBatching() {
@Builder.Default
private final int maxFailureHistory = 10;

private final TerminalFailureReaction terminalFailureReaction;

public enum TerminalFailureReaction {
SHUTDOWN,
SKIP,
// DLQ, TODO
}

/**
* @return the combined target of the desired concurrency by the configured batch size
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,6 @@
import lombok.*;
import lombok.extern.slf4j.Slf4j;
import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.clients.consumer.MockConsumer;
import org.apache.kafka.clients.consumer.internals.ConsumerCoordinator;
Expand Down Expand Up @@ -53,13 +52,12 @@
public abstract class AbstractParallelEoSStreamProcessor<K, V> implements ParallelConsumer<K, V>, ConsumerRebalanceListener, Closeable {

public static final String MDC_INSTANCE_ID = "pcId";
public static final String MDC_OFFSET_MARKER = "offset";

/**
* Key for the work container descriptor that will be added to the {@link MDC diagnostic context} while inside a
* user function.
*/
private static final String MDC_WORK_CONTAINER_DESCRIPTOR = "offset";
static final String MDC_WORK_CONTAINER_DESCRIPTOR = "offset";

@Getter(PROTECTED)
protected final ParallelConsumerOptions options;
Expand Down Expand Up @@ -765,14 +763,16 @@ protected <R> void submitWorkToPool(Function<PollContextInternal<K, V>, List<R>>
}
}


private <R> void submitWorkToPoolInner(final Function<PollContextInternal<K, V>, List<R>> usersFunction,
final Consumer<R> callback,
final List<WorkContainer<K, V>> batch) {
// for each record, construct dispatch to the executor and capture a Future
log.trace("Sending work ({}) to pool", batch);
Future outputRecordFuture = workerThreadPool.submit(() -> {
addInstanceMDC();
return runUserFunction(usersFunction, callback, batch);
UserFunctionRunner<K, V> runner = new UserFunctionRunner<>(this);
return runner.runUserFunction(usersFunction, callback, batch);
});
// for a batch, each message in the batch shares the same result
for (final WorkContainer<K, V> workContainer : batch) {
Expand Down Expand Up @@ -1087,61 +1087,6 @@ private void updateLastCommitCheckTime() {
lastCommitCheckTime = Instant.now();
}

/**
* Run the supplied function.
*/
protected <R> List<ParallelConsumer.Tuple<ConsumerRecord<K, V>, R>> runUserFunction(Function<PollContextInternal<K, V>, List<R>> usersFunction,
Consumer<R> callback,
List<WorkContainer<K, V>> workContainerBatch) {
// call the user's function
List<R> resultsFromUserFunction;
try {
if (log.isDebugEnabled()) {
// first offset of the batch
MDC.put(MDC_WORK_CONTAINER_DESCRIPTOR, workContainerBatch.get(0).offset() + "");
}
log.trace("Pool received: {}", workContainerBatch);

//
boolean workIsStale = wm.checkIfWorkIsStale(workContainerBatch);
if (workIsStale) {
// when epoch's change, we can't remove them from the executor pool queue, so we just have to skip them when we find them
log.debug("Pool found work from old generation of assigned work, skipping message as epoch doesn't match current {}", workContainerBatch);
return null;
}

PollContextInternal<K, V> context = new PollContextInternal<>(workContainerBatch);
resultsFromUserFunction = usersFunction.apply(context);

for (final WorkContainer<K, V> kvWorkContainer : workContainerBatch) {
onUserFunctionSuccess(kvWorkContainer, resultsFromUserFunction);
}

// capture each result, against the input record
var intermediateResults = new ArrayList<Tuple<ConsumerRecord<K, V>, R>>();
for (R result : resultsFromUserFunction) {
log.trace("Running users call back...");
callback.accept(result);
}

// fail or succeed, either way we're done
for (var kvWorkContainer : workContainerBatch) {
addToMailBoxOnUserFunctionSuccess(kvWorkContainer, resultsFromUserFunction);
}
log.trace("User function future registered");

return intermediateResults;
} catch (Exception e) {
// handle fail
log.error("Exception caught in user function running stage, registering WC as failed, returning to mailbox", e);
for (var wc : workContainerBatch) {
wc.onUserFunctionFailure(e);
addToMailbox(wc); // always add on error
}
throw e; // trow again to make the future failed
}
}

protected void addToMailBoxOnUserFunctionSuccess(WorkContainer<K, V> wc, List<?> resultsFromUserFunction) {
addToMailbox(wc);
}
Expand Down
Loading