Skip to content

Conversation

@zhangyue19921010
Copy link
Contributor

@zhangyue19921010 zhangyue19921010 commented Nov 10, 2022

Change Logs

Followed PR #5416

Add a new Executor Type named SIMPLE for hoodie records writer.

This Simple executor is Single Writer and Single Reader mode. Also this SimpleHoodieExecutor has no inner message queue and no inner lock which can consume and writing records from iterator directly.

Advantages: There is no need for additional memory and cpu resources due to lock or multithreading.

Disadvantages: lost some benefits such as speed limit. And can not de-coupe the network read (shuffle read) and network write (writing objects/files to storage) anymore which may cause a little lower throughput compared with DisruptorExecutor.

Also I did a quick benchmark using hoodie benchmark framework org.apache.spark.sql.execution.benchmark.BoundInMemoryExecutorBenchmark

Minimize the impact of producers and consumers efficiency issue as much as possible to focus on testing the throughput limit of the inner queue

The result are

OpenJDK 64-Bit Server VM 1.8.0_342-b07 on Linux 5.10.62-55.141.amzn2.x86_64
Intel(R) Xeon(R) Platinum 8259CL CPU @ 2.50GHz
COW Ingestion:                            Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
BoundInMemory Executor                            34661          35143         292          0.3        3466.1       1.0X
Simple Executor                                   17347          17796         681          0.6        1734.7       2.0X
Disruptor Executor                                15803          16535         936          0.6        1580.3       2.2X

this Simple Executor has good throughput and minimal resource usage.

Also add the corresponding UTs

Of course we need to unify all these executor/messageQueue related UTs as much as possible.
Create a new Tickets https://issues.apache.org/jira/browse/HUDI-5106
Will do it as next step

Impact

No impact

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

low

Documentation Update

no

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 Author

zhangyue19921010 commented Nov 11, 2022

Hi @alexeykudinkin and @nsivabalan
Sorry to bother u.
Would u mind to take a further look at this PR?
Really appreciate it if it's possible!

@zhangyue19921010
Copy link
Contributor Author

Hi @alexeykudinkin Sorry to bother u.
As we discuss in #6843
I review our executor and queue related interface.
image

Just keep on common method in HoodieMessageQueue which is necessary.

So would u mind to take a look at your convert?

Really appreciate it!

Copy link
Contributor

@alexeykudinkin alexeykudinkin left a comment

Choose a reason for hiding this comment

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

@zhangyue19921010 thanks for following up on this one!

* Advantages: there is no need for additional memory and cpu resources due to lock or multithreading.
* Disadvantages: lost some benefits such as speed limit. And maybe lower throughput.
*/
public class SimpleHoodieExecutor<I, O, E> extends HoodieExecutorBase<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.

Let's actually simplify this even further and just inherit from HoodieExecutor

@alexeykudinkin
Copy link
Contributor

@zhangyue19921010 let's also rebase on top of this one: #7238

@alexeykudinkin alexeykudinkin changed the title [HUDI-5190] Consuming records from Iterator directly instead of using inner message queue [HUDI-5190][HUDI-5023] Consuming records from Iterator directly instead of using inner message queue Nov 18, 2022
@alexeykudinkin
Copy link
Contributor

For those queries about relative performance of BIMQ/Disruptor/Simple on EMR cluster, i've captured this in: #6843 (comment)

Pasting the results in here:

#
# BoundedInMemoryQueue (baseline, master)
#

==================================================
Total time taken by all rounds (hudi): 5119338
Per round: List(204855, 504984, 585545, 646676, 516914, 554007, 510029, 533198, 554438, 508692)
==================================================


#
# Disruptor (master)
# Performance Improvement (from baseline): ~2%
#

===================================================
Total time taken by all rounds (hudi): 5027185
Per round: List(142641, 450997, 544476, 576815, 552873, 585657, 526764, 552790, 563460, 530712)
==================================================

#
# Simple (no queue)
# Performance Improvement (from baseline): ~26%
#

