Uploaded image for project: 'Ignite'
  1. Ignite
  2. IGNITE-23505

[FLAKY] UpdateCommandImpl cannot be cast to SafeTimeSyncCommand while inserting

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Open
    • Major
    • Resolution: Unresolved
    • 3.0.0-beta1
    • None
    • persistence
    • 3 nodes, each on separate host

    • Docs Required, Release Notes Required

    Description

      Steps to reproduce:

      1. Start 3 nodes cluster, each server on separate host.
      2. Create 50 tables using JDBC.
      3. Assert 50 tables are created.
      4. Insert 1 row in each table

      Expected:

      Steps are executed successfully.

      Actual:

      Server logs: server_logs.zip

      Exception on the client side:

      java.sql.SQLException: class org.apache.ignite.internal.partition.replicator.network.command.UpdateCommandImpl cannot be cast to class org.apache.ignite.internal.replicator.command.SafeTimeSyncCommand (org.apache.ignite.internal.partition.replicator.network.command.UpdateCommandImpl and org.apache.ignite.internal.replicator.command.SafeTimeSyncCommand are in unnamed module of loader 'app')  at app//org.junit.jupiter.api.AssertionFailureBuilder.build(AssertionFailureBuilder.java:152)  at app//org.junit.jupiter.api.AssertDoesNotThrow.createAssertionFailedError(AssertDoesNotThrow.java:84)  at app//org.junit.jupiter.api.AssertDoesNotThrow.assertDoesNotThrow(AssertDoesNotThrow.java:53)  at app//org.junit.jupiter.api.AssertDoesNotThrow.assertDoesNotThrow(AssertDoesNotThrow.java:40)  at app//org.junit.jupiter.api.Assertions.assertDoesNotThrow(Assertions.java:3183)  at app//org.gridgain.ai3tests.tests.amountcapacity.TablesAmountCapacityBaseTest.lambda$invokeTasks$14(TablesAmountCapacityBaseTest.java:381)  at app//io.qameta.allure.Allure.lambda$step$0(Allure.java:113)  at app//io.qameta.allure.Allure.lambda$step$1(Allure.java:127)  at app//io.qameta.allure.Allure.step(Allure.java:181)  at app//io.qameta.allure.Allure.step(Allure.java:125)  at app//io.qameta.allure.Allure.step(Allure.java:112)  at app//org.gridgain.ai3tests.tests.amountcapacity.TablesAmountCapacityBaseTest.step(TablesAmountCapacityBaseTest.java:271)  at app//org.gridgain.ai3tests.tests.amountcapacity.TablesAmountCapacityBaseTest.invokeTasks(TablesAmountCapacityBaseTest.java:376)  at app//org.gridgain.ai3tests.tests.amountcapacity.TablesAmountCapacityBaseTest.insertRowInTablesParallel(TablesAmountCapacityBaseTest.java:183)  at app//org.gridgain.ai3tests.tests.amountcapacity.TablesAmountCapacityBaseTest.testTablesAmount(TablesAmountCapacityBaseTest.java:92)  at app//org.gridgain.ai3tests.tests.amountcapacity.TablesAmountCapacityMultiNodeTest.createTablesWithSinglePartition(TablesAmountCapacityMultiNodeTest.java:78)  at java.base@11.0.17/java.lang.reflect.Method.invoke(Method.java:566)  at java.base@11.0.17/java.util.concurrent.FutureTask.run(FutureTask.java:264)  at java.base@11.0.17/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)  at java.base@11.0.17/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)  at java.base@11.0.17/java.lang.Thread.run(Thread.java:834)Caused by: java.util.concurrent.ExecutionException: java.sql.SQLException: class org.apache.ignite.internal.partition.replicator.network.command.UpdateCommandImpl cannot be cast to class org.apache.ignite.internal.replicator.command.SafeTimeSyncCommand (org.apache.ignite.internal.partition.replicator.network.command.UpdateCommandImpl and org.apache.ignite.internal.replicator.command.SafeTimeSyncCommand are in unnamed module of loader 'app')  at java.base/java.util.concurrent.FutureTask.report(FutureTask.java:122)  at java.base/java.util.concurrent.FutureTask.get(FutureTask.java:191)  at org.gridgain.ai3tests.tests.amountcapacity.TablesAmountCapacityBaseTest.lambda$invokeTasks$13(TablesAmountCapacityBaseTest.java:383)  at org.junit.jupiter.api.AssertDoesNotThrow.assertDoesNotThrow(AssertDoesNotThrow.java:49)  ... 18 moreCaused by: java.sql.SQLException: class org.apache.ignite.internal.partition.replicator.network.command.UpdateCommandImpl cannot be cast to class org.apache.ignite.internal.replicator.command.SafeTimeSyncCommand (org.apache.ignite.internal.partition.replicator.network.command.UpdateCommandImpl and org.apache.ignite.internal.replicator.command.SafeTimeSyncCommand are in unnamed module of loader 'app')  at org.apache.ignite.internal.jdbc.proto.IgniteQueryErrorCode.createJdbcSqlException(IgniteQueryErrorCode.java:57)  at org.apache.ignite.internal.jdbc.JdbcStatement.execute0(JdbcStatement.java:154)  at org.apache.ignite.internal.jdbc.JdbcPreparedStatement.executeWithArguments(JdbcPreparedStatement.java:765)  at org.apache.ignite.internal.jdbc.JdbcPreparedStatement.executeUpdate(JdbcPreparedStatement.java:173)  at org.gridgain.ai3tests.tests.amountcapacity.TablesAmountCapacityBaseTest.lambda$insertRowInTablesParallel$4(TablesAmountCapacityBaseTest.java:175)  ... 4 more 

      Exception in the server logs:

      2024-10-21 02:56:46:762 +0000 [INFO][%TablesAmountCapacityMultiNodeTest_cluster_0%sql-execution-pool-2][JdbcQueryEventHandlerImpl] Exception while executing query [query=insert into test_table_2(id) values (?)]
      org.apache.ignite.sql.SqlException: IGN-CMN-65535 TraceId:f8f96c95-d82a-48ba-a2eb-ffefb1691649 class org.apache.ignite.internal.partition.replicator.network.command.UpdateCommandImpl cannot be cast to class org.apache.ignite.internal.replicator.command.SafeTimeSyncCommand (org.apache.ignite.internal.partition.replicator.network.command.UpdateCommandImpl and org.apache.ignite.internal.replicator.command.SafeTimeSyncCommand are in unnamed module of loader 'app')
      	at org.apache.ignite.internal.lang.SqlExceptionMapperUtil.mapToPublicSqlException(SqlExceptionMapperUtil.java:61)
      	at org.apache.ignite.internal.sql.engine.SqlQueryProcessor$PrefetchCallback.onPrefetchComplete(SqlQueryProcessor.java:1053)
      	at org.apache.ignite.internal.sql.engine.prepare.KeyValueModifyPlan.lambda$execute$3(KeyValueModifyPlan.java:141)
      	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.run(CompletableFuture.java:478)
      	at org.apache.ignite.internal.sql.engine.exec.ExecutionContext.lambda$execute$0(ExecutionContext.java:332)
      	at org.apache.ignite.internal.sql.engine.exec.QueryTaskExecutorImpl.lambda$execute$0(QueryTaskExecutorImpl.java:83)
      	at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
      	at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
      	at java.base/java.lang.Thread.run(Thread.java:834)
      Caused by: org.apache.ignite.lang.IgniteException: IGN-CMN-65535 TraceId:f8f96c95-d82a-48ba-a2eb-ffefb1691649 class org.apache.ignite.internal.partition.replicator.network.command.UpdateCommandImpl cannot be cast to class org.apache.ignite.internal.replicator.command.SafeTimeSyncCommand (org.apache.ignite.internal.partition.replicator.network.command.UpdateCommandImpl and org.apache.ignite.internal.replicator.command.SafeTimeSyncCommand are in unnamed module of loader 'app')
      	at org.apache.ignite.internal.lang.IgniteExceptionMapperUtil.lambda$mapToPublicException$2(IgniteExceptionMapperUtil.java:88)
      	at org.apache.ignite.internal.lang.IgniteExceptionMapperUtil.mapCheckingResultIsPublic(IgniteExceptionMapperUtil.java:141)
      	at org.apache.ignite.internal.lang.IgniteExceptionMapperUtil.mapToPublicException(IgniteExceptionMapperUtil.java:137)
      	at org.apache.ignite.internal.lang.IgniteExceptionMapperUtil.mapToPublicException(IgniteExceptionMapperUtil.java:88)
      	at org.apache.ignite.internal.lang.SqlExceptionMapperUtil.mapToPublicSqlException(SqlExceptionMapperUtil.java:51)
      	... 10 more
      Caused by: java.lang.ClassCastException: class org.apache.ignite.internal.partition.replicator.network.command.UpdateCommandImpl cannot be cast to class org.apache.ignite.internal.replicator.command.SafeTimeSyncCommand (org.apache.ignite.internal.partition.replicator.network.command.UpdateCommandImpl and org.apache.ignite.internal.replicator.command.SafeTimeSyncCommand are in unnamed module of loader 'app')
      	at org.apache.ignite.internal.replicator.command.SafeTimeSyncCommandSerializer.writeMessage(SafeTimeSyncCommandSerializer.java:8)
      	at org.apache.ignite.internal.network.direct.stream.DirectByteBufferStreamImplV1.writeMessage(DirectByteBufferStreamImplV1.java:824)
      	at org.apache.ignite.internal.raft.util.OptimizedMarshaller.marshall(OptimizedMarshaller.java:128)
      	at org.apache.ignite.internal.table.distributed.schema.ThreadLocalPartitionCommandsMarshaller.marshall(ThreadLocalPartitionCommandsMarshaller.java:46)
      	at org.apache.ignite.internal.raft.RaftGroupServiceImpl.run(RaftGroupServiceImpl.java:478)
      	at org.apache.ignite.internal.raft.client.TopologyAwareRaftGroupService.run(TopologyAwareRaftGroupService.java:463)
      	at org.apache.ignite.internal.raft.ExecutorInclinedRaftCommandRunner.run(ExecutorInclinedRaftCommandRunner.java:44)
      	at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.applyCmdWithRetryOnSafeTimeReorderException(PartitionReplicaListener.java:2665)
      	at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.applyCmdWithRetryOnSafeTimeReorderException(PartitionReplicaListener.java:2655)
      	at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.applyCmdWithExceptionHandling(PartitionReplicaListener.java:2641)
      	at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.applyUpdateCommand(PartitionReplicaListener.java:2749)
      	at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.applyUpdateCommand(PartitionReplicaListener.java:2802)
      	at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.lambda$processSingleEntryAction$130(PartitionReplicaListener.java:3018)
      	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.lambda$processSingleEntryAction$132(PartitionReplicaListener.java:3017)
      	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.lambda$processSingleEntryAction$135(PartitionReplicaListener.java:3016)
      	at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.continueResolvingByPk(PartitionReplicaListener.java:1992)
      	at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.lambda$resolveRowByPk$75(PartitionReplicaListener.java:1966)
      	at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.resolveRowByPk(PartitionReplicaListener.java:1980)
      	at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.processSingleEntryAction(PartitionReplicaListener.java:3008)
      	at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.lambda$processOperationRequest$7(PartitionReplicaListener.java:696)
      	at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.appendTxCommand(PartitionReplicaListener.java:2024)
      	at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.processOperationRequest(PartitionReplicaListener.java:696)
      	at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.processOperationRequestWithTxRwCounter(PartitionReplicaListener.java:3977)
      	at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.lambda$processRequest$5(PartitionReplicaListener.java:554)
      	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.processRequest(PartitionReplicaListener.java:553)
      	at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.lambda$invoke$1(PartitionReplicaListener.java:470)
      	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.invoke(PartitionReplicaListener.java:470)
      	at org.apache.ignite.internal.replicator.ReplicaImpl.processRequest(ReplicaImpl.java:155)
      	at org.apache.ignite.internal.replicator.ReplicaManager.handleReplicaRequest(ReplicaManager.java:446)
      	at org.apache.ignite.internal.replicator.ReplicaManager.onReplicaMessageReceived(ReplicaManager.java:379)
      	at org.apache.ignite.internal.network.DefaultMessagingService.handleStartingWithFirstHandler(DefaultMessagingService.java:545)
      	at org.apache.ignite.internal.network.DefaultMessagingService.lambda$handleMessageFromNetwork$5(DefaultMessagingService.java:440)
      	... 3 more 

      Attachments

        1. server_logs.zip
          106 kB
          Igor

        Activity

          People

            Unassigned Unassigned
            lunigorn Igor
            Votes:
            0 Vote for this issue
            Watchers:
            1 Start watching this issue

            Dates

              Created:
              Updated: