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

Add util/groupcommit package in core subproject #1607

Merged
merged 25 commits into from
Apr 9, 2024

Conversation

komamitsu
Copy link
Contributor

@komamitsu komamitsu commented Mar 14, 2024

Description

ScalarDB may sometimes need to work with a high latency storage (e.g., multi-region replicated database). In such a case, the throughput of application using ScalarDB would be affected by the high latency transactions. With a group commit mechanism, high latency issue can be hidden with concurrent transactions.

In this PR, we introduce util/groupcommit package in core subproject which provides a mechanism to support the ScalarDB protocol. The package alone doesn't affect the existing implementations. A following PR will integrate the new package and the existing implementations.

Related issues and/or PRs

N/A

Changes made

This PR basically adds the following classes.
image

The util/groupcommit package has a concept about keys and they should be described before looking into the classes.

  • Keys
    • Child Key : This is passed by a client (e.g., UUID v4 like ScalarDB transaction ID)
    • Parent Key : This is generated by KeyManipulator class which is injected by a client and contains logics to handle Keys
    • Full Key : This consists of Child Key and Parent Key

The main components introduced by this PR are as follows:

  • Group
    • A unit of (group) commit
    • Contains slots (see below about slots). The number of slots in a group is decided by the configuration
    • Possible status are as follows:
      • OPEN : Initial status. Accepting new slot reservation as the number of slots isn't fixed yet
      • CLOSED : Not accepting new slot reservations since the number of slots is already fixed. Waiting all the slots are set with values
      • READY : All the slots are set with values. Ready to commit
      • DONE : Group commit is done and all the clients have get the results
    • There are 2 types of Groups
      • NormalGroup
        • Contains multiple slots and commits them at once. Ideally only this Group should be used in terms of performance
        • Parent Key is used to identify NormalGroup
      • DelayedGroup
        • Contains only a single slot which is delayed and removed from the NormalGroup
        • Full Key is used to identify DelayedGroup
  • Slot
    • A container for a value. Values are supposed to be emitted at once by this group commit mechanism
    • Each Slot has the corresponding Child Key
  • GroupManager
    • Manages all the ongoing Groups
    • It internally contains 2 maps
      • NormalGroupMap : Key=Parent-Key, Value=NormalGroup
      • DelayedGroupMap : Key=Full-Key, Value=DelayedGroup
  • Worker threads
    • Group-Close Thread : Fixes the number of slot (or close the group) when the Group is timed out
    • Delayed-Slot-Move Thread : Moves delayed Slots from NormalGroupMap to DelayedGroupMap
    • Group-Cleanup Thread : Removes Group entries that are already done from NormalGroupMap or DelayedGroupMap. It's a sort of garbage collector
  • GroupCommitter
    • Manages all the above classes
    • Has the following public APIs
      • reserve(Child Key) returns Full Key
        • Allocates a Slot in the current group that is associated with a Full Key
        • The allocated slot status is initially NOT_READY
      • ready(Full Key)
        • Marks the slot associated with the Full Key READY
        • Waits until all the slots in the group are READY
        • Internally calls the Emittable.emit() which is injected by the client with either of the following keys once the group gets READY
          • Parent Key : In most cases, this type of Key is used to emit multiple values in NormalGroup
          • Full Key : This type of Key is used only when it takes long time to call this API and the delayed slot is separated from the original NormalGroup to a new DelayedGroup to prevent other slots in the Group from being delayed

Very rough overall architecture of the group commit
image
Replace txid with Key when reading the description.

Lifecycle of Group
image

Checklist

The following is a best-effort checklist. If any items in this checklist are not applicable to this PR or are dependent on other, unmerged PRs, please still mark the checkboxes after you have read and understood each item.

  • I have commented my code, particularly in hard-to-understand areas.
  • I have updated the documentation to reflect the changes.
  • Any remaining open issues linked to this PR are documented and up-to-date (Jira, GitHub, etc.).
  • Tests (unit, integration, etc.) have been added for the changes.
  • My changes generate no new warnings.
  • Any dependent changes in other PRs have been merged and published.

Additional notes (optional)

The current implementation contains the following 3 TODOs. They'll be handled in other PRs.

  • Possible performance improvement with tweaking the scope of locks
  • Replace the tentative metrics monitoring with DropWizard's one
  • Performance improvement of DelayedSlotMoveWorker
    • The worker basically waits for every item, so the performance improvement is required
    • Introducing a priority queue is an option, which was suggested by @brfrn169

Release notes

