You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ignite.apache.org by "Alexander Belyak (Jira)" <ji...@apache.org> on 2022/12/09 09:03:00 UTC

[jira] [Created] (IGNITE-18358) IGN-TX-5 on concurrent transactional single key load

Alexander Belyak created IGNITE-18358:
-----------------------------------------

             Summary: IGN-TX-5 on concurrent transactional single key load
                 Key: IGNITE-18358
                 URL: https://issues.apache.org/jira/browse/IGNITE-18358
             Project: Ignite
          Issue Type: Improvement
          Components: persistence
    Affects Versions: 3.0.0-beta1
            Reporter: Alexander Belyak


On a single node (embedded) cluster I get

 
{noformat}
java.util.concurrent.ExecutionException: java.lang.Exception: org.apache.ignite.lang.IgniteException: IGN-CMN-65535 TraceId:3c65b23f-95ee-4226-a929-7f7aefd17e16 Failed to acquire a lock due to a conflict [txId=000e3ab5-de28-0000-f1d3-67fae4082a69, waiter=WaiterImpl [txId=000e3ab5-de29-0000-f1d3-67fae4082a69, upgraded=false, prevLockMode=null, lockMode=S, locked=true, isDone=true]]    at java.base/java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:395)
    at java.base/java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1999)
    at org.gridgain.poc.framework.starter.Ignite3StarterTest.test(Ignite3StarterTest.java:110)
    at org.gridgain.poc.framework.starter.Ignite3StarterTest.startSingleNodeTest(Ignite3StarterTest.java:65)
    at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
    at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
    at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
    at java.base/java.lang.reflect.Method.invoke(Method.java:566)
    at org.junit.platform.commons.util.ReflectionUtils.invokeMethod(ReflectionUtils.java:725)
    at org.junit.jupiter.engine.execution.MethodInvocation.proceed(MethodInvocation.java:60)
    at org.junit.jupiter.engine.execution.InvocationInterceptorChain$ValidatingInvocation.proceed(InvocationInterceptorChain.java:131)
    at org.junit.jupiter.engine.extension.TimeoutExtension.intercept(TimeoutExtension.java:149)
    at org.junit.jupiter.engine.extension.TimeoutExtension.interceptTestableMethod(TimeoutExtension.java:140)
    at org.junit.jupiter.engine.extension.TimeoutExtension.interceptTestMethod(TimeoutExtension.java:84)
    at org.junit.jupiter.engine.execution.ExecutableInvoker$ReflectiveInterceptorCall.lambda$ofVoidMethod$0(ExecutableInvoker.java:115)
    at org.junit.jupiter.engine.execution.ExecutableInvoker.lambda$invoke$0(ExecutableInvoker.java:105)
    at org.junit.jupiter.engine.execution.InvocationInterceptorChain$InterceptedInvocation.proceed(InvocationInterceptorChain.java:106)
    at org.junit.jupiter.engine.execution.InvocationInterceptorChain.proceed(InvocationInterceptorChain.java:64)
    at org.junit.jupiter.engine.execution.InvocationInterceptorChain.chainAndInvoke(InvocationInterceptorChain.java:45)
    at org.junit.jupiter.engine.execution.InvocationInterceptorChain.invoke(InvocationInterceptorChain.java:37)
    at org.junit.jupiter.engine.execution.ExecutableInvoker.invoke(ExecutableInvoker.java:104)
    at org.junit.jupiter.engine.execution.ExecutableInvoker.invoke(ExecutableInvoker.java:98)
    at org.junit.jupiter.engine.descriptor.TestMethodTestDescriptor.lambda$invokeTestMethod$7(TestMethodTestDescriptor.java:214)
    at org.junit.platform.engine.support.hierarchical.ThrowableCollector.execute(ThrowableCollector.java:73)
    at org.junit.jupiter.engine.descriptor.TestMethodTestDescriptor.invokeTestMethod(TestMethodTestDescriptor.java:210)
    at org.junit.jupiter.engine.descriptor.TestMethodTestDescriptor.execute(TestMethodTestDescriptor.java:135)
    at org.junit.jupiter.engine.descriptor.TestMethodTestDescriptor.execute(TestMethodTestDescriptor.java:66)
    at org.junit.platform.engine.support.hierarchical.NodeTestTask.lambda$executeRecursively$6(NodeTestTask.java:151)
    at org.junit.platform.engine.support.hierarchical.ThrowableCollector.execute(ThrowableCollector.java:73)
    at org.junit.platform.engine.support.hierarchical.NodeTestTask.lambda$executeRecursively$8(NodeTestTask.java:141)
    at org.junit.platform.engine.support.hierarchical.Node.around(Node.java:137)
    at org.junit.platform.engine.support.hierarchical.NodeTestTask.lambda$executeRecursively$9(NodeTestTask.java:139)
    at org.junit.platform.engine.support.hierarchical.ThrowableCollector.execute(ThrowableCollector.java:73)
    at org.junit.platform.engine.support.hierarchical.NodeTestTask.executeRecursively(NodeTestTask.java:138)
    at org.junit.platform.engine.support.hierarchical.NodeTestTask.execute(NodeTestTask.java:95)
    at java.base/java.util.ArrayList.forEach(ArrayList.java:1541)
    at org.junit.platform.engine.support.hierarchical.SameThreadHierarchicalTestExecutorService.invokeAll(SameThreadHierarchicalTestExecutorService.java:41)
    at org.junit.platform.engine.support.hierarchical.NodeTestTask.lambda$executeRecursively$6(NodeTestTask.java:155)
    at org.junit.platform.engine.support.hierarchical.ThrowableCollector.execute(ThrowableCollector.java:73)
    at org.junit.platform.engine.support.hierarchical.NodeTestTask.lambda$executeRecursively$8(NodeTestTask.java:141)
    at org.junit.platform.engine.support.hierarchical.Node.around(Node.java:137)
    at org.junit.platform.engine.support.hierarchical.NodeTestTask.lambda$executeRecursively$9(NodeTestTask.java:139)
    at org.junit.platform.engine.support.hierarchical.ThrowableCollector.execute(ThrowableCollector.java:73)
    at org.junit.platform.engine.support.hierarchical.NodeTestTask.executeRecursively(NodeTestTask.java:138)
    at org.junit.platform.engine.support.hierarchical.NodeTestTask.execute(NodeTestTask.java:95)
    at java.base/java.util.ArrayList.forEach(ArrayList.java:1541)
    at org.junit.platform.engine.support.hierarchical.SameThreadHierarchicalTestExecutorService.invokeAll(SameThreadHierarchicalTestExecutorService.java:41)
    at org.junit.platform.engine.support.hierarchical.NodeTestTask.lambda$executeRecursively$6(NodeTestTask.java:155)
    at org.junit.platform.engine.support.hierarchical.ThrowableCollector.execute(ThrowableCollector.java:73)
    at org.junit.platform.engine.support.hierarchical.NodeTestTask.lambda$executeRecursively$8(NodeTestTask.java:141)
    at org.junit.platform.engine.support.hierarchical.Node.around(Node.java:137)
    at org.junit.platform.engine.support.hierarchical.NodeTestTask.lambda$executeRecursively$9(NodeTestTask.java:139)
    at org.junit.platform.engine.support.hierarchical.ThrowableCollector.execute(ThrowableCollector.java:73)
    at org.junit.platform.engine.support.hierarchical.NodeTestTask.executeRecursively(NodeTestTask.java:138)
    at org.junit.platform.engine.support.hierarchical.NodeTestTask.execute(NodeTestTask.java:95)
    at org.junit.platform.engine.support.hierarchical.SameThreadHierarchicalTestExecutorService.submit(SameThreadHierarchicalTestExecutorService.java:35)
    at org.junit.platform.engine.support.hierarchical.HierarchicalTestExecutor.execute(HierarchicalTestExecutor.java:57)
    at org.junit.platform.engine.support.hierarchical.HierarchicalTestEngine.execute(HierarchicalTestEngine.java:54)
    at org.junit.platform.launcher.core.EngineExecutionOrchestrator.execute(EngineExecutionOrchestrator.java:107)
    at org.junit.platform.launcher.core.EngineExecutionOrchestrator.execute(EngineExecutionOrchestrator.java:88)
    at org.junit.platform.launcher.core.EngineExecutionOrchestrator.lambda$execute$0(EngineExecutionOrchestrator.java:54)
    at org.junit.platform.launcher.core.EngineExecutionOrchestrator.withInterceptedStreams(EngineExecutionOrchestrator.java:67)
    at org.junit.platform.launcher.core.EngineExecutionOrchestrator.execute(EngineExecutionOrchestrator.java:52)
    at org.junit.platform.launcher.core.DefaultLauncher.execute(DefaultLauncher.java:114)
    at org.junit.platform.launcher.core.DefaultLauncher.execute(DefaultLauncher.java:86)
    at org.junit.platform.launcher.core.DefaultLauncherSession$DelegatingLauncher.execute(DefaultLauncherSession.java:86)
    at org.junit.platform.launcher.core.SessionPerRequestLauncher.execute(SessionPerRequestLauncher.java:53)
    at com.intellij.junit5.JUnit5IdeaTestRunner.startRunnerWithArgs(JUnit5IdeaTestRunner.java:71)
    at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38)
    at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11)
    at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35)
    at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:235)
    at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:54)
