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

IGNITE-23304 #4821

Open
wants to merge 33 commits into
base: main
Choose a base branch
from
Open

IGNITE-23304 #4821

wants to merge 33 commits into from

Conversation

ascherbakoff
Copy link
Contributor

@ascherbakoff ascherbakoff commented Dec 3, 2024

This PR addresses the safe timestamp generation behavior for partition replication groups.

safeTs is an entity tied to majority based replication protocols and is used for serializable backups reads.
Each raft command is assigned a mototonic ts and a replica updates its local ts value on receiving replication commands.
All reads at safe ts are serializable
Currently safeTs is assigined on primary replica, which involves additional synchronizatoin (currently uses huge critical section) and involves retries (added latency).
Also it's bad from the pluggable replication point of view, because not all protocols require this concept.

Safe ts behavior was modified in the following way:

  1. Safe timestamp generation is moved outside primary replica to a replication layer, making it protocol specific. All request sycnhronization is removed from primary replica.
  2. Generated timestamp is applied by binary paching to a command then it enters raft pipeline on a leader.
  3. Added guaranties on monotonic ts generation if raft leader has changed:
    3.1 raft election timeout now accounts max clock skew. Then a new election starts on a node, it has local time higher than last generated safe ts.
    3.2 hlc is propagated in timoutnow requests, then a leader directly transfers ownership to other candidate to maintain proper clock ordering.
  4. safe timestamp reordering now counts as assertion condition which never should happen. corresponding error code is removed as user should never see it.

Benchmark results
oracle JDK 21.0.4, Xeon Silver 4314, aipersist engine (20G pagecache size)

  1. direct writes to storage (IGNITE_SKIP_REPLICATION_IN_BENCHMARK=true, IGNITE_SKIP_STORAGE_UPDATE_IN_BENCHMARK=false)
    master revision=32737c0dc9fcd0632ba37e2949a40b199429fddb

8 thread(new)

Benchmark (batch) (fsync) (partitionCount) Mode Cnt Score Error Units
UpsertKvBenchmark.upsert 1 false 1 thrpt 20 197936.874 ± 12727.709 ops/s

16 threads(new)

Benchmark (batch) (fsync) (partitionCount) Mode Cnt Score Error Units
UpsertKvBenchmark.upsert 1 false 1 thrpt 20 254981.169 ± 21278.635 ops/s

32 threads(new)

Benchmark (batch) (fsync) (partitionCount) Mode Cnt Score Error Units
UpsertKvBenchmark.upsert 1 false 1 thrpt 20 286127.032 ± 16145.256 ops/s

8 threads(old)

Benchmark (batch) (fsync) (partitionCount) Mode Cnt Score Error Units
UpsertKvBenchmark.upsert 1 false 1 thrpt 20 86624.141 ± 3472.632 ops/s

16 threads(old)

Benchmark (batch) (fsync) (partitionCount) Mode Cnt Score Error Units
UpsertKvBenchmark.upsert 1 false 1 thrpt 20 89446.504 ± 6623.490 ops/s

32 threads(old)

Benchmark (batch) (fsync) (partitionCount) Mode Cnt Score Error Units
UpsertKvBenchmark.upsert 1 false 1 thrpt 20 89516.016 ± 6092.740 ops/s

It's obvious old version has zero scaling on writing to partition.

  1. Full raft pipeline, same hardware
    LOGIT_STORAGE_ENABLED=true
    IGNITE_SKIP_REPLICATION_IN_BENCHMARK=false
    IGNITE_SKIP_STORAGE_UPDATE_IN_BENCHMARK=false

32 threads(new)

Benchmark (batch) (fsync) (partitionCount) Mode Cnt Score Error Units
UpsertKvBenchmark.upsert 1 false 32 thrpt 20 229083.089 ± 36856.962 ops/s

32 thread(old)

Benchmark (batch) (fsync) (partitionCount) Mode Cnt Score Error Units
UpsertKvBenchmark.upsert 1 false 32 thrpt 20 181908.090 ± 26821.026 ops/s

@@ -72,31 +63,40 @@ public final long nowLong() {
}

