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-24242 Add RO/RW Transaction Timeout configuration schema #5093

Merged
merged 21 commits into from
Jan 30, 2025
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
Show all changes
21 commits
Select commit Hold shift + click to select a range
68f1fa8
IGNITE-24242 Add RO/RW Transaction Timeout configuration schema
PakhomovAlexander Jan 21, 2025
2e7c24c
Review fixes: javadoc and configurations naming
PakhomovAlexander Jan 23, 2025
c4fcc59
Use suppliers for ro/rw timeouts in TableManager, so if the config ar…
PakhomovAlexander Jan 24, 2025
03a0274
Use timeout from TransationOptions rather than from configuration.
PakhomovAlexander Jan 24, 2025
7dcb643
Fix config in test
PakhomovAlexander Jan 24, 2025
1e21f92
Fix test
PakhomovAlexander Jan 27, 2025
a34ded1
Update modules/transactions/src/main/java/org/apache/ignite/internal/…
PakhomovAlexander Jan 27, 2025
9b6203a
Merge branch 'IGNITE-24242' of github.com:unisonteam/ignite-3 into IG…
PakhomovAlexander Jan 27, 2025
17f37e4
Use TimeUnit for defaults in configuration
PakhomovAlexander Jan 27, 2025
4340ec3
Fix comment in constructor
PakhomovAlexander Jan 27, 2025
db2fcef
Fix test, add todo
PakhomovAlexander Jan 27, 2025
71ac52d
Fix checkstyle
PakhomovAlexander Jan 27, 2025
ba756bc
Add todos for the next ticket
PakhomovAlexander Jan 27, 2025
c05476d
Change default timeout for rw transactions to make tests pass
PakhomovAlexander Jan 28, 2025
fa7f5df
fix
PakhomovAlexander Jan 28, 2025
cc9cb48
Fix
PakhomovAlexander Jan 28, 2025
ae00d30
Change default RW timeout to make test pass
PakhomovAlexander Jan 29, 2025
e1ee039
Fix test
PakhomovAlexander Jan 29, 2025
2f1e409
Update modules/transactions/src/main/java/org/apache/ignite/internal/…
PakhomovAlexander Jan 30, 2025
c0b0e40
IGNITE-24242 Add RO/RW Transaction Timeout configuration schema
PakhomovAlexander Jan 30, 2025
2f13beb
Fix checkstyle
PakhomovAlexander Jan 30, 2025
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -273,6 +273,7 @@ private static class TestInternalTableImpl extends InternalTableImpl {
mock(PlacementDriver.class),
mock(TransactionInflights.class),
3_000,
3_000,
0,
null,
mock(StreamerReceiverRunner.class)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -347,6 +347,7 @@ public NetworkMessage clone() {
new TestPlacementDriver(clusterNode),
transactionInflights,
3_000,
3_000,
0,
null,
mock(StreamerReceiverRunner.class)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -418,7 +418,9 @@ public class TableManager implements IgniteTablesInternal, IgniteComponent {

private final PartitionReplicaLifecycleManager partitionReplicaLifecycleManager;

private long implicitTransactionTimeout;
private long roTransactionTimeout;
Copy link
Contributor

Choose a reason for hiding this comment

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

Tx timeout is tx specific attribute, meaning that tx1 will have 100ms as timeout and tx2 that is running at the same time 2000ms. In that case why roTransactionTimeout and rwTransactionTimeout are here?
Besides that, it's just an encapsulation leak - simply TableManager should not know about such things.
I'd rather say that on tx creation we should populate TransacionOptions with default timeout from cfg if it's not specified by the user and within tx flow use timeout from the tx context - no need to propagate txTimeout to InternalTableImpl etc.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Do you agree that implicitTransactionTimeout should be removed with this PR? If yes, then it is clear why these options are there. If we should keep implicitTransactionTimeout then, probably, I should rollback the change.

I suppose we should remove implicitTransactionTimeout and refactor TableManager as you suggested. Am I right?

Copy link
Contributor

Choose a reason for hiding this comment

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

+1 to remove implicitTransactionTimeout

Copy link
Contributor

@rpuch rpuch Jan 27, 2025

Choose a reason for hiding this comment

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

In that case why roTransactionTimeout and rwTransactionTimeout are here?

I thought these properties define default TX timeouts. A default is global and not specific to a transaction.

Copy link
Contributor

Choose a reason for hiding this comment

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

I suppose we should remove implicitTransactionTimeout and refactor TableManager as you suggested. Am I right?

Yes.


private long rwTransactionTimeout;

private int attemptsObtainLock;

Expand Down Expand Up @@ -679,7 +681,9 @@ public CompletableFuture<Void> startAsync(ComponentContext componentContext) {

partitionReplicatorNodeRecovery.start();

implicitTransactionTimeout = txCfg.implicitTransactionTimeout().value();
roTransactionTimeout = txCfg.roTimeout().value();
rwTransactionTimeout = txCfg.rwTimeout().value();

attemptsObtainLock = txCfg.attemptsObtainLock().value();

executorInclinedPlacementDriver.listen(PrimaryReplicaEvent.PRIMARY_REPLICA_EXPIRED, this::onPrimaryReplicaExpired);
Expand Down Expand Up @@ -1628,7 +1632,8 @@ private TableImpl createTableImpl(
observableTimestampTracker,
executorInclinedPlacementDriver,
transactionInflights,
implicitTransactionTimeout,
roTransactionTimeout,
rwTransactionTimeout,
attemptsObtainLock,
this::streamerFlushExecutor,
Objects.requireNonNull(streamerReceiverRunner)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -198,8 +198,9 @@ public class InternalTableImpl implements InternalTable {
/** Map update guarded by {@link #updatePartitionMapsMux}. */
private volatile Int2ObjectMap<PendingComparableValuesTracker<Long, Void>> storageIndexTrackerByPartitionId = emptyMap();

/** Implicit transaction timeout. */
private final long implicitTransactionTimeout;
private final long roTransactionTimeout;

private final long rwTransactionTimeout;

/** Attempts to take lock. */
private final int attemptsObtainLock;
Expand All @@ -218,7 +219,8 @@ public class InternalTableImpl implements InternalTable {
* @param clockService A hybrid logical clock service.
* @param placementDriver Placement driver.
* @param transactionInflights Transaction inflights.
* @param implicitTransactionTimeout Implicit transaction timeout.
* @param roTransactionTimeout Read-only transaction timeout.
* @param rwTransactionTimeout RW transaction timeout.
* @param attemptsObtainLock Attempts to take lock.
*/
public InternalTableImpl(
Expand All @@ -234,7 +236,8 @@ public InternalTableImpl(
HybridTimestampTracker observableTimestampTracker,
PlacementDriver placementDriver,
TransactionInflights transactionInflights,
long implicitTransactionTimeout,
long roTransactionTimeout,
long rwTransactionTimeout,
int attemptsObtainLock,
Supplier<ScheduledExecutorService> streamerFlushExecutor,
StreamerReceiverRunner streamerReceiverRunner
Expand All @@ -251,7 +254,8 @@ public InternalTableImpl(
this.observableTimestampTracker = observableTimestampTracker;
this.placementDriver = placementDriver;
this.transactionInflights = transactionInflights;
this.implicitTransactionTimeout = implicitTransactionTimeout;
this.roTransactionTimeout = roTransactionTimeout;
this.rwTransactionTimeout = rwTransactionTimeout;
this.attemptsObtainLock = attemptsObtainLock;
this.streamerFlushExecutor = streamerFlushExecutor;
this.streamerReceiverRunner = streamerReceiverRunner;
Expand Down Expand Up @@ -368,9 +372,11 @@ private <R> CompletableFuture<R> enlistInTx(
return postEnlist(fut, false, actualTx, actualTx.implicit()).handle((r, e) -> {
if (e != null) {
if (actualTx.implicit()) {
long timeout = actualTx.isReadOnly() ? roTransactionTimeout : rwTransactionTimeout;

long ts = (txStartTs == null) ? actualTx.startTimestamp().getPhysical() : txStartTs;

if (exceptionAllowsImplicitTxRetry(e) && coarseCurrentTimeMillis() - ts < implicitTransactionTimeout) {
if (exceptionAllowsImplicitTxRetry(e) && coarseCurrentTimeMillis() - ts < timeout) {
Copy link
Contributor

Choose a reason for hiding this comment

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

How about extracting the if condition to a method? It's used a few times in this same form

return enlistInTx(row, null, fac, noWriteChecker, ts);
}
}
Expand Down Expand Up @@ -486,9 +492,11 @@ private <T> CompletableFuture<T> enlistInTx(
return postEnlist(fut, actualTx.implicit() && !singlePart, actualTx, full).handle((r, e) -> {
if (e != null) {
if (actualTx.implicit()) {
long timeout = actualTx.isReadOnly() ? roTransactionTimeout : rwTransactionTimeout;

long ts = (txStartTs == null) ? actualTx.startTimestamp().getPhysical() : txStartTs;

if (exceptionAllowsImplicitTxRetry(e) && coarseCurrentTimeMillis() - ts < implicitTransactionTimeout) {
if (exceptionAllowsImplicitTxRetry(e) && coarseCurrentTimeMillis() - ts < timeout) {
return enlistInTx(keyRows, null, fac, reducer, noOpChecker, ts);
}
}
Expand Down Expand Up @@ -1185,9 +1193,11 @@ private CompletableFuture<Void> updateAllWithRetry(
// Will be finished in one RTT.
return postEnlist(fut, false, tx, true).handle((r, e) -> {
if (e != null) {
long timeout = tx.isReadOnly() ? roTransactionTimeout : rwTransactionTimeout;

long ts = (txStartTs == null) ? tx.startTimestamp().getPhysical() : txStartTs;

if (exceptionAllowsImplicitTxRetry(e) && coarseCurrentTimeMillis() - ts < implicitTransactionTimeout) {
if (exceptionAllowsImplicitTxRetry(e) && coarseCurrentTimeMillis() - ts < timeout) {
return updateAllWithRetry(rows, deleted, partition, ts);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -211,6 +211,7 @@ void setUp(
new TransactionInflights(placementDriver, clockService),
0,
0,
0,
() -> null,
mock(StreamerReceiverRunner.class)
);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -73,6 +73,7 @@ void testUpdatePartitionTrackers() {
mock(PlacementDriver.class),
mock(TransactionInflights.class),
3_000,
3_000,
0,
null,
mock(StreamerReceiverRunner.class)
Expand Down Expand Up @@ -123,6 +124,7 @@ void testRowBatchByPartitionId() {
mock(PlacementDriver.class),
mock(TransactionInflights.class),
3_000,
3_000,
0,
null,
null
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -648,6 +648,7 @@ public TableViewInternal startTable(String tableName, SchemaDescriptor schemaDes
placementDriver,
clientTransactionInflights,
500,
500,
0,
null,
mock(StreamerReceiverRunner.class)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -269,6 +269,7 @@ public DummyInternalTableImpl(
placementDriver,
transactionInflights,
3_000,
3_000,
0,
null,
mock(StreamerReceiverRunner.class)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -85,7 +85,8 @@ protected void customizeInitParameters(InitParametersBuilder builder) {

builder.clusterConfiguration("ignite {"
+ " transaction: {"
+ " implicitTransactionTimeout: 30000,"
+ " roTimeout: 30000,"
+ " rwTimeout: 30000,"
+ " txnResourceTtl: 2"
+ " },"
+ " replication: {"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,10 +41,29 @@ public class TransactionConfigurationSchema {
@Value(hasDefault = true)
public final long timeout = 10_000;
Copy link
Contributor

Choose a reason for hiding this comment

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

This still looks like a breaking change. Let's consult @ptupitsyn whether we are allowed to directly remove a configuration property

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. But why should we consult Pavel about changing the property we've added a couple of weeks ago?

Copy link
Contributor

Choose a reason for hiding this comment

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

Because 3.0.0 already contains transaction.timeout config property. If we just remove it, then our next release will potentially break them.


/** Timeout for implicit transactions (milliseconds). */
@Range(min = 1)
@Value(hasDefault = true)
public final long implicitTransactionTimeout = 3_000;
public final long minRoTimeout = 1;

@Range(min = 1)
@Value(hasDefault = true)
public final long maxRoTimeout = Long.MAX_VALUE;
Copy link
Contributor

Choose a reason for hiding this comment

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

Why it's not equal to dataAvailabilityTime?

Copy link
Contributor

@rpuch rpuch Jan 23, 2025

Choose a reason for hiding this comment

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

(deleted as irrelevant)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

property deleted

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 purpose of
minRoTimeout maxRoTimeout minRwTimeout maxRwTimeout ?
It doesn't make sense to me.

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 agree. Will drop these properties.

Copy link
Contributor

Choose a reason for hiding this comment

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

Here and there: for such properties it's reasonable to have milis(or any other) suffix.

Copy link
Contributor

Choose a reason for hiding this comment

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

There was a proposal to switch duration-related properties type to Duration and allow the user specify units in the config. Having this in mind, it seems that we should not include units in property names.

My comment only makes sense if we still want to have that feature in the configuration.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

property deleted


@Range(min = 1)
@Value(hasDefault = true)
public final long minRwTimeout = 1;

@Range(min = 1)
@Value(hasDefault = true)
public final long maxRwTimeout = Long.MAX_VALUE;
Copy link
Contributor

Choose a reason for hiding this comment

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

Is it too much for default max value?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

property deleted


@Range(min = 1)
@Value(hasDefault = true)
public final long roTimeout = 10_000;
Copy link
Contributor

Choose a reason for hiding this comment

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

How these defaults are calculated?
They seem too low.
I suggest 10 minutes for RO txns, and 30 seconds for RW txns

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 suggestion.


@Range(min = 1)
@Value(hasDefault = true)
public final long rwTimeout = 3_000;
Copy link
Contributor

Choose a reason for hiding this comment

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

There is already timeout which defines default timeouts. Now, rwTimeout and roTimeout are added. Do they serve the same purpose? If yes, timeout should probably be removed.

But if it's removed, it's a breaking change (as 3.0 code freeze has already happened) and it should be handled in a different way, probably by declaring it as deprecated and, maybe, using the existing value for some time?

Copy link
Contributor

Choose a reason for hiding this comment

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

I believe we still can do breaking changes in configuration APIs.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

timeout removed


/** A transaction tries to take lock several times until it throws an exception {@lonk org.apache.ignite.tx.TransactionException}. */
@Range(min = 0)
Expand Down