Caused by: java.lang.Exception: org.apache.ignite.lang.IgniteException: IGN-CMN-65535 TraceId:3c65b23f-95ee-4226-a929-7f7aefd17e16 Failed to acquire a lock due to a conflict [txId=000e3ab5-de28-0000-f1d3-67fae4082a69, waiter=WaiterImpl [txId=000e3ab5-de29-0000-f1d3-67fae4082a69, upgraded=false, prevLockMode=null, lockMode=S, locked=true, isDone=true]]
    at org.gridgain.qaaas.utils.ExecutorUtils.lambda$runAsync$3(ExecutorUtils.java:122)
    at org.gridgain.qaaas.utils.ExecutorUtils.lambda$runAsync$4(ExecutorUtils.java:164)
    at java.base/java.lang.Thread.run(Thread.java:829)
Caused by: org.apache.ignite.lang.IgniteException: IGN-CMN-65535 TraceId:3c65b23f-95ee-4226-a929-7f7aefd17e16 Failed to acquire a lock due to a conflict [txId=000e3ab5-de28-0000-f1d3-67fae4082a69, waiter=WaiterImpl [txId=000e3ab5-de29-0000-f1d3-67fae4082a69, upgraded=false, prevLockMode=null, lockMode=S, locked=true, isDone=true]]
    at org.apache.ignite.internal.table.AbstractTableView.convertException(AbstractTableView.java:80)
    at org.apache.ignite.internal.table.AbstractTableView.sync(AbstractTableView.java:62)
    at org.apache.ignite.internal.table.KeyValueViewImpl.put(KeyValueViewImpl.java:151)
    at org.gridgain.poc.framework.starter.Ignite3StarterTest.lambda$test$0(Ignite3StarterTest.java:92)
    at org.apache.ignite.tx.IgniteTransactions.lambda$runInTransaction$0(IgniteTransactions.java:130)
    at org.apache.ignite.tx.IgniteTransactions.runInTransaction(IgniteTransactions.java:175)
    at org.apache.ignite.tx.IgniteTransactions.runInTransaction(IgniteTransactions.java:129)
    at org.gridgain.poc.framework.starter.Ignite3StarterTest.lambda$test$1(Ignite3StarterTest.java:89)
    at org.gridgain.qaaas.utils.ExecutorUtils.lambda$runAsync$3(ExecutorUtils.java:120)
    ... 2 more
    Suppressed: org.apache.ignite.tx.TransactionException: IGN-CMN-65535 TraceId:155ee916-a698-423d-ba54-5e5715c92e29 Fail to finish the transaction txId = 000e3ab5-de28-0000-f1d3-67fae4082a69 because of inconsistent state = TxMeta [txState=ABORTED, commitTimestamp=HybridTimestamp [physical=1670573739564, logical=3]], expected state = null, state to set = TxMeta [txState=ABORTED, commitTimestamp=HybridTimestamp [physical=1670573739758, logical=0]]
        at org.apache.ignite.internal.util.ExceptionUtils.lambda$withCause$0(ExceptionUtils.java:346)
        at org.apache.ignite.internal.util.ExceptionUtils.withCauseInternal(ExceptionUtils.java:429)
        at org.apache.ignite.internal.util.ExceptionUtils.withCause(ExceptionUtils.java:346)
        at org.apache.ignite.internal.tx.impl.IgniteAbstractTransactionImpl.rollback(IgniteAbstractTransactionImpl.java:90)
        at org.apache.ignite.tx.IgniteTransactions.runInTransaction(IgniteTransactions.java:183)
        ... 5 more
    Caused by: java.util.concurrent.ExecutionException: org.apache.ignite.lang.IgniteInternalException: IGN-CMN-65535 TraceId:155ee916-a698-423d-ba54-5e5715c92e29 IGN-TX-3 TraceId:e218a2c6-3122-4a00-820d-47235225e1d9 Fail to finish the transaction txId = 000e3ab5-de28-0000-f1d3-67fae4082a69 because of inconsistent state = TxMeta [txState=ABORTED, commitTimestamp=HybridTimestamp [physical=1670573739564, logical=3]], expected state = null, state to set = TxMeta [txState=ABORTED, commitTimestamp=HybridTimestamp [physical=1670573739758, logical=0]]
        at java.base/java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:395)
        at java.base/java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1999)
        at org.apache.ignite.internal.tx.impl.IgniteAbstractTransactionImpl.rollback(IgniteAbstractTransactionImpl.java:88)
        ... 6 more
    Caused by: org.apache.ignite.lang.IgniteInternalException: IGN-CMN-65535 TraceId:155ee916-a698-423d-ba54-5e5715c92e29 IGN-TX-3 TraceId:e218a2c6-3122-4a00-820d-47235225e1d9 Fail to finish the transaction txId = 000e3ab5-de28-0000-f1d3-67fae4082a69 because of inconsistent state = TxMeta [txState=ABORTED, commitTimestamp=HybridTimestamp [physical=1670573739564, logical=3]], expected state = null, state to set = TxMeta [txState=ABORTED, commitTimestamp=HybridTimestamp [physical=1670573739758, logical=0]]
        at jdk.internal.reflect.GeneratedConstructorAccessor15.newInstance(Unknown Source)
        at java.base/jdk.internal.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
        at java.base/java.lang.reflect.Constructor.newInstance(Constructor.java:490)
        at org.apache.ignite.raft.jraft.rpc.impl.RaftGroupServiceImpl$1.accept(RaftGroupServiceImpl.java:660)
        at org.apache.ignite.raft.jraft.rpc.impl.RaftGroupServiceImpl$1.accept(RaftGroupServiceImpl.java:574)
        at java.base/java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:859)
        at java.base/java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:837)
        at java.base/java.util.concurrent.CompletableFuture$Completion.exec(CompletableFuture.java:479)
        at java.base/java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:290)
        at java.base/java.util.concurrent.ForkJoinPool$WorkQueue.topLevelExec(ForkJoinPool.java:1020)
        at java.base/java.util.concurrent.ForkJoinPool.scan(ForkJoinPool.java:1656)
        at java.base/java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1594)
        at java.base/java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:183)
