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

Make persists concurrent with adding rows in batch ingestion #11536

Merged
merged 21 commits into from
Sep 8, 2021

Conversation

loquisgon
Copy link

@loquisgon loquisgon commented Aug 3, 2021

Recent testing have shown a potential performance degradation in the bounded memory work for segment creation of batch ingestion. The reason is most probably the fact that intermediate persists are blocking, serial, with ingestion. We will add back concurrent persists and test performance to validate that previous performance is back.

Changes:
Intermediate persists are made concurrent with adding rows in the batch appenderator.

We are also interested in supporting three main modes for the batch appenderator, this PR is implementing also the following requirements:

  • Fix the performance regression by making persists & push concurrent
  • Have a rollback flag called batchProcessingMode. This configuration setting will be able to take three string values: LEGACY, CLOSED_SEGMENTS, CLOSED_SEGMENTS_SINKS (see table below). The default value will be: CLOSED_SEGMENTS. Note that most existing tests will exercise the default but there are specific tests for the other two modes (see configuration docs to know what each of these mean).
  • Update docs to document the new processing mode

This PR has:

  • [ X] been self-reviewed.
  • added documentation for new or modified features or behaviors.
  • added Javadocs for most classes and all non-trivial methods. Linked related entities via Javadoc links.
  • added or updated version, license, or notice information in licenses.yaml
  • [X ] added comments explaining the "why" and the intent of the code wherever would not be obvious for an unfamiliar reader.
  • [X ] added unit tests or modified existing tests to cover new code paths, ensuring the threshold for code coverage is met.
  • added integration tests.
  • been tested in a test Druid cluster.

@loquisgon loquisgon changed the title Make persists concurrent with ingestion Make persists concurrent with adding rows in batch ingestion Aug 3, 2021
@jihoonson jihoonson added the Bug label Aug 3, 2021
@loquisgon
Copy link
Author

I replaced the semaphore using the style of concurrency used in the appenderator. Now persists and push are ran in the background.

@@ -158,6 +169,11 @@

maxBytesTuningConfig = tuningConfig.getMaxBytesInMemoryOrDefault();
skipBytesInMemoryOverheadCheck = tuningConfig.isSkipBytesInMemoryOverheadCheck();
if (tuningConfig.getMaxPendingPersists() < 1) {
maxPendingPersists = DEFAULT_PENDING_PERSISTS;
Copy link
Contributor

Choose a reason for hiding this comment

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

What is the rationale for the default of 2? The previous default was 0 which is infinite. I don't think we ever need to change this in production. The doc for maxPendingPersists was not updated in #11294, so whatever we change here, we should fix the doc too.

Copy link
Author

@loquisgon loquisgon Aug 10, 2021

Choose a reason for hiding this comment

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

I see it is already documented in external docs that its default is zero. Good catch thanks!


private void initializeExecutors()
{
log.info("There will be up to[%d] pending persists", maxPendingPersists);
Copy link
Contributor

Choose a reason for hiding this comment

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

How could this be useful except for debugging? If this is only useful for debugging, it should be not info.

Copy link
Author

@loquisgon loquisgon Aug 10, 2021

Choose a reason for hiding this comment

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

Changed to debug in next commit

/**
* The following sinks metadata map and associated class are the way to retain metadata now that sinks
* are being completely removed from memory after each incremental persist.
*/
private final Map<SegmentIdWithShardSpec, SinkMetadata> sinksMetadata = new HashMap<>();
private final ConcurrentHashMap<SegmentIdWithShardSpec, SinkMetadata> sinksMetadata = new ConcurrentHashMap<>();
Copy link
Contributor

Choose a reason for hiding this comment

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

Please document details of the concurrent access pattern.

Copy link
Author

Choose a reason for hiding this comment

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

Added comments about the usage for this in the javadoc (next commit)


// This variable updated in add(), persist(), and drop()
private int rowsCurrentlyInMemory = 0;
private final AtomicInteger rowsCurrentlyInMemory = new AtomicInteger();
Copy link
Contributor

Choose a reason for hiding this comment

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

What threads can access this and bytesCurrentlyInMemory concurrently? If there are any, please document details of the concurrent access pattern.

Copy link
Author

Choose a reason for hiding this comment

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

Removed unnecessary atomics in next committ.

private volatile Throwable persistError;


private final ConcurrentHashMap<SegmentIdWithShardSpec, Sink> sinks = new ConcurrentHashMap<>();
Copy link
Contributor

Choose a reason for hiding this comment

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

Can multiple threads access this map at the same time? I don't see any unless I'm missing something. If there are any, please document details of the concurrent access pattern. It helps people a lot including reviewers, other developers, and your future-self to understand and remember how things work. Also please check out https://github.com/apache/druid/blob/master/dev/code-review/concurrency.md. We have reasons for including it in the PR template. I would highly suggest reading it and marking the concurrency self-review item in your PR checklist.

Copy link
Author

Choose a reason for hiding this comment

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

No concurrency control required for this map, removed in next commit.

@@ -806,7 +831,8 @@ public void testTotalRowCount() throws Exception
appenderator.add(IDENTIFIERS.get(2), createInputRow("2001", "bob", 1), null);
Assert.assertEquals(4, appenderator.getTotalRowCount());

appenderator.persistAll(null).get();
appenderator.persistAll(null);
waitForPersists();
Copy link
Contributor

Choose a reason for hiding this comment

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

Why not persistAll(null).get()?

Copy link
Author

Choose a reason for hiding this comment

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

Typo --- fixed in next commit.

@@ -913,6 +939,10 @@ static InputRow createInputRow(String ts, String dim, Object met)
);
}

