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

IGNITE-15568 Support batching in striped disruptor #3588

Open
wants to merge 2 commits into
base: main
Choose a base branch
from
Open
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -27,13 +27,18 @@
import com.lmax.disruptor.dsl.Disruptor;
import com.lmax.disruptor.dsl.ProducerType;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ThreadFactory;
import java.util.function.BiConsumer;
import java.util.function.BiFunction;
import org.apache.ignite.internal.logger.IgniteLogger;
import org.apache.ignite.internal.logger.Loggers;
import org.apache.ignite.internal.thread.NamedThreadFactory;
import org.apache.ignite.internal.util.CollectionUtils;
import org.apache.ignite.raft.jraft.entity.NodeId;

/**
Expand Down Expand Up @@ -237,11 +242,16 @@ public RingBuffer<T> queue(NodeId nodeId) {
private class StripeEntryHandler implements EventHandler<T> {
private final ConcurrentHashMap<NodeId, EventHandler<T>> subscribers;

/** The cache is used to correct handling the disruptor batch. */
private final Map<NodeId, T> eventCache;

/**
* The constructor.
*/
StripeEntryHandler() {
subscribers = new ConcurrentHashMap<>();

eventCache = supportsBatches ? null : new HashMap<>();
}

/**
Expand All @@ -265,13 +275,38 @@ void unsubscribe(NodeId nodeId) {

/** {@inheritDoc} */
@Override public void onEvent(T event, long sequence, boolean endOfBatch) throws Exception {
EventHandler<T> handler = subscribers.get(event.nodeId());

// TODO: IGNITE-20536 Need to add assert that handler is not null and to implement a no-op handler.
if (handler != null) {
handler.onEvent(event, sequence, endOfBatch || subscribers.size() > 1 && !supportsBatches);
if (supportsBatches || subscribers.size() <= 1) {
EventHandler<T> handler = subscribers.get(event.nodeId());

// TODO: IGNITE-20536 Need to add assert that handler is not null and to implement a no-op handler.
if (handler != null) {
handler.onEvent(event, sequence, endOfBatch);
} else {
LOG.warn(format("Group of the event is unsupported [nodeId={}, event={}, endOfBatch={}]",
event.nodeId(), event, endOfBatch));
}
} else {
LOG.warn(format("Group of the event is unsupported [nodeId={}, event={}]", event.nodeId(), event));
T prevEvent = eventCache.put(event.nodeId(), event);

if (prevEvent != null) {
EventHandler<T> grpHandler = subscribers.get(event.nodeId());

if (grpHandler != null) {
grpHandler.onEvent(prevEvent, sequence, false);
}
}

if (endOfBatch) {
for (Map.Entry<NodeId, T> grpEvent : eventCache.entrySet()) {
EventHandler<T> grpHandler = subscribers.get(grpEvent.getKey());

if (grpHandler != null) {
grpHandler.onEvent(grpEvent.getValue(), sequence, true);
}
}

eventCache.clear();
}
}
}
}
Expand Down