Caused by: org.apache.ignite.internal.tx.LockException: IGN-TX-5 TraceId:ac562fc9-c4f9-4f20-87c8-6edded6231b2 Failed to acquire a lock due to a conflict [txId=000e3ab5-de28-0000-f1d3-67fae4082a69, waiter=WaiterImpl [txId=000e3ab5-de29-0000-f1d3-67fae4082a69, upgraded=false, prevLockMode=null, lockMode=S, locked=true, isDone=true]]
    at org.apache.ignite.internal.tx.impl.HeapLockManager$LockState.tryAcquire(HeapLockManager.java:223)
    at org.apache.ignite.internal.tx.impl.HeapLockManager.acquire(HeapLockManager.java:90)
    at org.apache.ignite.internal.table.distributed.HashIndexLocker.locksForInsert(HashIndexLocker.java:75)
    at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.takePutLockOnIndexes(PartitionReplicaListener.java:1511)
    at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.lambda$takeLocksForInsert$94(PartitionReplicaListener.java:1496)
    at java.base/java.util.concurrent.CompletableFuture.uniComposeStage(CompletableFuture.java:1106)
    at java.base/java.util.concurrent.CompletableFuture.thenCompose(CompletableFuture.java:2235)
    at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.takeLocksForInsert(PartitionReplicaListener.java:1496)
    at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.lambda$processSingleEntryAction$81(PartitionReplicaListener.java:1418)
    at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.lambda$resolveRowByPk$45(PartitionReplicaListener.java:1042)
    at java.base/java.util.concurrent.CompletableFuture.uniComposeStage(CompletableFuture.java:1106)
    at java.base/java.util.concurrent.CompletableFuture.thenCompose(CompletableFuture.java:2235)
    at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.resolveRowByPk(PartitionReplicaListener.java:1032)
    at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.processSingleEntryAction(PartitionReplicaListener.java:1412)
    at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.lambda$invoke$0(PartitionReplicaListener.java:248)
    at java.base/java.util.concurrent.CompletableFuture$UniCompose.tryFire(CompletableFuture.java:1072)
    at java.base/java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:506)
    at java.base/java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:2073)
    at org.apache.ignite.raft.jraft.rpc.impl.RaftGroupServiceImpl$1.accept(RaftGroupServiceImpl.java:678)
    at org.apache.ignite.raft.jraft.rpc.impl.RaftGroupServiceImpl$1.accept(RaftGroupServiceImpl.java:574)
    at java.base/java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:859)
    at java.base/java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:837)
    at java.base/java.util.concurrent.CompletableFuture$Completion.exec(CompletableFuture.java:479)
    at java.base/java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:290)
    at java.base/java.util.concurrent.ForkJoinPool$WorkQueue.topLevelExec(ForkJoinPool.java:1020)
    at java.base/java.util.concurrent.ForkJoinPool.scan(ForkJoinPool.java:1656)
    at java.base/java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1594)
    at java.base/java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:183)