private void waitForPersists() throws InterruptedException
{
Thread.sleep(500);
Copy link
Contributor

Choose a reason for hiding this comment

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

Sleeps are bad. These will make the unit testing slower. Also, I bet all the sleeps you added will make these tests quite flaky.

Copy link
Author

Choose a reason for hiding this comment

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

The use case here is somehow wait for a concurrent task to check on its effect, it seems like a legitimate use for sleep. Let me know what you are thinking that could be better.

Copy link
Author

Choose a reason for hiding this comment

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

oh I see your suggestion below...

@@ -382,12 +387,14 @@ public void testTaskDoesNotFailAsExceededMemoryWithSkipBytesInMemoryOverheadChec

appenderator.startJob();
appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo", 1), null);
waitForPersists();
Copy link
Contributor

Choose a reason for hiding this comment

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

I think you could keep the future of persist triggered in add() in a variable. Then you can add a method used only for testing that returns the persist future. Then you can finally wait for the future to be done instead of sleeping.

Copy link
Author

Choose a reason for hiding this comment

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

I was trying to avoid more changes to the class (exposing methods etc) but I can pursue your suggestion

@jihoonson
Copy link
Contributor

BTW, I assumed that the core logic is not changed and reviewed only the concurrency part. Let me know if this is not true.

@loquisgon
Copy link
Author

You are correct, @jihoonson, core logic is not changed.

@loquisgon loquisgon closed this Sep 2, 2021
@loquisgon loquisgon reopened this Sep 2, 2021
@loquisgon
Copy link
Author

loquisgon commented Sep 2, 2021

Changes I added in the latest commit (507c2f6):

  • Make sure close() waits for persists & pushes before closing
  • Merged master branch to pick up fixes to a previous issue (serialization/deserialization of JSON interval when it contains UTC timestamps)
  • Renamed BatchAppenderator#clearSinkMetadata to BatchAppenderator#clearSinkMemoryCountersAndDiskStoredData to make intent clearer

DatasourceBundle::new
);

Appenderator appenderator = Appenderators.createLegacyOffline(
Copy link
Author

Choose a reason for hiding this comment

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

This is wrong, it should be createClosedSegmentsOffline, I will fix it

Copy link
Author

Choose a reason for hiding this comment

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

Done

@lgtm-com
Copy link

lgtm-com bot commented Sep 4, 2021

This pull request introduces 1 alert when merging 37e0856 into 59d2578 - view on LGTM.com

new alerts:

  • 1 for Uncontrolled data used in path expression

@lgtm-com
Copy link

lgtm-com bot commented Sep 6, 2021

This pull request introduces 1 alert when merging 291420e into 60efbb5 - view on LGTM.com

new alerts:

  • 1 for Uncontrolled data used in path expression

@lgtm-com
Copy link

lgtm-com bot commented Sep 7, 2021

This pull request introduces 1 alert when merging 1b1fd7b into 60efbb5 - view on LGTM.com

new alerts:

  • 1 for Uncontrolled data used in path expression

@@ -43,6 +44,15 @@
"org.apache.hadoop:hadoop-client:2.8.5"
);