N/A (This PR itself isn't a user facing change).

@brfrn169 brfrn169 added the enhancement New feature or request label Mar 15, 2024
Copy link
Collaborator

@brfrn169 brfrn169 left a comment

Choose a reason for hiding this comment

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

My first review is complete, and I've left several comments. Please take a look when you have time! I will conduct a second review later on. Thank you for your great work!

// TODO: This should be replaced by other metrics mechanism.
private void startMonitorExecutorService() {
Runnable print =
() -> logger.info("[MONITOR] Timestamp={}, Metrics={}", Instant.now(), getMetrics());
Copy link
Collaborator

Choose a reason for hiding this comment

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

I feel we can make it a debug log. What do you think?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Sounds good! I'll update it.

BTW, this will be soon replaced with #1614 which uses DropWizard Metrics.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Fixed in ade2864

private void reserveSlot(Slot<PARENT_KEY, CHILD_KEY, FULL_KEY, EMIT_KEY, V> slot) {
Slot<?, ?, ?, ?, ?> oldSlot = slots.put(slot.key(), slot);
if (oldSlot != null) {
logger.warn("An old slot exist unexpectedly. {}", oldSlot.fullKey());
Copy link
Collaborator

Choose a reason for hiding this comment

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

This shouldn't occur, right? If so, just showing the warning message is okay?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Good catch! Right. I'll update it to throw AssertionError or IllegalStateException.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Fixed in 8d96fd4

*
* @param fullKey A full key to specify the slot.
*/
public void remove(FULL_KEY fullKey) {
Copy link
Collaborator

Choose a reason for hiding this comment

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

When is this remove method intended to be called?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Good question.

Let's say a transaction executes some CRUD operations, starts PREPARE, and fails before the group commit for coordinator state. In this case, a group commit slot is allocated for the transaction and needed to be removed by calling this method not to leave garbage slots.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Updated the comment in 4ff928a

* @param value A value to be set to the slot.
* @throws GroupCommitException when group commit fails
*/
public void ready(FULL_KEY fullKey, V value) throws GroupCommitException {
Copy link
Collaborator

Choose a reason for hiding this comment

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

What is the purpose of this value argument? Maybe we use it for a transaction ID (a child key) for each transaction within a group?

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 the case of coordinator state commit, V is supposed to be Snapshot and a passed value will be committed with other snapshots in the same group.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Updated the comment to make it clearer in fa5e726

* @param <EMIT_KEY> A key type that Emitter can interpret.
* @param <V> A value type to be set to a slot.
*/
public class GroupCommitter<PARENT_KEY, CHILD_KEY, FULL_KEY, EMIT_KEY, V> implements Closeable {
Copy link
Collaborator

Choose a reason for hiding this comment

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

I think we can put @ThreadSafe or @NotThreadSafe for each class.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Fixed in e581918

and simplify some code
Fix a race condition bug that a newly created DelayedGroup is already closed when reserving a slot.

The scenario that the issue occurs is as follows
- DelayedSlotMoveWorker adds the new DelayedGroup into GroupCleanupWorker
- GroupCleanupWorker calls DelayedGroup.updateState() before DelayedSlotMoveWorker reserves a slot on the DelayedGroup
  - The slots of the DelayedGroup is still empty and it’s marked as DONE
- DelayedSlotMoveWorker fails to reserve a slot on the DelayedGroup since it’s closed (DONE)
@komamitsu komamitsu requested a review from brfrn169 March 26, 2024 03:00
Copy link
Contributor

@feeblefakie feeblefakie left a comment

Choose a reason for hiding this comment

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

Thank you for the great improvement and very sorry for the late review.
I took a brief look and left some comments and questions. PTAL!
(Let me check this one more time.)

Let me ask additional questions here.

  • We discussed before that we might need to access the coordinator table twice in some recovery cases, but how does the current code handle the case?
  • If one transaction in a group fails to commit, will the transaction be handled in a DelayedGroup?

// Waiting all the slots are set with values.
//
// Groups with OPEN status can move to CLOSED.
CLOSED(true, false, false),
Copy link
Contributor

Choose a reason for hiding this comment

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

CLOSED corresponds to OPEN so it's good, but it sounds like the group has already finished processing.
How about FIXED?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Thanks for the suggestion! Actually it was first like SIZE-FIXED and I changed it to CLOSED to sound to reject new requests. But as you said, it also sounds things are done. I'll update it to use FIX.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Fixed in 5000335. Used SIZE_FIXED to make it clearer.

// If it returns null, the Group is already closed and a retry is needed.
@Nullable
FULL_KEY reserveNewSlot(CHILD_KEY childKey) {
long stamp = lock.writeLock();
Copy link
Contributor

Choose a reason for hiding this comment

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

I might miss something, but since we have only one active group at a time, I'm wondering how much it badly affects performance if there are many concurrent transactions.
We could use multiple active groups to mitigate the potential issue, but it might complicate the mechanism.
Any thoughts on this?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Good point. I also thought and tried multiple current (== active) groups before. But, the performance with multiple active groups didn't have a good performance. I think how fast to fill up a group with slots (and values) is a very important factor to move it to the next state (i.e., ready to commit) in the group commit and using multiple active groups doesn't fit.

I benchmarked this util/groupcommit pacakge itself and the average throughput was about 65K TPS without any wait while the throughput with some delays to emulate actual latencies of underlying database had far low throughputs. It means current single active group won't be a bottleneck in usual usages of ScalarDB, I think.

Copy link
Contributor

Choose a reason for hiding this comment

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

Thank you for the detailed explanation!
It makes sense. Let's leave it as is.

@komamitsu
Copy link
Contributor Author

@feeblefakie Thanks for the questions!

We discussed before that we might need to access the coordinator table twice in some recovery cases, but how does the current code handle the case?

This PR contains doesn't contain any integration with the coordinator state commit, so let me refer to another PR's change (https://github.com/scalar-labs/scalardb/pull/1523/files#diff-8049e66d11b81cdd5a1dc82c4977b5336741bcee24dd7ddd54c6a6940f09497bR83-R89). I tried the idea to put child-tx-id in a clustering key before in order to make it possible to fetch both group-committed-transactions' ID and separately-committed-transactions' ID. But, there wasn't significant performance improvement with YCSB-F benchmark despite it would require cumbersome table schema migrations. So, current implementation is to access the coordinator table twice.

If one transaction in a group fails to commit, will the transaction be handled in a DelayedGroup?

No. Slots are moved to DelayedGroup only when it's delayed. If a NormalGroup fails to commit due to network or underlying database issues, all the slots in the group fail. BTW, it's about Commit-State phase. If a transaction fails in Prepare or Validate, it'll be just removed from the NormalGroup without being moved to DelayedGroup.

@feeblefakie
Copy link
Contributor

@komamitsu

I tried the idea to put child-tx-id in a clustering key before in order to make it possible to fetch both group-committed-transactions' ID and separately-committed-transactions' ID. But, there wasn't significant performance improvement with YCSB-F benchmark despite it would require cumbersome table schema migrations. So, current implementation is to access the coordinator table twice.

Thank you for the explanation.
I think how much performance improves depends on how much the workload triggers conflicts,
so it might be worth trying TPC-C to see how it goes since there are not many conflicts in YCSB.

No. Slots are moved to DelayedGroup only when it's delayed. If a NormalGroup fails to commit due to network or underlying database issues, all the slots in the group fail. BTW, it's about Commit-State phase. If a transaction fails in Prepare or Validate, it'll be just removed from the NormalGroup without being moved to DelayedGroup.

Thank you!
I got it.

private final GroupCommitConfig config;

GroupManager(
String label,
Copy link
Collaborator

Choose a reason for hiding this comment

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

Nit: It loos like this label is never used.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Fixed it in 0007ba3


FULL_KEY fullKey(PARENT_KEY parentKey, CHILD_KEY childKey);

boolean isFullKey(Object obj);
Copy link
Collaborator

Choose a reason for hiding this comment

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

Nit: It looks like it's never used.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Ah, right. Indeed, it's not used in this PR. But, it'll be used in the subsequent PR https://github.com/scalar-labs/scalardb/pull/1523/files#diff-8049e66d11b81cdd5a1dc82c4977b5336741bcee24dd7ddd54c6a6940f09497bR75. Let me keep it if possible.

package com.scalar.db.util.groupcommit;

@FunctionalInterface
interface ThrowableRunnable {
Copy link
Collaborator

Choose a reason for hiding this comment

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

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Fixed it in e8faee2

Copy link
Contributor

@feeblefakie feeblefakie left a comment

Choose a reason for hiding this comment

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

Overall, looking good!
Left one question. PTAL!

T removed = queue.poll();
// Check if the removed group is expected just in case.
if (removed == null || !removed.equals(item)) {
logger.error(
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 fully understand it, but I'm wondering in what cases this happens.
If it happens even in normal cases, should it be warning instead of error?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Good point. It shouldn't happen in normal cases. Throwing AssertionError or IllegalStateException is an option. But throwing either of them from background worker doesn't make sense so much and I think just outputting error log is enough.

But, I noticed the error message probably looks unclear thanks to your comment. I updated it in bdf344f

Copy link
Contributor

Choose a reason for hiding this comment

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

Looking good!
Yeah, it cannot stop processing even such a case, so having the updated comment makes sense.

@komamitsu
Copy link
Contributor Author

so it might be worth trying TPC-C to see how it goes since there are not many conflicts in YCSB.

@feeblefakie Yeah, good point. I'll try TPC-C with the clustering key version group commit before submitting a PR for the integration of the group commit with the coordinator state commit.

and add some comments
@komamitsu
Copy link
Contributor Author

@feeblefakie @brfrn169 Thanks for reviewing this PR!
This is a small FYI, but I renamed asyncEmit() to delegateEmitTaskToWaiter() in 545ad20 since the old method name was misleading (actually, the emit task is executed by one of the slots in the group synchronously in waitUntilEmit())

@brfrn169 The performance improvement task of DelayedSlotMoveWorker will be handled in another PR 545ad20#diff-21491e996128b6ee7175154186a58be20ca714274559ba304eb39c59be8db4faR110-R113

Copy link
Collaborator

@brfrn169 brfrn169 left a comment

Choose a reason for hiding this comment

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

Left very minor suggestions. It looks good to me, but let me review this once again later.

long stamp = lock.writeLock();
try {
// TODO: NormalGroup.removeNotReadySlots() calls updateStatus() potentially resulting in
// asyncEmit(). Maybe it should be called outside the lock.
Copy link
Collaborator

Choose a reason for hiding this comment

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

Nit:

Suggested change
// asyncEmit(). Maybe it should be called outside the lock.
// delegateEmitTaskToWaiter(). Maybe it should be called outside the lock.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Thanks! Fixed it in 8658300

private final CHILD_KEY key;
// If a result value is null, the value is already emitted.
// Otherwise, the result lambda must be emitted by the receiver's thread.
private final CompletableFuture<ThrowableRunnable> completableFuture = new CompletableFuture<>();
Copy link
Collaborator

Choose a reason for hiding this comment

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

Suggested change
private final CompletableFuture<ThrowableRunnable> completableFuture = new CompletableFuture<>();
private final CompletableFuture<ThrowableRunnable<Exception>> completableFuture = new CompletableFuture<>();

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Thanks! Fixed it in 8658300

try {
// If a result value is null, the value is already emitted.
// Otherwise, the result lambda must be emitted by the receiver's thread.
ThrowableRunnable taskToEmit = completableFuture.get();
Copy link
Collaborator

Choose a reason for hiding this comment

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

Suggested change
ThrowableRunnable taskToEmit = completableFuture.get();
ThrowableRunnable<Exception> taskToEmit = completableFuture.get();

}
throw new GroupCommitException(
String.format("Group commit failed. Group: %s", parentGroup.get()), cause);
} catch (Throwable e) {
Copy link
Collaborator

Choose a reason for hiding this comment

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

Suggested change
} catch (Throwable e) {
} catch (Exception e) {


// Delegates the emit task to the client. The client receiving this task needs to handle the emit
// task.
void delegateTaskToWaiter(ThrowableRunnable task) {
Copy link
Collaborator

Choose a reason for hiding this comment

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

Suggested change
void delegateTaskToWaiter(ThrowableRunnable task) {
void delegateTaskToWaiter(ThrowableRunnable<Exception> task) {

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Thanks! Fixed it in 8658300


// This task is passed to only the first slot, so the slot will be resumed.
// Other slots will be blocked until `markAsXxxx()` is called.
ThrowableRunnable<GroupCommitException> taskForEmitterSlot =
Copy link
Collaborator

Choose a reason for hiding this comment

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

Suggested change
ThrowableRunnable<GroupCommitException> taskForEmitterSlot =
ThrowableRunnable<Exception> taskForEmitterSlot =

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Thanks! Fixed it in 8658300

Copy link
Contributor

@feeblefakie feeblefakie left a comment

Choose a reason for hiding this comment

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

LGTM! Thank you!

Copy link
Collaborator

@brfrn169 brfrn169 left a comment

Choose a reason for hiding this comment

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

LGTM! Thank you for your great work!

@brfrn169 brfrn169 merged commit cca9421 into master Apr 9, 2024
23 checks passed
@brfrn169 brfrn169 deleted the add-group-commit-package branch April 9, 2024 00:30
feeblefakie pushed a commit that referenced this pull request Apr 15, 2024
feeblefakie pushed a commit that referenced this pull request Apr 15, 2024
feeblefakie pushed a commit that referenced this pull request Apr 15, 2024
feeblefakie pushed a commit that referenced this pull request Apr 15, 2024
feeblefakie pushed a commit that referenced this pull request Apr 15, 2024
feeblefakie pushed a commit that referenced this pull request Apr 15, 2024
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
enhancement New feature or request
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

3 participants