diff --git a/modules/api/src/main/java/org/apache/ignite/tx/TransactionOptions.java b/modules/api/src/main/java/org/apache/ignite/tx/TransactionOptions.java
index 2b785ccd764..96594b5f4fd 100644
--- a/modules/api/src/main/java/org/apache/ignite/tx/TransactionOptions.java
+++ b/modules/api/src/main/java/org/apache/ignite/tx/TransactionOptions.java
@@ -21,14 +21,14 @@
* Ignite transaction options.
*/
public class TransactionOptions {
- /** Transaction timeout. */
+ /** Transaction timeout. 0 means 'use default timeout'. */
private long timeoutMillis = 0;
/** Read-only transaction. */
private boolean readOnly = false;
/**
- * Returns transaction timeout, in milliseconds.
+ * Returns transaction timeout, in milliseconds. 0 means 'use default timeout'.
*
* @return Transaction timeout, in milliseconds.
*/
@@ -36,13 +36,22 @@ public long timeoutMillis() {
return timeoutMillis;
}
+ // TODO: remove a note that timeouts are not supported for RW after IGNITE-15936 is implemented.
/**
* Sets transaction timeout, in milliseconds.
*
- * @param timeoutMillis Transaction timeout, in milliseconds.
+ * @param timeoutMillis Transaction timeout, in milliseconds. Cannot be negative; 0 means 'use default timeout'.
+ *
+ * - For RO transactions, the default timeout is configured via ignite.transaction.timeout configuration property.
+ * - For RW transactions, timeouts are not supported yet.
+ *
* @return {@code this} for chaining.
*/
public TransactionOptions timeoutMillis(long timeoutMillis) {
+ if (timeoutMillis < 0) {
+ throw new IllegalArgumentException("Negative timeoutMillis: " + timeoutMillis);
+ }
+
this.timeoutMillis = timeoutMillis;
return this;
diff --git a/modules/api/src/test/java/org/apache/ignite/tx/TransactionOptionsTest.java b/modules/api/src/test/java/org/apache/ignite/tx/TransactionOptionsTest.java
new file mode 100644
index 00000000000..3f305415329
--- /dev/null
+++ b/modules/api/src/test/java/org/apache/ignite/tx/TransactionOptionsTest.java
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.tx;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.is;
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
+import static org.junit.jupiter.api.Assertions.assertSame;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+import org.junit.jupiter.api.Test;
+
+class TransactionOptionsTest {
+ @Test
+ void readOnlyIsFalseByDefault() {
+ assertThat(new TransactionOptions().readOnly(), is(false));
+ }
+
+ @Test
+ void readOnlyStatusIsSet() {
+ var options = new TransactionOptions();
+
+ options.readOnly(true);
+
+ assertThat(options.readOnly(), is(true));
+ }
+
+ @Test
+ void readOnlySetterReturnsSameObject() {
+ var options = new TransactionOptions();
+
+ TransactionOptions afterSetting = options.readOnly(true);
+
+ assertSame(options, afterSetting);
+ }
+
+ @Test
+ void timeoutIsZeroByDefault() {
+ assertThat(new TransactionOptions().timeoutMillis(), is(0L));
+ }
+
+ @Test
+ void timeoutIsSet() {
+ var options = new TransactionOptions();
+
+ options.timeoutMillis(3333);
+
+ assertThat(options.timeoutMillis(), is(3333L));
+ }
+
+ @Test
+ void timeoutSetterReturnsSameObject() {
+ var options = new TransactionOptions();
+
+ TransactionOptions afterSetting = options.timeoutMillis(3333);
+
+ assertSame(options, afterSetting);
+ }
+
+ @Test
+ void positiveTimeoutIsAllowed() {
+ assertDoesNotThrow(() -> new TransactionOptions().timeoutMillis(0));
+ }
+
+ @Test
+ void zeroTimeoutIsAllowed() {
+ assertDoesNotThrow(() -> new TransactionOptions().timeoutMillis(0));
+ }
+
+ @Test
+ void negativeTimeoutIsRejected() {
+ IllegalArgumentException ex = assertThrows(IllegalArgumentException.class, () -> new TransactionOptions().timeoutMillis(-1));
+
+ assertThat(ex.getMessage(), is("Negative timeoutMillis: -1"));
+ }
+}
diff --git a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/ClientHandlerModule.java b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/ClientHandlerModule.java
index 18ec268eb94..6b3c71c5123 100644
--- a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/ClientHandlerModule.java
+++ b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/ClientHandlerModule.java
@@ -27,7 +27,6 @@
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelFutureListener;
-import io.netty.channel.ChannelHandler;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelOption;
import io.netty.handler.ssl.SslContext;
@@ -130,7 +129,7 @@ public class ClientHandlerModule implements IgniteComponent {
@TestOnly
@SuppressWarnings("unused")
- private volatile ChannelHandler handler;
+ private volatile ClientInboundMessageHandler handler;
/**
* Constructor.
@@ -396,4 +395,9 @@ private ClientInboundMessageHandler createInboundMessageHandler(ClientConnectorV
partitionOperationsExecutor
);
}
+
+ @TestOnly
+ public ClientInboundMessageHandler handler() {
+ return handler;
+ }
}
diff --git a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/ClientInboundMessageHandler.java b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/ClientInboundMessageHandler.java
index c5dc979615f..c7e83e5580b 100644
--- a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/ClientInboundMessageHandler.java
+++ b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/ClientInboundMessageHandler.java
@@ -139,6 +139,7 @@
import org.apache.ignite.security.exception.UnsupportedAuthenticationTypeException;
import org.apache.ignite.sql.SqlBatchException;
import org.jetbrains.annotations.Nullable;
+import org.jetbrains.annotations.TestOnly;
/**
* Handles messages from thin clients.
@@ -1139,4 +1140,9 @@ private static Set authenticationEventsToSubscribe() {
AuthenticationEvent.USER_REMOVED
);
}
+
+ @TestOnly
+ public ClientResourceRegistry resources() {
+ return resources;
+ }
}
diff --git a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/JdbcQueryEventHandlerImpl.java b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/JdbcQueryEventHandlerImpl.java
index 3662ce2abce..4614a044236 100644
--- a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/JdbcQueryEventHandlerImpl.java
+++ b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/JdbcQueryEventHandlerImpl.java
@@ -67,6 +67,7 @@
import org.apache.ignite.internal.sql.engine.property.SqlPropertiesHelper;
import org.apache.ignite.internal.tx.HybridTimestampTracker;
import org.apache.ignite.internal.tx.InternalTransaction;
+import org.apache.ignite.internal.tx.InternalTxOptions;
import org.apache.ignite.internal.tx.TxManager;
import org.apache.ignite.internal.util.AsyncCursor.BatchedResult;
import org.apache.ignite.lang.CancelHandle;
@@ -186,8 +187,8 @@ public HybridTimestampTracker getTimestampTracker() {
}
private static SqlProperties createProperties(
- JdbcStatementType stmtType,
- boolean multiStatement,
+ JdbcStatementType stmtType,
+ boolean multiStatement,
ZoneId timeZoneId,
long queryTimeoutMillis
) {
@@ -452,7 +453,7 @@ ZoneId timeZoneId() {
* @return Transaction associated with the current connection.
*/
InternalTransaction getOrStartTransaction(HybridTimestampTracker timestampProvider) {
- return tx == null ? tx = txManager.begin(timestampProvider, false) : tx;
+ return tx == null ? tx = txManager.beginExplicitRw(timestampProvider, InternalTxOptions.defaults()) : tx;
}
/**
diff --git a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTableCommon.java b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTableCommon.java
index fdc1aa3b71c..e1d156a7d2d 100644
--- a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTableCommon.java
+++ b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTableCommon.java
@@ -44,6 +44,7 @@
import org.apache.ignite.internal.table.TableViewInternal;
import org.apache.ignite.internal.tx.HybridTimestampTracker;
import org.apache.ignite.internal.tx.InternalTransaction;
+import org.apache.ignite.internal.tx.InternalTxOptions;
import org.apache.ignite.internal.tx.TxManager;
import org.apache.ignite.internal.type.DecimalNativeType;
import org.apache.ignite.internal.type.NativeType;
@@ -439,32 +440,53 @@ public static TableNotFoundException tableIdNotFoundException(Integer tableId) {
if (tx == null) {
// Implicit transactions do not use an observation timestamp because RW never depends on it, and implicit RO is always direct.
// The direct transaction uses a current timestamp on the primary replica by definition.
- tx = startTx(out, txManager, null, true, readOnly);
+ tx = startImplicitTx(out, txManager, null, readOnly);
}
return tx;
}
/**
- * Start a transaction.
+ * Starts an explicit transaction.
*
* @param out Packer.
* @param txManager Ignite transactions.
* @param currentTs Current observation timestamp or {@code null} if it is not defined.
- * @param implicit Implicit transaction flag.
* @param readOnly Read only flag.
+ * @param options Transaction options.
* @return Transaction.
*/
- public static InternalTransaction startTx(
+ public static InternalTransaction startExplicitTx(
+ ClientMessagePacker out,
+ TxManager txManager,
+ @Nullable HybridTimestamp currentTs,
+ boolean readOnly,
+ InternalTxOptions options
+ ) {
+ return txManager.beginExplicit(
+ HybridTimestampTracker.clientTracker(currentTs, ts -> {}),
+ readOnly,
+ options
+ );
+ }
+
+ /**
+ * Starts an implicit transaction.
+ *
+ * @param out Packer.
+ * @param txManager Ignite transactions.
+ * @param currentTs Current observation timestamp or {@code null} if it is not defined.
+ * @param readOnly Read only flag.
+ * @return Transaction.
+ */
+ public static InternalTransaction startImplicitTx(
ClientMessagePacker out,
TxManager txManager,
@Nullable HybridTimestamp currentTs,
- boolean implicit,
boolean readOnly
) {
- return txManager.begin(
- HybridTimestampTracker.clientTracker(currentTs, implicit ? out::meta : ts -> {}),
- implicit,
+ return txManager.beginImplicit(
+ HybridTimestampTracker.clientTracker(currentTs, out::meta),
readOnly
);
}
diff --git a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/tx/ClientTransactionBeginRequest.java b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/tx/ClientTransactionBeginRequest.java
index 5237f6bf91e..6f804045b05 100644
--- a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/tx/ClientTransactionBeginRequest.java
+++ b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/tx/ClientTransactionBeginRequest.java
@@ -17,7 +17,7 @@
package org.apache.ignite.client.handler.requests.tx;
-import static org.apache.ignite.client.handler.requests.table.ClientTableCommon.startTx;
+import static org.apache.ignite.client.handler.requests.table.ClientTableCommon.startExplicitTx;
import java.util.concurrent.CompletableFuture;
import org.apache.ignite.client.handler.ClientHandlerMetricSource;
@@ -27,6 +27,7 @@
import org.apache.ignite.internal.client.proto.ClientMessageUnpacker;
import org.apache.ignite.internal.hlc.HybridTimestamp;
import org.apache.ignite.internal.lang.IgniteInternalCheckedException;
+import org.apache.ignite.internal.tx.InternalTxOptions;
import org.apache.ignite.internal.tx.TxManager;
import org.jetbrains.annotations.Nullable;
@@ -52,12 +53,20 @@ public class ClientTransactionBeginRequest {
ClientHandlerMetricSource metrics
) throws IgniteInternalCheckedException {
boolean readOnly = in.unpackBoolean();
+ long timeoutMillis = in.unpackLong();
- // Timestamp makes sense only for read-only transactions.
- HybridTimestamp observableTs = readOnly ? HybridTimestamp.nullableHybridTimestamp(in.unpackLong()) : null;
+ HybridTimestamp observableTs = null;
+ if (readOnly) {
+ // Timestamp makes sense only for read-only transactions.
+ observableTs = HybridTimestamp.nullableHybridTimestamp(in.unpackLong());
+ }
+
+ InternalTxOptions txOptions = InternalTxOptions.builder()
+ .timeoutMillis(timeoutMillis)
+ .build();
// NOTE: we don't use beginAsync here because it is synchronous anyway.
- var tx = startTx(out, txManager, observableTs, false, readOnly);
+ var tx = startExplicitTx(out, txManager, observableTs, readOnly, txOptions);
if (readOnly) {
// For read-only tx, override observable timestamp that we send to the client:
diff --git a/modules/client-handler/src/test/java/org/apache/ignite/client/handler/JdbcQueryEventHandlerImplTest.java b/modules/client-handler/src/test/java/org/apache/ignite/client/handler/JdbcQueryEventHandlerImplTest.java
index 69f7e6df60a..4df09296b48 100644
--- a/modules/client-handler/src/test/java/org/apache/ignite/client/handler/JdbcQueryEventHandlerImplTest.java
+++ b/modules/client-handler/src/test/java/org/apache/ignite/client/handler/JdbcQueryEventHandlerImplTest.java
@@ -29,7 +29,6 @@
import static org.hamcrest.Matchers.notNullValue;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
@@ -133,7 +132,7 @@ public void contextClosedDuringBatchQuery() throws Exception {
CountDownLatch registryCloseLatch = new CountDownLatch(1);
long connectionId = acquireConnectionId();
- when(txManager.begin(any(), eq(false))).thenAnswer(v -> {
+ when(txManager.beginExplicitRw(any(), any())).thenAnswer(v -> {
registryCloseLatch.countDown();
assertThat(startTxLatch.await(timeout, TimeUnit.SECONDS), is(true));
@@ -161,13 +160,13 @@ public void explicitTxRollbackOnCloseRegistry() {
InternalTransaction tx = mock(InternalTransaction.class);
when(tx.rollbackAsync()).thenReturn(nullCompletedFuture());
- when(txManager.begin(any(), eq(false))).thenReturn(tx);
+ when(txManager.beginExplicitRw(any(), any())).thenReturn(tx);
long connectionId = acquireConnectionId();
await(eventHandler.batchAsync(connectionId, createExecuteBatchRequest("x", "UPDATE 1")));
- verify(txManager).begin(any(), eq(false));
+ verify(txManager).beginExplicitRw(any(), any());
verify(tx, times(0)).rollbackAsync();
resourceRegistry.close();
@@ -183,10 +182,10 @@ public void singleTxUsedForMultipleOperations() {
InternalTransaction tx = mock(InternalTransaction.class);
when(tx.commitAsync()).thenReturn(nullCompletedFuture());
when(tx.rollbackAsync()).thenReturn(nullCompletedFuture());
- when(txManager.begin(any(), eq(false))).thenReturn(tx);
+ when(txManager.beginExplicitRw(any(), any())).thenReturn(tx);
long connectionId = acquireConnectionId();
- verify(txManager, times(0)).begin(any(), eq(false));
+ verify(txManager, times(0)).beginExplicitRw(any(), any());
String schema = "schema";
JdbcStatementType type = JdbcStatementType.SELECT_STATEMENT_TYPE;
@@ -194,21 +193,21 @@ public void singleTxUsedForMultipleOperations() {
await(eventHandler.queryAsync(
connectionId, createExecuteRequest(schema, "SELECT 1", type)
));
- verify(txManager, times(1)).begin(any(), eq(false));
+ verify(txManager, times(1)).beginExplicitRw(any(), any());
await(eventHandler.batchAsync(connectionId, createExecuteBatchRequest("schema", "UPDATE 1", "UPDATE 2")));
- verify(txManager, times(1)).begin(any(), eq(false));
+ verify(txManager, times(1)).beginExplicitRw(any(), any());
await(eventHandler.finishTxAsync(connectionId, false));
verify(tx).rollbackAsync();
await(eventHandler.batchAsync(connectionId, createExecuteBatchRequest("schema", "UPDATE 1", "UPDATE 2")));
- verify(txManager, times(2)).begin(any(), eq(false));
+ verify(txManager, times(2)).beginExplicitRw(any(), any());
await(eventHandler.queryAsync(
connectionId, createExecuteRequest(schema, "SELECT 2", type)
));
- verify(txManager, times(2)).begin(any(), eq(false));
+ verify(txManager, times(2)).beginExplicitRw(any(), any());
await(eventHandler.batchAsync(connectionId, createExecuteBatchRequest("schema", "UPDATE 3", "UPDATE 4")));
- verify(txManager, times(2)).begin(any(), eq(false));
+ verify(txManager, times(2)).beginExplicitRw(any(), any());
await(eventHandler.finishTxAsync(connectionId, true));
verify(tx).commitAsync();
@@ -223,7 +222,7 @@ void simpleQueryCancellation() {
long connectionId = acquireConnectionId();
- JdbcQueryExecuteRequest executeRequest = createExecuteRequest("schema", "SELECT 1", JdbcStatementType.SELECT_STATEMENT_TYPE);
+ JdbcQueryExecuteRequest executeRequest = createExecuteRequest("schema", "SELECT 1", JdbcStatementType.SELECT_STATEMENT_TYPE);
CompletableFuture extends Response> resultFuture = eventHandler.queryAsync(connectionId, executeRequest);
diff --git a/modules/client/src/main/java/org/apache/ignite/internal/client/tx/ClientLazyTransaction.java b/modules/client/src/main/java/org/apache/ignite/internal/client/tx/ClientLazyTransaction.java
index 133ec00f7b2..6e727999d41 100644
--- a/modules/client/src/main/java/org/apache/ignite/internal/client/tx/ClientLazyTransaction.java
+++ b/modules/client/src/main/java/org/apache/ignite/internal/client/tx/ClientLazyTransaction.java
@@ -165,7 +165,10 @@ private synchronized CompletableFuture ensureStarted(
return tx0;
}
- ClientTransaction startedTx() {
+ /**
+ * Returns actual {@link ClientTransaction} started by this transaction.
+ */
+ public ClientTransaction startedTx() {
var tx0 = tx;
assert tx0 != null : "Transaction is not started";
diff --git a/modules/client/src/main/java/org/apache/ignite/internal/client/tx/ClientTransactions.java b/modules/client/src/main/java/org/apache/ignite/internal/client/tx/ClientTransactions.java
index 325ed680a7c..ece4c13b584 100644
--- a/modules/client/src/main/java/org/apache/ignite/internal/client/tx/ClientTransactions.java
+++ b/modules/client/src/main/java/org/apache/ignite/internal/client/tx/ClientTransactions.java
@@ -62,9 +62,9 @@ static CompletableFuture beginAsync(
@Nullable String preferredNodeName,
@Nullable TransactionOptions options,
long observableTimestamp) {
- if (options != null && options.timeoutMillis() != 0) {
+ if (options != null && options.timeoutMillis() != 0 && !options.readOnly()) {
// TODO: IGNITE-16193
- throw new UnsupportedOperationException("Timeouts are not supported yet");
+ throw new UnsupportedOperationException("Timeouts are not supported yet for RW transactions");
}
boolean readOnly = options != null && options.readOnly();
@@ -73,6 +73,7 @@ static CompletableFuture beginAsync(
ClientOp.TX_BEGIN,
w -> {
w.out().packBoolean(readOnly);
+ w.out().packLong(options == null ? 0 : options.timeoutMillis());
w.out().packLong(observableTimestamp);
},
r -> readTx(r, readOnly),
diff --git a/modules/client/src/test/java/org/apache/ignite/client/fakes/FakeTxManager.java b/modules/client/src/test/java/org/apache/ignite/client/fakes/FakeTxManager.java
index eb5dbeb74e7..8cefbecb722 100644
--- a/modules/client/src/test/java/org/apache/ignite/client/fakes/FakeTxManager.java
+++ b/modules/client/src/test/java/org/apache/ignite/client/fakes/FakeTxManager.java
@@ -31,9 +31,9 @@
import org.apache.ignite.internal.replicator.TablePartitionId;
import org.apache.ignite.internal.tx.HybridTimestampTracker;
import org.apache.ignite.internal.tx.InternalTransaction;
+import org.apache.ignite.internal.tx.InternalTxOptions;
import org.apache.ignite.internal.tx.LockManager;
import org.apache.ignite.internal.tx.TxManager;
-import org.apache.ignite.internal.tx.TxPriority;
import org.apache.ignite.internal.tx.TxState;
import org.apache.ignite.internal.tx.TxStateMeta;
import org.apache.ignite.network.ClusterNode;
@@ -63,17 +63,16 @@ public CompletableFuture stopAsync(ComponentContext componentContext) {
}
@Override
- public InternalTransaction begin(HybridTimestampTracker tracker, boolean implicit) {
- return begin(tracker, implicit, false);
+ public InternalTransaction beginImplicit(HybridTimestampTracker timestampTracker, boolean readOnly) {
+ return begin(timestampTracker, true, readOnly, InternalTxOptions.defaults());
}
@Override
- public InternalTransaction begin(HybridTimestampTracker timestampTracker, boolean implicit, boolean readOnly) {
- return begin(timestampTracker, implicit, readOnly, TxPriority.NORMAL);
+ public InternalTransaction beginExplicit(HybridTimestampTracker timestampTracker, boolean readOnly, InternalTxOptions txOptions) {
+ return begin(timestampTracker, false, readOnly, txOptions);
}
- @Override
- public InternalTransaction begin(HybridTimestampTracker tracker, boolean implicit, boolean readOnly, TxPriority priority) {
+ private InternalTransaction begin(HybridTimestampTracker tracker, boolean implicit, boolean readOnly, InternalTxOptions options) {
return new InternalTransaction() {
private final UUID id = UUID.randomUUID();
diff --git a/modules/distribution-zones/src/integrationTest/java/org/apache/ignite/internal/rebalance/ItRebalanceDistributedTest.java b/modules/distribution-zones/src/integrationTest/java/org/apache/ignite/internal/rebalance/ItRebalanceDistributedTest.java
index 76bb0b0c87d..383308447c4 100644
--- a/modules/distribution-zones/src/integrationTest/java/org/apache/ignite/internal/rebalance/ItRebalanceDistributedTest.java
+++ b/modules/distribution-zones/src/integrationTest/java/org/apache/ignite/internal/rebalance/ItRebalanceDistributedTest.java
@@ -331,7 +331,7 @@ public class ItRebalanceDistributedTest extends BaseIgniteAbstractTest {
private Path workDir;
@InjectExecutorService
- private static ScheduledExecutorService commonScheduledExecutorService;
+ private ScheduledExecutorService commonScheduledExecutorService;
private StaticNodeFinder finder;
@@ -1411,7 +1411,8 @@ private class Node {
new TestLocalRwTxCounter(),
resourcesRegistry,
transactionInflights,
- lowWatermark
+ lowWatermark,
+ commonScheduledExecutorService
);
rebalanceScheduler = new ScheduledThreadPoolExecutor(REBALANCE_SCHEDULER_POOL_SIZE,
diff --git a/modules/partition-replicator/src/integrationTest/java/org/apache/ignite/internal/partition/replicator/ItReplicaLifecycleTest.java b/modules/partition-replicator/src/integrationTest/java/org/apache/ignite/internal/partition/replicator/ItReplicaLifecycleTest.java
index 6280a5e0f6c..972dd217c59 100644
--- a/modules/partition-replicator/src/integrationTest/java/org/apache/ignite/internal/partition/replicator/ItReplicaLifecycleTest.java
+++ b/modules/partition-replicator/src/integrationTest/java/org/apache/ignite/internal/partition/replicator/ItReplicaLifecycleTest.java
@@ -1237,7 +1237,8 @@ public CompletableFuture invoke(
new TestLocalRwTxCounter(),
resourcesRegistry,
transactionInflights,
- lowWatermark
+ lowWatermark,
+ threadPoolsManager.commonScheduler()
);
replicaManager = new ReplicaManager(
diff --git a/modules/platforms/cpp/ignite/client/detail/transaction/transactions_impl.h b/modules/platforms/cpp/ignite/client/detail/transaction/transactions_impl.h
index cbaedbb0597..7c00c610a5c 100644
--- a/modules/platforms/cpp/ignite/client/detail/transaction/transactions_impl.h
+++ b/modules/platforms/cpp/ignite/client/detail/transaction/transactions_impl.h
@@ -57,6 +57,7 @@ class transactions_impl {
IGNITE_API void begin_async(ignite_callback callback) {
auto writer_func = [this](protocol::writer &writer) {
writer.write_bool(false); // readOnly.
+ writer.write(std::int64_t(0)); // timeoutMillis.
writer.write(m_connection->get_observable_timestamp());
};
diff --git a/modules/platforms/cpp/ignite/odbc/sql_connection.cpp b/modules/platforms/cpp/ignite/odbc/sql_connection.cpp
index d043b3b536a..ea707d48799 100644
--- a/modules/platforms/cpp/ignite/odbc/sql_connection.cpp
+++ b/modules/platforms/cpp/ignite/odbc/sql_connection.cpp
@@ -433,6 +433,7 @@ void sql_connection::transaction_start() {
network::data_buffer_owning response =
sync_request(protocol::client_operation::TX_BEGIN, [&](protocol::writer &writer) {
writer.write_bool(false); // read_only.
+ writer.write(std::int64_t(0)); // timeoutMillis.
});
protocol::reader reader(response.get_bytes_view());
diff --git a/modules/platforms/dotnet/Apache.Ignite.Tests/FakeServer.cs b/modules/platforms/dotnet/Apache.Ignite.Tests/FakeServer.cs
index c0bdd678ac2..550e178434a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Tests/FakeServer.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Tests/FakeServer.cs
@@ -309,6 +309,7 @@ protected override void Handle(Socket handler, CancellationToken cancellationTok
case ClientOp.TxBegin:
reader.Skip(); // Read only.
+ reader.Skip(); // TimeoutMillis.
LastClientObservableTimestamp = reader.ReadInt64();
Send(handler, requestId, new byte[] { 0 }.AsMemory());
diff --git a/modules/platforms/dotnet/Apache.Ignite/Internal/Transactions/LazyTransaction.cs b/modules/platforms/dotnet/Apache.Ignite/Internal/Transactions/LazyTransaction.cs
index c2976cf5fa5..933ad33b336 100644
--- a/modules/platforms/dotnet/Apache.Ignite/Internal/Transactions/LazyTransaction.cs
+++ b/modules/platforms/dotnet/Apache.Ignite/Internal/Transactions/LazyTransaction.cs
@@ -197,6 +197,7 @@ void Write()
{
var w = writer.MessageWriter;
w.Write(_options.ReadOnly);
+ w.Write(_options.TimeoutMillis);
w.Write(failoverSocket.ObservableTimestamp);
}
}
diff --git a/modules/platforms/dotnet/Apache.Ignite/Transactions/TransactionOptions.cs b/modules/platforms/dotnet/Apache.Ignite/Transactions/TransactionOptions.cs
index acc005bdccc..8a94718c2a4 100644
--- a/modules/platforms/dotnet/Apache.Ignite/Transactions/TransactionOptions.cs
+++ b/modules/platforms/dotnet/Apache.Ignite/Transactions/TransactionOptions.cs
@@ -25,4 +25,10 @@ namespace Apache.Ignite.Transactions;
/// Read-only transactions provide a snapshot view of data at a certain point in time.
/// They are lock-free and perform better than normal transactions, but do not permit data modifications.
///
-public readonly record struct TransactionOptions(bool ReadOnly);
+///
+/// Transaction timeout. 0 means 'use default timeout'.
+/// For RO transactions, the default timeout is data availability time configured via ignite.gc.lowWatermark.dataAvailabilityTime
+/// configuration setting.
+/// For RW transactions, timeouts are not supported yet. TODO: IGNITE-15936.
+///
+public readonly record struct TransactionOptions(bool ReadOnly, long TimeoutMillis = 0);
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteNodeRestartTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteNodeRestartTest.java
index 90e67ba95e7..fee0635e903 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteNodeRestartTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteNodeRestartTest.java
@@ -631,7 +631,8 @@ public CompletableFuture invoke(Condition condition, List su
threadPoolsManager.partitionOperationsExecutor(),
resourcesRegistry,
transactionInflights,
- lowWatermark
+ lowWatermark,
+ threadPoolsManager.commonScheduler()
);
ResourceVacuumManager resourceVacuumManager = new ResourceVacuumManager(
diff --git a/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java b/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
index a2015120784..2c86070a636 100644
--- a/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
+++ b/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
@@ -60,6 +60,7 @@
import org.apache.ignite.catalog.IgniteCatalog;
import org.apache.ignite.client.handler.ClientHandlerMetricSource;
import org.apache.ignite.client.handler.ClientHandlerModule;
+import org.apache.ignite.client.handler.ClientInboundMessageHandler;
import org.apache.ignite.client.handler.ClusterInfo;
import org.apache.ignite.client.handler.configuration.ClientConnectorConfiguration;
import org.apache.ignite.client.handler.configuration.ClientConnectorExtensionConfiguration;
@@ -358,7 +359,7 @@ public class IgniteImpl implements Ignite {
private final ReplicaManager replicaMgr;
/** Transactions manager. */
- private final TxManager txManager;
+ private final TxManagerImpl txManager;
/** Distributed table manager. */
private final TableManager distributedTblMgr;
@@ -989,10 +990,11 @@ public class IgniteImpl implements Ignite {
threadPoolsManager.partitionOperationsExecutor(),
resourcesRegistry,
transactionInflights,
- lowWatermark
+ lowWatermark,
+ threadPoolsManager.commonScheduler()
);
- systemViewManager.register((TxManagerImpl) txManager);
+ systemViewManager.register(txManager);
resourceVacuumManager = new ResourceVacuumManager(
name,
@@ -1878,17 +1880,17 @@ public LogStorageFactory partitionsLogStorageFactory() {
return partitionsLogStorageFactory;
}
- @TestOnly
- public LogStorageFactory volatileLogStorageFactory() {
- return volatileLogStorageFactoryCreator.factory(raftMgr.volatileRaft().logStorageBudget().value());
- }
-
/** Returns the node's transaction manager. */
@TestOnly
public TxManager txManager() {
return txManager;
}
+ @TestOnly
+ public ClientInboundMessageHandler clientInboundMessageHandler() {
+ return clientHandlerModule.handler();
+ }
+
/** Returns the node's placement driver service. */
@TestOnly
public PlacementDriver placementDriver() {
diff --git a/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/sqllogic/ItSqlLogicTest.java b/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/sqllogic/ItSqlLogicTest.java
index bee186f5e3e..49df7bca717 100644
--- a/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/sqllogic/ItSqlLogicTest.java
+++ b/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/sqllogic/ItSqlLogicTest.java
@@ -351,7 +351,8 @@ private static void startNodes() {
.clusterName("cluster")
.clusterConfiguration("ignite {"
+ "metaStorage.idleSyncTimeInterval: " + METASTORAGE_IDLE_SYNC_TIME_INTERVAL_MS + ",\n"
- + "gc.lowWatermark.dataAvailabilityTime: 5000,\n"
+ // TODO: Set dataAvailabilityTime to 5000 after IGNITE-24002 is fixed.
+ + "gc.lowWatermark.dataAvailabilityTime: 30000,\n"
+ "gc.lowWatermark.updateInterval: 1000,\n"
+ "metrics.exporters.logPush.exporterName: logPush,\n"
+ "metrics.exporters.logPush.period: 5000\n"
diff --git a/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/sqllogic/ScriptContext.java b/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/sqllogic/ScriptContext.java
index a8815808c68..641a83b4397 100644
--- a/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/sqllogic/ScriptContext.java
+++ b/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/sqllogic/ScriptContext.java
@@ -27,6 +27,7 @@
import java.util.HashMap;
import java.util.List;
import java.util.Map;
+import java.util.concurrent.TimeUnit;
import org.apache.ignite.internal.logger.IgniteLogger;
import org.apache.ignite.internal.sql.sqllogic.SqlScriptRunner.RunnerRuntime;
import org.apache.ignite.sql.IgniteSql;
@@ -78,7 +79,9 @@ final class ScriptContext {
List> executeQuery(String sql) {
sql = replaceVars(sql);
- log.info("Execute: " + sql);
+ log.info("Execute: {}", sql);
+
+ long startNanos = System.nanoTime();
try (ResultSet rs = ignSql.execute(null, sql)) {
if (rs.hasRowSet()) {
@@ -100,6 +103,9 @@ List> executeQuery(String sql) {
} else {
return Collections.singletonList(Collections.singletonList(rs.wasApplied()));
}
+ } finally {
+ long tookNanos = System.nanoTime() - startNanos;
+ log.info("Execution took {} ms", TimeUnit.NANOSECONDS.toMillis(tookNanos));
}
}
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/tx/QueryTransactionContextImpl.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/tx/QueryTransactionContextImpl.java
index f1db775f409..ee5275787ee 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/tx/QueryTransactionContextImpl.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/tx/QueryTransactionContextImpl.java
@@ -24,6 +24,7 @@
import org.apache.ignite.internal.sql.engine.exec.TransactionTracker;
import org.apache.ignite.internal.tx.HybridTimestampTracker;
import org.apache.ignite.internal.tx.InternalTransaction;
+import org.apache.ignite.internal.tx.InternalTxOptions;
import org.apache.ignite.internal.tx.TxManager;
import org.apache.ignite.tx.TransactionException;
import org.jetbrains.annotations.Nullable;
@@ -58,7 +59,12 @@ public QueryTransactionWrapper getOrStartSqlManaged(boolean readOnly, boolean im
QueryTransactionWrapper result;
if (tx == null) {
- transaction = txManager.begin(observableTimeTracker, implicit, readOnly);
+ if (implicit) {
+ transaction = txManager.beginImplicit(observableTimeTracker, readOnly);
+ } else {
+ transaction = txManager.beginExplicit(observableTimeTracker, readOnly, InternalTxOptions.defaults());
+ }
+
result = new QueryTransactionWrapperImpl(transaction, true, txTracker);
} else {
transaction = tx.unwrap();
diff --git a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/QueryTransactionWrapperSelfTest.java b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/QueryTransactionWrapperSelfTest.java
index 6cc20636a42..394241d9d74 100644
--- a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/QueryTransactionWrapperSelfTest.java
+++ b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/QueryTransactionWrapperSelfTest.java
@@ -131,7 +131,7 @@ public void throwsExceptionForNestedScriptTransaction() {
);
IgniteSqlStartTransaction txStartStmt = mock(IgniteSqlStartTransaction.class);
- when(txManager.begin(any(), anyBoolean(), anyBoolean())).thenAnswer(inv -> {
+ when(txManager.beginExplicit(any(), anyBoolean(), any())).thenAnswer(inv -> {
boolean implicit = inv.getArgument(1, Boolean.class);
return NoOpTransaction.readWrite("test", implicit);
@@ -222,12 +222,11 @@ public void testScriptTransactionWrapperTxInflightsInteraction() {
}
private void prepareTransactionsMocks() {
- when(txManager.begin(any(), anyBoolean(), anyBoolean())).thenAnswer(
+ when(txManager.beginExplicit(any(), anyBoolean(), any())).thenAnswer(
inv -> {
- boolean implicit = inv.getArgument(1, Boolean.class);
- boolean readOnly = inv.getArgument(2, Boolean.class);
+ boolean readOnly = inv.getArgument(1, Boolean.class);
- return readOnly ? NoOpTransaction.readOnly("test-ro", implicit) : NoOpTransaction.readWrite("test-rw", implicit);
+ return readOnly ? NoOpTransaction.readOnly("test-ro", false) : NoOpTransaction.readWrite("test-rw", false);
}
);
}
diff --git a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/rel/TableScanNodeExecutionTest.java b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/rel/TableScanNodeExecutionTest.java
index d98e0cbbf14..c7b780f271b 100644
--- a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/rel/TableScanNodeExecutionTest.java
+++ b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/rel/TableScanNodeExecutionTest.java
@@ -36,6 +36,7 @@
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.Flow.Publisher;
import java.util.concurrent.Flow.Subscription;
+import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ThreadLocalRandom;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
@@ -75,6 +76,8 @@
import org.apache.ignite.internal.storage.engine.MvTableStorage;
import org.apache.ignite.internal.table.StreamerReceiverRunner;
import org.apache.ignite.internal.table.distributed.storage.InternalTableImpl;
+import org.apache.ignite.internal.testframework.ExecutorServiceExtension;
+import org.apache.ignite.internal.testframework.InjectExecutorService;
import org.apache.ignite.internal.tx.HybridTimestampTracker;
import org.apache.ignite.internal.tx.TxManager;
import org.apache.ignite.internal.tx.configuration.TransactionConfiguration;
@@ -97,12 +100,16 @@
* Tests execution flow of TableScanNode.
*/
@ExtendWith(ConfigurationExtension.class)
+@ExtendWith(ExecutorServiceExtension.class)
public class TableScanNodeExecutionTest extends AbstractExecutionTest