==================================================
Total time taken by all rounds (hudi): 3798022
Per round: List(189704, 467253, 512628, 468869, 363557, 361869, 351934, 357554, 364299, 360355)
==================================================

@alexeykudinkin alexeykudinkin changed the title [HUDI-5190][HUDI-5023] Consuming records from Iterator directly instead of using inner message queue [HUDI-5023] Consuming records from Iterator directly instead of using inner message queue Nov 30, 2022
@nsivabalan nsivabalan added the release-0.12.2 Patches targetted for 0.12.2 label Dec 6, 2022
@codope
Copy link
Member

codope commented Dec 7, 2022

@zhangyue19921010 Can you please rebase?

@zhangyue19921010
Copy link
Contributor Author

Hi @alexeykudinkin and @codope Rebased this PR.
Also re-run this benchmark and get the same result.

/**
* Wrapper of input records iterator
*/
public class SimpleHoodieMessageQueue<I, O> implements HoodieMessageQueue<I, O>, Iterable<O> {
Copy link
Contributor

Choose a reason for hiding this comment

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

@zhangyue19921010 i think we need to clean this one up -- we don't actually need it as we use to only pass an iterator essentially

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Changed.

* Advantages: there is no need for additional memory and cpu resources due to lock or multithreading.
* Disadvantages: lost some benefits such as speed limit. And maybe lower throughput.
*/
public class SimpleHoodieExecutor<I, O, E> extends BaseHoodieQueueBasedExecutor<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.

This is not queue-based, let's make it inherit from HoodieExecutor instead

Copy link
Contributor

Choose a reason for hiding this comment

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

Do you mind cleaning up some generics which are not used anymore w/in HoodieExecutor?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

All done. Also verify the performance.
Thanks for your review!

import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;

public class TestSimpleExecutionInSpark extends HoodieClientTestHarness {
Copy link
Contributor Author

@zhangyue19921010 zhangyue19921010 Dec 11, 2022

Choose a reason for hiding this comment

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

Hi @@alexeykudinkin
Not involved testInterruptExecutor in this SimpleExecutionSpark Test.
Bcz there is no inner Thread pool in this new SimpleExecutor so there is nothing to interrupt right?

Is that looks good to u ?

Copy link
Contributor

Choose a reason for hiding this comment

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

Yeah, that makes sense

@zhangyue19921010
Copy link
Contributor Author

@hudi-bot run azure

import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;

public class TestSimpleExecutionInSpark extends HoodieClientTestHarness {
Copy link
Contributor

Choose a reason for hiding this comment

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

Yeah, that makes sense

// records iterator
protected final Iterator<I> it;
private final Function<I, O> transformFunction;
private final Runnable preExecuteRunnable;
Copy link
Contributor

Choose a reason for hiding this comment

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

We don't need this one (this one is only required for these that spin up new threads)

*
* BoundInMemory Executor 34661 35143 292 0.3 3466.1 1.0X
* Simple Executor 17347 17796 681 0.6 1734.7 2.0X
* Disruptor Executor 15803 16535 936 0.6 1580.3 2.2X
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 amend the commentary at the top that this benchmark has been run w/ unconstrained parallelism which obviously is beneficial to Disruptor more than it's for Simple

return isWriteDone.get();
}

public boolean isRunning() {
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 one (we only use it in test, and can actually avoid that assertion altogether, since we're asserting # of records)

private final Function<I, O> transformFunction;
private final Runnable preExecuteRunnable;
private final AtomicBoolean isWriteDone = new AtomicBoolean(false);
private final AtomicBoolean isShutdown = new AtomicBoolean(false);
Copy link
Contributor

Choose a reason for hiding this comment

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

There's actually no state for us to clean up so we can just kill these 2

LOG.info("Starting consumer, consuming records from the records iterator directly");
preExecuteRunnable.run();
while (it.hasNext()) {
if (isShutdown.get()) {
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 check since

  • We don't expect to shutdown executor in the middle of iteration (in the normal course)
  • In case of exception this would be interrupted naturally

try {
exec = new SimpleHoodieExecutor(hoodieRecords.iterator(), consumer, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), getPreExecuteRunnable());
int result = exec.execute();
// It should buffer and write 100 records
Copy link
Contributor

Choose a reason for hiding this comment

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

There's no buffering in here, let's update this one

@zhangyue19921010
Copy link
Contributor Author

Hey @alexeykudinkin All comments addressed. PTAL

Copy link
Contributor

@alexeykudinkin alexeykudinkin left a comment

Choose a reason for hiding this comment

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

Thanks for addressing the feedback @zhangyue19921010!

LGTM, left minor comments regarding the test, PTAL

cleanupResources();
}

private Runnable getPreExecuteRunnable() {
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 one

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Removed.

// collect all records and assert that consumed records are identical to produced ones
// assert there's no tampering, and that the ordering is preserved
assertEquals(hoodieRecords, consumedRecords);
for (int i = 0; i < hoodieRecords.size(); i++) {
Copy link
Contributor

Choose a reason for hiding this comment

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

We don't need this for-loop, right?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yeap. Removed. Sorry for missing this one.

});

HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class);
when(hoodieWriteConfig.getDisruptorWriteBufferSize()).thenReturn(Option.of(16));
Copy link
Contributor

Choose a reason for hiding this comment

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

We need to clean up these

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Changed.

@zhangyue19921010
Copy link
Contributor Author

Hi @alexeykudinkin All comments are addressed. PTAL.
Also really appreciate for your review!

@hudi-bot
Copy link
Collaborator

CI report:

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

@alexeykudinkin alexeykudinkin merged commit 8d13a7e into apache:master Dec 15, 2022
@alexeykudinkin
Copy link
Contributor

Awesome job @zhangyue19921010! Thanks for taking this up from idea to completion!

@Override
public E execute() {
checkState(this.consumer.isPresent());

Copy link
Contributor

Choose a reason for hiding this comment

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

Since consumer aleays exists, there is no need to keep it in an Option, we can then avoids the invocation Option#get actually.

Copy link
Contributor

Choose a reason for hiding this comment

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

@danny0405 addressing this in #7476

fengjian428 pushed a commit to fengjian428/hudi that referenced this pull request Apr 5, 2023
… inner message queue (apache#7174)

Followed PR apache#5416

Add a new Executor Type named SIMPLE for hoodie records writer.

This Simple executor is Single Writer and Single Reader mode. Also this SimpleHoodieExecutor has no inner message queue and no inner lock which can consume and writing records from iterator directly.

Advantages: There is no need for additional memory and cpu resources due to lock or multithreading.

Disadvantages: lost some benefits such as speed limit. And can not de-coupe the network read (shuffle read) and network write (writing objects/files to storage) anymore which may cause a little lower throughput compared with DisruptorExecutor.

Also I did a quick benchmark using hoodie benchmark framework org.apache.spark.sql.execution.benchmark.BoundInMemoryExecutorBenchmark

Minimize the impact of producers and consumers efficiency issue as much as possible to focus on testing the throughput limit of the inner queue

The result are

OpenJDK 64-Bit Server VM 1.8.0_342-b07 on Linux 5.10.62-55.141.amzn2.x86_64
Intel(R) Xeon(R) Platinum 8259CL CPU @ 2.50GHz
COW Ingestion:                            Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
BoundInMemory Executor                            34661          35143         292          0.3        3466.1       1.0X
Simple Executor                                   17347          17796         681          0.6        1734.7       2.0X
Disruptor Executor                                15803          16535         936          0.6        1580.3       2.2X
this Simple Executor has good throughput and minimal resource usage.

Also add the corresponding UTs
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

priority:critical Production degraded; pipelines stalled release-0.12.2 Patches targetted for 0.12.2

Projects

Archived in project

Development

Successfully merging this pull request may close these issues.

7 participants