Process finished with exit code 255
{noformat}
with simple load:

 

100 thread do: read the key via KeyValueView and runInTransaction get&put the same key.
{code:java}
public void test(Ignite ignite) throws ExecutionException, InterruptedException {
    try (Session ses = ignite.sql().createSession()) {
        try (ResultSet ts = ses.execute(null,
                "create table if not exists totals (tableName varchar primary key, totalBalance bigint not null)")) {
            // NoOp
        }
    }


    KeyValueView<String, Long> kv = ignite.tables().table("totals").keyValueView(String.class, Long.class);

    RunnableX r = () -> {
        Long val = kv.get(null, "k1");

        boolean done = false;

        while (!done) {
            try {
                ignite.transactions().runInTransaction(tx -> {
                    Long v1 = kv.get(tx, "k1");
                    v1 = (v1 == null && val == null) ? 1 : v1 + 1;
                    kv.put(tx, "k1", v1);
                });

                done = true;
            } catch (TransactionException e) {
                System.out.println("repeat " + Thread.currentThread().getName());
            }
        }

    };

    List<CompletableFuture<?>> futs = new ArrayList<>();

    for (int i = 0; i < 100; i++) {
        futs.add(ExecutorUtils.runAsync(r));
    }

    for(CompletableFuture<?> cf : futs) {
        cf.get();
    }

    try (Session ses = ignite.sql().createSession()) {
        try (ResultSet ts = ses.execute(null,
                "drop table totals")) {
            // NoOp
        }
    }
}{code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)