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

@@ -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

Copy link
Contributor

Choose a reason for hiding this comment

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

Could you please share a place in code where raft awaits previous leader lease to expire prior to proposing the new 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.

This happens here [1] and [2]
[1] org.apache.ignite.raft.jraft.core.NodeImpl#handleElectionTimeout
[2] org.apache.ignite.raft.jraft.core.NodeImpl#handlePreVoteRequest
Elections don't start if current leader is active by lease.

Copy link
Contributor

Choose a reason for hiding this comment

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

So, it's required for getLeaderLeaseTimeoutMs to be >= maxClockSkew, right? Seems that it's not guaranteed because it's possible to set any value using org.apache.ignite.raft.jraft.option.NodeOptions#setElectionTimeoutMs. With defaults it should work though.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Not quite. We are choosing max of both, so the final value is safe to use:
Math.max(nodeOptions.getElectionTimeoutMs(), groupOptions.maxClockSkew())

…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