@Override
public final long currentLong() {
long current = currentTime();
public final long nowLong(HybridTimestamp causal) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Why do you rewrite the logic instead of reuse HLC#update method? Moreover the update method is better, because more frequently use increment.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes, it seems I unintentionally re-wrote update method.
Removed these methods and switched to existing "update".

*
* @return The timestamp.
*/
default @Nullable HybridTimestamp initiatorTime() {
Copy link
Contributor

Choose a reason for hiding this comment

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

  1. Do you mean that ReadCommand should also be initiatorTime aware?
  2. Do you mean that all WriteCommand should be initiatorTime aware?
  3. Why it's @nullable?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

  1. optional
  2. optional
  3. this is nullable because this field is optional (protocol dependent). but it will always be not null for raft + writecommand.

Copy link
Contributor

@sanpwc sanpwc Dec 20, 2024

Choose a reason for hiding this comment

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

  1. Do we ever set initiatorTime to any ReadCommand?
  2. Is it ever valid not to provide initiatorTime to corresponding commands in case of jraft?

this is nullable because this field is optional (protocol dependent).

Are we going to have anything besides jraft in nearest future (say a year)? If not I'd rather remove @nullable, because otherwise we may forget to set the value while using jraft.

but it will always be not null for raft + writecommand.

Why HybridTimestamp initiatorTime() is not in WriteCommand in that case?
BTW, do you mean that every WriteCommand should be initiatorTime aware?

Copy link
Contributor Author

@ascherbakoff ascherbakoff Dec 24, 2024

Choose a reason for hiding this comment

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

  1. No. I've moved initiatorTime to WriteCommand.
  2. It's valid, I was wrong before. It can be null for CMG for example and any other group not requring mvcc extensions.
    So it's not possible to remove @nullable in WriteCommand.
    Correctness is ensured by SafeTimestampPropagatingCommand, where this annotation is removed, so it's not possible to create SafeTimestampPropagatingCommand without initialTime.

Copy link
Contributor

Choose a reason for hiding this comment

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

As far as I understand SafeTimePropagatingCommand should always have initiatorTime and any other commands should never have initiatorTime, is that correct? If true, adding @nullable initiatorTime to WriteCommand seems like a bad design to me. I do understand that it's likely because of patching code in NodeImpl

            // Patch the command.
            if (event.done instanceof WriteCommandClosure) {
                WriteCommandClosure clo = (WriteCommandClosure) event.done;
                WriteCommand command = clo.command();
                HybridTimestamp timestamp = command.initiatorTime();

I guess that since you move safe time logic into raft you should also move SafeTimePropagatingCommand from
package org.apache.ignite.internal.replicator.command;
to
package org.apache.ignite.internal.raft;

Copy link
Contributor Author

@ascherbakoff ascherbakoff Dec 27, 2024

Choose a reason for hiding this comment

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

You are correct. I've tried to move initiatorTime to SafeTimeCommand, but failed due to dependency issues.
Should I create a ticket for refactoring ?

@@ -447,6 +447,10 @@ public boolean startRaftNode(
// Thread pools are shared by all raft groups.
NodeOptions nodeOptions = opts.copy();

// Then a new election starts on a node, it has local physical time higher than last generated safe ts
// because we wait out the clock skew.
nodeOptions.setElectionTimeoutMs(Math.max(nodeOptions.getElectionTimeoutMs(), groupOptions.maxClockSkew()));
Copy link
Contributor

Choose a reason for hiding this comment

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

How are you going to guarantee it has local physical time higher than last generated safe ts in case of immediate leader election?
E.g. if there is only one node in partition. (let's say that partition was rebalanced from A to B)
I'm not sure whether it's the only case of immediate leader election attempt.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Generally, leader lease timeout enforces this condition.
I know only one scenario, where manual ordering propagation is required, see below comment on timeoutnowrequest.
for a single node partition I see zero issues.
can you provide more details ?

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've investigated this scenario and ensured everything is ok, because:

  1. Then a configuration is changed from A to B, on new configuration commit A steps down and sends timeoutnowrequest to B
  2. If it dies before sending the request, B will elect self a leader after previos leader (A) lease timeout.

Added a new test for this scenario: org.apache.ignite.distributed.ReplicasSafeTimePropagationTest#testSafeTimeReorderingOnClusterShrink

Copy link
Contributor

Choose a reason for hiding this comment

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

B will elect self a leader after previos leader (A) lease timeout

Do you mean raft-leader-lease timeout or primary-replica-lease timeout here?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

raft-leader-lease

} else {
assert false : "Command was not found [cmd=" + command + ']';
}

if (applied[0]) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Agree with Roma, that decreases the readability.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

How do you suggest to fix this? Java doesn't support returning tuples unfortunately.

Copy link
Contributor

Choose a reason for hiding this comment

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

You may introduce a special class to return.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Refactored to return IgniteBiTuple.
Not sure if it became better.

Copy link
Contributor

Choose a reason for hiding this comment

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

Well, IgniteBiTuple doesn't sound as a special class, why not to introduce result wrapper something like

CommandProcessingResult
  Serializable result
  Boolean applied

explaining in javadoc why the class is needed and what applied means?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

The tuple works fine to me.

…nto ignite-23304

# Conflicts:
#	modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

6 participants