public enum BatchProcesingMode
Copy link
Contributor

Choose a reason for hiding this comment

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

BatchProcesingMode -> BatchProcessingMode

@@ -1343,7 +1343,8 @@ Additional peon configs include:
|`druid.peon.mode`|Choices are "local" and "remote". Setting this to local means you intend to run the peon as a standalone process (Not recommended).|remote|
|`druid.indexer.task.baseDir`|Base temporary working directory.|`System.getProperty("java.io.tmpdir")`|
|`druid.indexer.task.baseTaskDir`|Base temporary working directory for tasks.|`${druid.indexer.task.baseDir}/persistent/task`|
|`druid.indexer.task.useLegacyBatchProcessing`|If false, native batch ingestion will use a new, recommended, code path with memory optimized code for the segment creation phase. If true it will use the previous code path for the create segments phase of batch ingestion. This does not apply to streaming ingestion, just to batch. This setting should only be used when a bug is suspected or found in the new optimized batch ingestion code. If a bug is suspected or found, you can set this flag to `true` to fall back to previous, working but more memory intensive, code path.|`false`|
|`druid.indexer.task.batchMemoryMappedIndex`|DEPRECATED: Use `druid.indexer.task.batchProcessingMode` instead. If false, native batch ingestion will use a new, recommended, code path with memory optimized code for the segment creation phase. If true it will use the previous code path for the create segments phase of batch ingestion. This does not apply to streaming ingestion, just to batch. This setting should only be used when a bug is suspected or found in the new optimized batch ingestion code. If a bug is suspected or found, you can set this flag to `true` to fall back to previous, working but more memory intensive, code path.|`false`|
Copy link
Contributor

Choose a reason for hiding this comment

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

Maybe call out that set batchMemoryMappedIndex to true will set batchProcessingMode to LEGACY and overwrite the batchProcessingMode value.

} else {
// batchProcessingMode input string is invalid, just use the default...log message somewhere???
this.batchProcessingMode = BatchProcesingMode.CLOSED_SEGMENTS; // Default
}
Copy link
Contributor

Choose a reason for hiding this comment

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

Can we log the batchProcessingMode value after this if block?

Copy link
Author

Choose a reason for hiding this comment

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

yeah

@clintropolis clintropolis added this to the 0.22.0 milestone Sep 8, 2021
@@ -314,7 +314,8 @@ public File getTaskReportsFile()
);

