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-5023] Cleaning up QueueBasedExecutor impls #7238

Merged
merged 29 commits into from Dec 7, 2022

Conversation

alexeykudinkin
Copy link
Contributor

@alexeykudinkin alexeykudinkin commented Nov 18, 2022

Change Logs

This is a follow-up PR for #5416 that is

  • Further cleaning up some of the historically inherited artifacts
  • Replacing remaining usages of the BIMQ, w/ QueueBasedExecutorFactory
  • Adding missing deps to bundles

This change is a precursor for both #7245 and #7174

Impact

No impact

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

Low

Documentation Update

N/A

Contributor's checklist

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

@zhangyue19921010
Copy link
Contributor

Ack Will finish my review this week.

* Start all producers at once.
*/
@Override
public CompletableFuture<Void> startProducers() {
Copy link
Contributor Author

Choose a reason for hiding this comment

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

This method (w/ modifications) have been moved into base class to be shared across impls

// Consumer
protected final Option<HoodieConsumer<O, E>> consumer;

public BaseHoodieQueueBasedExecutor(List<HoodieProducer<I>> producers,
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Ideas underpinning this class were

  • Abstract common logic across impls
  • Make sure all concurrency is handled exclusively in the base class (impl should just write sync code of how to produce/consume to/from the queue)
  • Make sure all resource lifecycle handling happens in the base class

Copy link
Contributor

Choose a reason for hiding this comment

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

Nice work here!

* HoodieExecutorBase holds common elements producerExecutorService, consumerExecutorService, producers and a single consumer.
* Also HoodieExecutorBase control the lifecycle of producerExecutorService and consumerExecutorService.
*/
public abstract class HoodieExecutorBase<I, O, E> implements HoodieExecutor<I, O, E> {
Copy link
Contributor Author

Choose a reason for hiding this comment

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

This class has been replaced by BaseHoodieQueueBasedExecutor

/**
* IteratorBasedHoodieMessageQueue implements HoodieMessageQueue with Iterable
*/
public abstract class HoodieIterableMessageQueue<I, O> implements HoodieMessageQueue<I, O>, Iterable<O> {
Copy link
Contributor Author

Choose a reason for hiding this comment

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

This is not needed anymore

/**
* Consume entries from queue and execute callback function.
*/
public abstract class IteratorBasedQueueConsumer<I, O> implements HoodieConsumer<I, O> {
Copy link
Contributor Author

Choose a reason for hiding this comment

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

This is not needed anymore

@@ -208,19 +205,33 @@ public BoundedMemoryRecords(
Schema logSchema,
Configuration hadoopConf,
org.apache.flink.configuration.Configuration flinkConf) {
HoodieUnMergedLogRecordScanner.Builder scannerBuilder = HoodieUnMergedLogRecordScanner.newBuilder()
Copy link
Contributor Author

Choose a reason for hiding this comment

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

This refactoring is necessary to avoid exposing internals of the executor (queue), instead relying on the provided interfaces to configure it

@zhangyue19921010
Copy link
Contributor

@hudi-bot run azure

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 Work!
Just left a few minor comments

Comment on lines 37 to 53
/**
* HoodieExecutorBase holds common elements producerExecutorService, consumerExecutorService, producers and a single consumer.
* Also HoodieExecutorBase control the lifecycle of producerExecutorService and consumerExecutorService.
*/
public abstract class BaseHoodieQueueBasedExecutor<I, O, E> implements HoodieExecutor<I, O, E> {

Copy link
Contributor

Choose a reason for hiding this comment

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

nit: HoodieExecutorBase => BaseHoodieQueueBasedExecutor

Comment on lines +88 to +115
public final CompletableFuture<Void> startProducingAsync() {
return allOf(producers.stream()
.map(producer -> CompletableFuture.supplyAsync(() -> {
doProduce(queue, producer);
return (Void) null;
}, producerExecutorService))
.collect(Collectors.toList())
)
.thenApply(ignored -> (Void) null)
.whenComplete((result, throwable) -> {
// Regardless of how producing has completed, we have to close producers
// to make sure resources are properly cleaned up
producers.forEach(HoodieProducer::close);
// Mark production as done so that consumer will be able to exit
queue.seal();
});
}
Copy link
Contributor

Choose a reason for hiding this comment

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

How about using

  public final CompletableFuture<Void> startProducingAsync() {

    return CompletableFuture.allOf(producers.stream().map(producer -> {
      return CompletableFuture.supplyAsync(() -> {
        doProduce(queue, producer);
        return (Void) null;
      }, producerExecutorService);
    }).toArray(CompletableFuture[]::new)).whenComplete(new BiConsumer<Void, Throwable>() {
      @Override
      public void accept(Void aVoid, Throwable throwable) {
        // Regardless of how producing has completed, we have to close producers
        // to make sure resources are properly cleaned up
        producers.forEach(HoodieProducer::close);
        // Mark production as done so that consumer will be able to exit
        queue.seal();
      }
    });
  }

may be no need to do .thenApply(ignored -> (Void) null)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

In #7245, i'm actually going to replace this w/ an allOf instance having slightly different semantic (allowing to cascade cancellations upon first encountered failure)

Comment on lines 163 to 191
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
.join();
} catch (Exception e) {
throw new HoodieException(e);
}
Copy link
Contributor

Choose a reason for hiding this comment

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

It seems that TestBoundedInMemoryExecutorInSpark#testInterruptExecutor is blocked here which case ut timeout.
Maybe we can revisit the logic here about how to handle interrupt exception.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yeah, join actually doesn't throw InterruptedException. Addressed

// Consumer
protected final Option<HoodieConsumer<O, E>> consumer;

public BaseHoodieQueueBasedExecutor(List<HoodieProducer<I>> producers,
Copy link
Contributor

Choose a reason for hiding this comment

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

Nice work here!

@Override
public void shutdownNow() {
producerExecutorService.shutdownNow();
consumerExecutorService.shutdownNow();
Copy link
Contributor

Choose a reason for hiding this comment

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

Maybe we need to close current queue into this shutdownNow func.

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 actually close the queue w/in the execute method even in the presence of exceptions (which will cascade from shutdownNow if invoked). Idea here is that shutdownNow should only be used as last-ditch effort of (quickly!) shutting down spun up threads only

Copy link
Contributor

Choose a reason for hiding this comment

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

That make sense !

@@ -31,11 +32,18 @@

public class QueueBasedExecutorFactory {
Copy link
Contributor

Choose a reason for hiding this comment

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

Maybe we can also rename QueueBasedExecutorFactory to ExecutorFactory because we will support simpleExecutor here which has no inner Queue

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Agreed. Let's take this up in the PR introducing SimpleExecutor

@alexeykudinkin alexeykudinkin force-pushed the ak/exeq-fix branch 2 times, most recently from 5e9080b to 3532ca4 Compare November 30, 2022 17:37
@alexeykudinkin alexeykudinkin changed the title [HUDI-3963] Cleaning up QueueBasedExecutor impls [HUDI-5023] Cleaning up QueueBasedExecutor impls Nov 30, 2022
@alexeykudinkin alexeykudinkin force-pushed the ak/exeq-fix branch 2 times, most recently from 0c62553 to a76c39f Compare December 1, 2022 16:25
@zhangyue19921010
Copy link
Contributor

@hudi-bot run azure

@codope codope added priority:critical production down; pipelines stalled; Need help asap. writer-core Issues relating to core transactions/write actions labels Dec 2, 2022
Copy link
Member

@codope codope left a comment

Choose a reason for hiding this comment

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

@alexeykudinkin @zhangyue19921010 Do you plan to re-run the same benchmark as in #5416 for this PR as well?

packaging/hudi-flink-bundle/pom.xml Show resolved Hide resolved
@@ -149,6 +153,7 @@ public void close() throws IOException {

@Override
public float getProgress() throws IOException {
return Math.min(parquetReader.getProgress(), logRecordScanner.getProgress());
// TODO fix to reflect scanner progress
Copy link
Member

Choose a reason for hiding this comment

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

Why would we need that for "unmerged" record reader?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I think this is just tracking progress in terms of how many records were processed from the base-file vs logs

Copy link
Member

Choose a reason for hiding this comment

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

Synced up offline. This will be addressed in followup PR.

})
.build();
// Scan all the delta-log files, filling in the queue
scanner.scan();
Copy link
Member

Choose a reason for hiding this comment

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

maybe assign scanner progress here, which can be used in L157.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Will take this up in #7245

@zhangyue19921010
Copy link
Contributor

zhangyue19921010 commented Dec 2, 2022

@alexeykudinkin @zhangyue19921010 Do you plan to re-run the same benchmark as in #5416 for this PR as well?

Yeap, I run this benchmark several times based on this PR and the result are the same as before. Thanks for reminding!

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.

LTGM! Thanks for this GREAT work!

After UTs green maybe we can land it. Maybe we can rebase master #7346 and have another try.

@hudi-bot
Copy link

hudi-bot commented Dec 3, 2022

CI report:

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

@@ -149,6 +153,7 @@ public void close() throws IOException {

@Override
public float getProgress() throws IOException {
return Math.min(parquetReader.getProgress(), logRecordScanner.getProgress());
// TODO fix to reflect scanner progress
Copy link
Member

Choose a reason for hiding this comment

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

Synced up offline. This will be addressed in followup PR.

@codope codope merged commit 20eea46 into apache:master Dec 7, 2022
fengjian428 pushed a commit to fengjian428/hudi that referenced this pull request Apr 5, 2023
- Further cleaning up some of the historically inherited artifacts
- Replacing remaining usages of the BIMQ, w/ QueueBasedExecutorFactory
- Adding missing deps to bundles
- Lifted up all concurrency management to `BaseHoodieQueueBasedExuecutor`
- Fixed handling of the case when there's no consumer provide closing the queue prematurely
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
priority:critical production down; pipelines stalled; Need help asap. 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

4 participants