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

[HUDI-5488] Make sure Disrupt queue start first, then insert records #7582

Merged
merged 4 commits into from
Jan 19, 2023
Merged
Show file tree
Hide file tree
Changes from 3 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 @@ -325,7 +325,7 @@ public Integer finish() {
};

DisruptorExecutor<HoodieRecord, Tuple2<HoodieRecord, Option<IndexedRecord>>, Integer> exec = new DisruptorExecutor(Option.of(1024),
producers, Option.of(consumer), getCloningTransformer(HoodieTestDataGenerator.AVRO_SCHEMA),
producers, consumer, getCloningTransformer(HoodieTestDataGenerator.AVRO_SCHEMA),
Option.of(WaitStrategyFactory.DEFAULT_STRATEGY), getPreExecuteRunnable());

final Throwable thrown = assertThrows(HoodieException.class, exec::execute,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -93,6 +93,8 @@ protected void doProduce(HoodieMessageQueue<I, O> queue, HoodieProducer<I> produ

protected abstract void doConsume(HoodieMessageQueue<I, O> queue, HoodieConsumer<O, E> consumer);

protected void setUp() {}

/**
* Start producing
*/
Expand Down Expand Up @@ -165,6 +167,7 @@ public boolean isRunning() {
public E execute() {
try {
checkState(this.consumer.isPresent());
setUp();
// Start consuming/producing asynchronously
CompletableFuture<Void> consuming = startConsumingAsync();
CompletableFuture<Void> producing = startProducingAsync();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,21 +34,24 @@ public class DisruptorExecutor<I, O, E> extends BaseHoodieQueueBasedExecutor<I,

public DisruptorExecutor(final Option<Integer> bufferSize, final Iterator<I> inputItr,
HoodieConsumer<O, E> consumer, Function<I, O> transformFunction, Option<String> waitStrategy, Runnable preExecuteRunnable) {
this(bufferSize, Collections.singletonList(new IteratorBasedQueueProducer<>(inputItr)), Option.of(consumer),
this(bufferSize, Collections.singletonList(new IteratorBasedQueueProducer<>(inputItr)), consumer,
transformFunction, waitStrategy, preExecuteRunnable);
}

public DisruptorExecutor(final Option<Integer> bufferSize, List<HoodieProducer<I>> producers,
Option<HoodieConsumer<O, E>> consumer, final Function<I, O> transformFunction,
HoodieConsumer<O, E> consumer, final Function<I, O> transformFunction,
final Option<String> waitStrategy, Runnable preExecuteRunnable) {
super(producers, consumer, new DisruptorMessageQueue<>(bufferSize, transformFunction, waitStrategy, producers.size(), preExecuteRunnable), preExecuteRunnable);
super(producers, Option.of(consumer), new DisruptorMessageQueue<>(bufferSize, transformFunction, waitStrategy, producers.size(), preExecuteRunnable), preExecuteRunnable);
}

@Override
protected void doConsume(HoodieMessageQueue<I, O> queue, HoodieConsumer<O, E> consumer) {
protected void setUp() {
DisruptorMessageQueue<I, O> disruptorQueue = (DisruptorMessageQueue<I, O>) queue;
// Before we start producing, we need to set up Disruptor's queue
disruptorQueue.setHandlers(consumer);
disruptorQueue.setHandlers(consumer.get());
disruptorQueue.start();
}

@Override
protected void doConsume(HoodieMessageQueue<I, O> queue, HoodieConsumer<O, E> consumer) {}
}
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@ public class DisruptorMessageQueue<I, O> implements HoodieMessageQueue<I, O> {
private final RingBuffer<HoodieDisruptorEvent> ringBuffer;

private boolean isShutdown = false;
private boolean isStart = false;

public DisruptorMessageQueue(Option<Integer> bufferSize, Function<I, O> transformFunction, Option<String> waitStrategyName, int totalProducers, Runnable preExecuteRunnable) {
WaitStrategy waitStrategy = WaitStrategyFactory.build(waitStrategyName);
Expand All @@ -60,6 +61,10 @@ public long size() {

@Override
public void insertRecord(I value) throws Exception {
if (!isStart) {
Copy link
Contributor

Choose a reason for hiding this comment

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

I don't think we need this, it would throw anyway if the queue has not been started yet

Copy link
Contributor Author

Choose a reason for hiding this comment

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

We assert in insertRecord that the queue is started

Here just assert the queue is started, in case others use this queue without calling the start

Copy link
Contributor

Choose a reason for hiding this comment

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

@boneanxs we need to keep in mind that this lies on the hot-path of being executed for every record, so we should keep the bar really high in terms of what we allow to be executed for every record.

That being said, i also see your point of adding it to make sure it would fail definitively unless queue has been started

Copy link
Contributor

Choose a reason for hiding this comment

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

Let's keep it, please rename it to isStarted

throw new HoodieException("Can't insert into the queue since the queue is not started yet");
}

if (isShutdown) {
throw new HoodieException("Can't insert into the queue after it had already been closed");
}
Expand Down Expand Up @@ -92,6 +97,7 @@ public void close() {
synchronized (this) {
if (!isShutdown) {
isShutdown = true;
isStart = false;
queue.shutdown();
}
}
Expand All @@ -104,7 +110,12 @@ protected void setHandlers(HoodieConsumer consumer) {
}

protected void start() {
queue.start();
synchronized (this) {
if (!isStart) {
queue.start();
isStart = true;
}
}
}

/**
Expand Down