final TaskToolbox box = new TaskToolbox(
new TaskConfig(null, null, null, null, null, false, null, null, null, false, false),
new TaskConfig(null, null, null, null, null, false, null, null, null, false, false,
TaskConfig.BatchProcessingMode.CLOSED_SEGMENTS.name()),
Copy link
Member

Choose a reason for hiding this comment

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

nit: should this use default instead of this explicit value?

Copy link
Author

Choose a reason for hiding this comment

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

done in next


/**
* This class is to support LEGACY and CLOSED_SEGMENTS appenderators. It is copied as-is
* from 0.21 and it is meant to keep for backward compatibility. For now though this class
Copy link
Member

Choose a reason for hiding this comment

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

nit: this isn't actually a copy of 0.21... its of some intermediary state between 0.21 and 0.22. I think StreamAppenderator is technically a copy of ApenderatorImpl in 0.21.

Copy link
Author

Choose a reason for hiding this comment

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

Updated in next

import java.util.Map;
import java.util.concurrent.CopyOnWriteArrayList;

public class LegacyAndClosedSegmentsAppenderatorTester implements AutoCloseable
Copy link
Member

Choose a reason for hiding this comment

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

super-nit: OpenAndClosed... instead of LegacyAndClosed

Copy link
Author

Choose a reason for hiding this comment

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

yeah

import java.util.function.Function;
import java.util.stream.Collectors;

public class LegacyAndClosedSegmentsBatchAppenderatorDriverTest extends EasyMockSupport
Copy link
Member

Choose a reason for hiding this comment

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

same super nit replacing Legacy with Open

Copy link
Author

Choose a reason for hiding this comment

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

check

import java.util.List;
import java.util.stream.Collectors;

public class LegacyAndClosedSegmentsBatchAppenderatorTest extends InitializedNullHandlingTest
Copy link
Member

Choose a reason for hiding this comment

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

same nit legacy/open

Copy link
Author

Choose a reason for hiding this comment

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

check

@lgtm-com
Copy link

lgtm-com bot commented Sep 8, 2021

This pull request introduces 1 alert when merging c85a201 into dcee99d - view on LGTM.com

new alerts:

  • 1 for Uncontrolled data used in path expression

@lgtm-com
Copy link

lgtm-com bot commented Sep 8, 2021

This pull request introduces 1 alert when merging 86973a1 into dcee99d - view on LGTM.com

new alerts:

  • 1 for Uncontrolled data used in path expression

@lgtm-com
Copy link

lgtm-com bot commented Sep 8, 2021

This pull request introduces 1 alert when merging bd83203 into dcee99d - view on LGTM.com

new alerts:

  • 1 for Uncontrolled data used in path expression

@lgtm-com
Copy link

lgtm-com bot commented Sep 8, 2021

This pull request introduces 1 alert when merging 7f004f1 into dcee99d - view on LGTM.com

new alerts:

  • 1 for Uncontrolled data used in path expression

…hod that was still using "legacy" rather than "openSegments"
@lgtm-com
Copy link

lgtm-com bot commented Sep 8, 2021

This pull request introduces 1 alert when merging d97615f into dcee99d - view on LGTM.com

new alerts:

  • 1 for Uncontrolled data used in path expression

@clintropolis clintropolis merged commit 9efa6cc into apache:master Sep 8, 2021
clintropolis added a commit to clintropolis/druid that referenced this pull request Sep 8, 2021
…11536)

* Make persists concurrent with ingestion

* Remove semaphore but keep concurrent persists (with add) and add push in the backround as well

* Go back to documented default persists (zero)

* Move to debug

* Remove unnecessary Atomics

* Comments on synchronization (or not) for sinks & sinkMetadata

* Some cleanup for unit tests but they still need further work

* Shutdown & wait for persists and push on close

* Provide support for three existing batch appenderators using batchProcessingMode flag

* Fix reference to wrong appenderator

* Fix doc typos

* Add BatchAppenderators class test coverage

* Add log message to batchProcessingMode final value, fix typo in enum name

* Another typo and minor fix to log message

* LEGACY->OPEN_SEGMENTS, Edit docs

* Minor update legacy->open segments log message

* More code comments, mostly small adjustments to naming etc

* fix spelling

* Exclude BtachAppenderators from Jacoco since it is fully tested but Jacoco still refuses to ack coverage

* Coverage for Appenderators & BatchAppenderators, name change of a method that was still using "legacy" rather than "openSegments"

Co-authored-by: Clint Wylie <cjwylie@gmail.com>
clintropolis added a commit that referenced this pull request Sep 9, 2021
…#11679)

* Make persists concurrent with ingestion

* Remove semaphore but keep concurrent persists (with add) and add push in the backround as well

* Go back to documented default persists (zero)

* Move to debug

* Remove unnecessary Atomics

* Comments on synchronization (or not) for sinks & sinkMetadata

* Some cleanup for unit tests but they still need further work

* Shutdown & wait for persists and push on close

* Provide support for three existing batch appenderators using batchProcessingMode flag

* Fix reference to wrong appenderator

* Fix doc typos

* Add BatchAppenderators class test coverage

* Add log message to batchProcessingMode final value, fix typo in enum name

* Another typo and minor fix to log message

* LEGACY->OPEN_SEGMENTS, Edit docs

* Minor update legacy->open segments log message

* More code comments, mostly small adjustments to naming etc

* fix spelling

* Exclude BtachAppenderators from Jacoco since it is fully tested but Jacoco still refuses to ack coverage

* Coverage for Appenderators & BatchAppenderators, name change of a method that was still using "legacy" rather than "openSegments"

Co-authored-by: Clint Wylie <cjwylie@gmail.com>

Co-authored-by: Agustin Gonzalez <agustin.gonzalez@imply.io>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

6 participants