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

Conversation

boneanxs
Copy link
Contributor

Change Logs

Describe context and summary for this change. Highlight if any code was copied.
We must to make sure to set up Disruptor's queue first, then producer can insert records to the queue. But currently we have no idea which thread start first, so this pr tries to fix it.

CompletableFuture<Void> consuming = startConsumingAsync();
CompletableFuture<Void> producing = startProducingAsync();

Also, I think the test TestDisruptorExecutionInSpark#testExecutor and TestDisruptorMessageQueue#testRecordReading failures relate to this bug.

Screenshot 2022-12-29 at 10 07 21

Impact

Describe any public API or user-facing feature change or any performance impact.
none

Risk level (write none, low medium or high below)

If medium or high, explain what verification was done to mitigate the risks.

none

Documentation Update

Describe any necessary documentation update if there is any new feature, config, or user-facing change

  • The config description must be updated if new configs are added or the default value of the configs are changed
  • Any new feature or user-facing change requires updating the Hudi website. Please create a Jira ticket, attach the
    ticket number here and follow the instruction to make
    changes to the website.

Contributor's checklist

  • Read through contributor's guide
  • Change Logs and Impact were stated clearly
  • Adequate tests were added if applicable
  • CI passed

@boneanxs
Copy link
Contributor Author

@alexeykudinkin @zhangyue19921010 Could you please help to take a look?

Copy link
Contributor

@zhangyue19921010 zhangyue19921010 left a comment

Choose a reason for hiding this comment

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

Nice catch @boneanxs !
We do have to make sure disruptor is set up before we are consuming records.

But do we need to have a synchronized check for each record consuming(performance concern)? Or we do a setup() step in BaseHoodieQueueBasedExecutor#execute before startConsumingAsync and startProducingAsync

Hi @alexeykudinkin what do u think :)

@zhangyue19921010
Copy link
Contributor

For example something like this

public class DisruptorExecutor<I, O, E> extends BaseHoodieQueueBasedExecutor<I, O, E> {

  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),
        transformFunction, waitStrategy, preExecuteRunnable);
  }

  public DisruptorExecutor(final Option<Integer> bufferSize, List<HoodieProducer<I>> producers,
                           Option<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);
  }

  @Override
  protected void doConsume(HoodieMessageQueue<I, O> queue, HoodieConsumer<O, E> consumer) {
    // no-op
    // will do consuming actions in disruptor
  }

  @Override
  public 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.get());
    disruptorQueue.start();
  }
}

and


  public void setup(){}

  /**
   * Main API to run both production and consumption.
   */
  @Override
  public E execute() {
    try {
      checkState(this.consumer.isPresent());
      setup();
      // Start consuming/producing asynchronously
      CompletableFuture<Void> consuming = startConsumingAsync();
      CompletableFuture<Void> producing = startProducingAsync();

      // NOTE: To properly support mode when there's no consumer, we have to fall back
      //       to producing future as the trigger for us to shut down the queue
      return producing.thenCombine(consuming, (aVoid, anotherVoid) -> null)
          .whenComplete((ignored, throwable) -> {
            // Close the queue to release the resources
            queue.close();
          })
          .thenApply(ignored -> consumer.get().finish())
          // Block until producing and consuming both finish
          .get();
    } catch (Exception e) {
      if (e instanceof InterruptedException) {
        // In case {@code InterruptedException} was thrown, resetting the interrupted flag
        // of the thread, we reset it (to true) again to permit subsequent handlers
        // to be interrupted as well
        Thread.currentThread().interrupt();
      }

      throw new HoodieException(e);
    }
  }

@boneanxs
Copy link
Contributor Author

But do we need to have a synchronized check for each record consuming(performance concern)?

This fix won't do the synchronize check for each record, it will only block the consuming when the queue is not start yet.

But I think adding a new setup() method is also an appropriate way. Pending this for a while if @alexeykudinkin has other suggestion. Or I'll add the setup() to fix it.

@biaoma-ty
Copy link
Contributor

typo in title

@zhangyue19921010
Copy link
Contributor

This PR may relate to https://issues.apache.org/jira/browse/HUDI-5369

@boneanxs boneanxs changed the title [HUDI-5488]Make sure Discrupt queue start first, then insert records [HUDI-5488]Make sure Disrupt queue start first, then insert records Jan 9, 2023
@boneanxs
Copy link
Contributor Author

Gentle ping @alexeykudinkin

@alexeykudinkin
Copy link
Contributor

@boneanxs @zhangyue19921010

We should do what @zhangyue19921010 suggested in his review

  1. Introduce empty setup method in the base class
  2. Move current contents of the doConsume for Disruptor into setup
  3. Keep doConsume for Disruptor empty

@alexeykudinkin alexeykudinkin changed the title [HUDI-5488]Make sure Disrupt queue start first, then insert records [HUDI-5488] Make sure Disrupt queue start first, then insert records Jan 10, 2023
@danny0405 danny0405 added the writer-core Issues relating to core transactions/write actions label Jan 12, 2023
@@ -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

@boneanxs
Copy link
Contributor Author

@hudi-bot run azure

1 similar comment
@zhangyue19921010
Copy link
Contributor

@hudi-bot run azure

@boneanxs
Copy link
Contributor Author

The test failure should not relate to this, I can run it successfully in local side:

Screenshot 2023-01-16 at 14 15 26

@boneanxs
Copy link
Contributor Author

@hudi-bot run azure

1 similar comment
@boneanxs
Copy link
Contributor Author

@hudi-bot run azure

@@ -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.

@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

@@ -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.

Let's keep it, please rename it to isStarted

Copy link
Contributor

@zhangyue19921010 zhangyue19921010 left a comment

Choose a reason for hiding this comment

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

LGTM. We can land it after this nit(Alex mentioned) changed. Thanks for your contribution!

@hudi-bot
Copy link

CI report:

Bot commands @hudi-bot supports the following commands:
  • @hudi-bot run azure re-run the last Azure build

@alexeykudinkin
Copy link
Contributor

Landing this since there's no material changes since last revision, and CI passed previously

@alexeykudinkin alexeykudinkin merged commit 124ab5f into apache:master Jan 19, 2023
fengjian428 pushed a commit to fengjian428/hudi that referenced this pull request Jan 31, 2023
…pache#7582)

We must to make sure to set up Disruptor's queue first, then producer can insert records to the queue. But currently we have no idea which thread start first, so this pr tries to fix it.
fengjian428 pushed a commit to fengjian428/hudi that referenced this pull request Apr 5, 2023
…pache#7582)

We must to make sure to set up Disruptor's queue first, then producer can insert records to the queue. But currently we have no idea which thread start first, so this pr tries to fix it.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
priority:blocker writer-core Issues relating to core transactions/write actions
Projects
Archived in project
Development

Successfully merging this pull request may close these issues.

None yet

6 participants