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

"SqlException: Table with name 'N' already exists" upon creating a non-existing table

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Blocker
    • Resolution: Won't Fix
    • None
    • 3.0
    • sql

    Description

      Steps

      • Start a cluster of 1 node with default settings, init the cluster.
      • Create 1000 unique tables in a loop using IgniteCluster.dataSource().getConnection().createStatement():

      Example of a DDL request:

       

      create table test_table_123(id INTEGER not null, column_1 VARCHAR(50) not null, column_2 VARCHAR(50) not null, column_3 VARCHAR(50) not null, column_4 VARCHAR(50) not null, primary key (id))

      Expected result

      All 1000 tables were created successfullt

      Actual result

      The following exception occurs on the server node:

      2023-12-21 02:52:27:298 +0000 [INFO][%TablesAmountCapacityTest_cluster_0%JRaft-FSMCaller-Disruptor-metastorage-_stripe_0-0][JdbcQueryEventHandlerImpl] Exception while executing query [query=create table test_table_523(id INTEGER not null, column_1 VARCHAR(50) not null, column_2 VARCHAR(50) not null, column_3 VARCHAR(50) not null, column_4 VARCHAR(50) not null, primary key (id))]
      org.apache.ignite.sql.SqlException: IGN-SQL-6 TraceId:12fb2fcf-71f1-4373-b85c-71f10b9b58aa Failed to validate query. Table with name 'PUBLIC.TEST_TABLE_523' already exists
      	at org.apache.ignite.internal.sql.engine.util.SqlExceptionMapperProvider.lambda$mappers$3(SqlExceptionMapperProvider.java:60)
      	at org.apache.ignite.internal.lang.IgniteExceptionMapper.map(IgniteExceptionMapper.java:61)
      	at org.apache.ignite.internal.lang.IgniteExceptionMapperUtil.map(IgniteExceptionMapperUtil.java:149)
      	at org.apache.ignite.internal.lang.IgniteExceptionMapperUtil.mapToPublicException(IgniteExceptionMapperUtil.java:103)
      	at org.apache.ignite.internal.lang.SqlExceptionMapperUtil.mapToPublicSqlException(SqlExceptionMapperUtil.java:49)
      	at org.apache.ignite.internal.sql.engine.AsyncSqlCursorImpl.wrapIfNecessary(AsyncSqlCursorImpl.java:191)
      	at org.apache.ignite.internal.sql.engine.AsyncSqlCursorImpl.lambda$requestNextAsync$2(AsyncSqlCursorImpl.java:123)
      	at java.base/java.util.concurrent.CompletableFuture.uniHandle(CompletableFuture.java:934)
      	at java.base/java.util.concurrent.CompletableFuture.uniHandleStage(CompletableFuture.java:950)
      	at java.base/java.util.concurrent.CompletableFuture.handle(CompletableFuture.java:2340)
      	at org.apache.ignite.internal.sql.engine.AsyncSqlCursorImpl.lambda$requestNextAsync$3(AsyncSqlCursorImpl.java:122)
      	at java.base/java.util.concurrent.CompletableFuture.uniExceptionally(CompletableFuture.java:990)
      	at java.base/java.util.concurrent.CompletableFuture$UniExceptionally.tryFire(CompletableFuture.java:974)
      	at java.base/java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:510)
      	at java.base/java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:2162)
      	at org.apache.ignite.internal.util.AsyncWrapper.lambda$requestNextAsync$2(AsyncWrapper.java:113)
      	at java.base/java.util.concurrent.CompletableFuture.uniExceptionally(CompletableFuture.java:990)
      	at java.base/java.util.concurrent.CompletableFuture$UniExceptionally.tryFire(CompletableFuture.java:974)
      	at java.base/java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:510)
      	at java.base/java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:2147)
      	at org.apache.ignite.internal.util.PendingComparableValuesTracker.lambda$completeWaitersOnUpdate$0(PendingComparableValuesTracker.java:169)
      	at java.base/java.util.concurrent.ConcurrentMap.forEach(ConcurrentMap.java:122)
      	at org.apache.ignite.internal.util.PendingComparableValuesTracker.completeWaitersOnUpdate(PendingComparableValuesTracker.java:169)
      	at org.apache.ignite.internal.util.PendingComparableValuesTracker.update(PendingComparableValuesTracker.java:103)
      	at org.apache.ignite.internal.catalog.CatalogManagerImpl$OnUpdateHandlerImpl.lambda$handle$1(CatalogManagerImpl.java:439)
      	at java.base/java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:863)
      	at java.base/java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:841)
      	at java.base/java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:510)
      	at java.base/java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:2147)
      	at org.apache.ignite.internal.raft.RaftGroupServiceImpl.lambda$sendWithRetry$40(RaftGroupServiceImpl.java:573)
      	at java.base/java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:863)
      	at java.base/java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:841)
      	at java.base/java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:510)
      	at java.base/java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:2147)
      	at org.apache.ignite.network.DefaultMessagingService.onInvokeResponse(DefaultMessagingService.java:398)
      	at org.apache.ignite.network.DefaultMessagingService.send0(DefaultMessagingService.java:213)
      	at org.apache.ignite.network.DefaultMessagingService.respond(DefaultMessagingService.java:156)
      	at org.apache.ignite.network.MessagingService.respond(MessagingService.java:111)
      	at org.apache.ignite.raft.jraft.rpc.impl.IgniteRpcServer$NetworkRpcContext.sendResponse(IgniteRpcServer.java:246)
      	at org.apache.ignite.raft.jraft.rpc.impl.ActionRequestProcessor$1.result(ActionRequestProcessor.java:192)
      	at org.apache.ignite.internal.raft.server.impl.JraftServerImpl$DelegatingStateMachine$1$1.result(JraftServerImpl.java:698)
      	at org.apache.ignite.internal.metastorage.server.raft.MetaStorageWriteHandler.handleWriteWithTime(MetaStorageWriteHandler.java:176)
      	at org.apache.ignite.internal.metastorage.server.raft.MetaStorageWriteHandler.handleWriteCommand(MetaStorageWriteHandler.java:97)
      	at java.base/java.util.Iterator.forEachRemaining(Iterator.java:133)
      	at org.apache.ignite.internal.metastorage.server.raft.MetaStorageListener.onWrite(MetaStorageListener.java:153)
      	at org.apache.ignite.internal.raft.server.impl.JraftServerImpl$DelegatingStateMachine.onApply(JraftServerImpl.java:659)
      	at org.apache.ignite.raft.jraft.core.FSMCallerImpl.doApplyTasks(FSMCallerImpl.java:557)
      	at org.apache.ignite.raft.jraft.core.FSMCallerImpl.doCommitted(FSMCallerImpl.java:525)
      	at org.apache.ignite.raft.jraft.core.FSMCallerImpl.runApplyTask(FSMCallerImpl.java:444)
      	at org.apache.ignite.raft.jraft.core.FSMCallerImpl$ApplyTaskHandler.onEvent(FSMCallerImpl.java:136)
      	at org.apache.ignite.raft.jraft.core.FSMCallerImpl$ApplyTaskHandler.onEvent(FSMCallerImpl.java:130)
      	at org.apache.ignite.raft.jraft.disruptor.StripedDisruptor$StripeEntryHandler.onEvent(StripedDisruptor.java:226)
      	at org.apache.ignite.raft.jraft.disruptor.StripedDisruptor$StripeEntryHandler.onEvent(StripedDisruptor.java:191)
      	at com.lmax.disruptor.BatchEventProcessor.run(BatchEventProcessor.java:137)
      	at java.base/java.lang.Thread.run(Thread.java:833)
      Caused by: org.apache.ignite.internal.catalog.TableExistsValidationException: IGN-CATALOG-1 TraceId:12fb2fcf-71f1-4373-b85c-71f10b9b58aa Table with name 'PUBLIC.TEST_TABLE_523' already exists
      	at org.apache.ignite.internal.catalog.CatalogParamsValidationUtils.ensureNoTableIndexOrSysViewExistsWithGivenName(CatalogParamsValidationUtils.java:116)
      	at org.apache.ignite.internal.catalog.commands.CreateTableCommand.get(CreateTableCommand.java:125)
      	at org.apache.ignite.internal.catalog.CatalogManagerImpl.saveUpdate(CatalogManagerImpl.java:363)
      	at org.apache.ignite.internal.catalog.CatalogManagerImpl.lambda$saveUpdate$5(CatalogManagerImpl.java:386)
      	at java.base/java.util.concurrent.CompletableFuture$UniCompose.tryFire(CompletableFuture.java:1150)
      	... 37 more

      The following TimeoutException is seen before that SQLException:

      2023-12-21 02:52:22:941 +0000 [WARNING][%TablesAmountCapacityTest_cluster_0%vault-1][WatchProcessor] Watch event processing has been too long [duration=154, keys=[placementdriver.leases]]
      2023-12-21 02:52:23:410 +0000 [WARNING][%TablesAmountCapacityTest_cluster_0%vault-2][WatchProcessor] Watch event processing has been too long [duration=103, keys=[placementdriver.leases]]
      2023-12-21 02:52:24:001 +0000 [INFO][TablesAmountCapacityTest_cluster_0-srv-worker-1][ClientInboundMessageHandler] Partition primary replica changed, notifying client [connectionId=1, remoteAddress=/127.0.0.1:33036]
      2023-12-21 02:52:27:012 +0000 [WARNING][CompletableFutureDelayScheduler][RaftGroupServiceImpl] Recoverable error during the request occurred (will be retried on the randomly selected node) [request=WriteActionRequestImpl [command=[112, 12, -33, 1, 9, -33, 1, 11, -33, 1, 8, 3, -33, 1, 6, 19, 99, 97, 116, 97, 108, 111, 103, 46, 117, 112, 100, 97, 116, 101, 46, 53, 50, 53, 2, -51, 21, -84, -19, 0, 5, 115, 114, 0, 58, 111, 114, 103, 46, 97, 112, 97, 99, 104, 101, 46, 105, 103, 110, 105, 116, 101, 46, 105, 110, 116, 101, 114, 110, 97, 108, 46, 99, 97, 116, 97, 108, 111, 103, 46, 115, 116, 111, 114, 97, 103, 101, 46, 86, 101, 114, 115, 105, 111, 110, 101, 100, 85, 112, 100, 97... and 2802 more], deserializedCommand=MultiInvokeCommandImpl [iif=IifImpl [andThen=UpdateStatementImpl [update=UpdateImpl [operations=List12 [OperationImpl [key=[99, 97, 116, 97, 108, 111, 103, 46, 117, 112, 100, 97, 116, 101, 46, 53, 50, 53], operationType=1, value=[-84, -19, 0, 5, 115, 114, 0, 58, 111, 114, 103, 46, 97, 112, 97, 99, 104, 101, 46, 105, 103, 110, 105, 116, 101, 46, 105, 110, 116, 101, 114, 110, 97, 108, 46, 99, 97, 116, 97, 108, 111, 103, 46, 115, 116, 111, 114, 97, 103, 101, 46, 86, 101, 114, 115, 105, 111, 110, 101, 100, 85, 112, 100, 97, 116, 101, 52, -71, 25, 6, 7, 14, 62, 87, 2, 0, 3, 74, 0, 15, 100, 101, 108, 97, 121, 68, 117, 114, 97, 116, 105, 111, 110, 77, 115, 73, 0, 7, 118, 101, 114... and 2664 more]], OperationImpl [key=[99, 97, 116, 97, 108, 111, 103, 46, 118, 101, 114, 115, 105, 111, 110], operationType=1, value=[0, 0, 2, 13]]], result=StatementResultImpl [result=[1]]]], condition=CompoundConditionImpl [compoundConditionType=1, leftCondition=SimpleConditionImpl [conditionType=13, key=[99, 97, 116, 97, 108, 111, 103, 46, 118, 101, 114, 115, 105, 111, 110]], rightCondition=ValueConditionImpl [conditionType=6, key=[99, 97, 116, 97, 108, 111, 103, 46, 118, 101, 114, 115, 105, 111, 110], value=[0, 0, 2, 12]]], orElse=UpdateStatementImpl [update=UpdateImpl [operations=ListN [], result=StatementResultImpl [result=[0]]]]], initiatorTimeLong=111616140509446144, safeTimeLong=111616140509446146], groupId=metastorage_group], peer=Peer [consistentId=TablesAmountCapacityTest_cluster_0, idx=0]].
      java.util.concurrent.CompletionException: java.util.concurrent.TimeoutException
      	at java.base/java.util.concurrent.CompletableFuture.encodeRelay(CompletableFuture.java:368)
      	at java.base/java.util.concurrent.CompletableFuture.completeRelay(CompletableFuture.java:377)
      	at java.base/java.util.concurrent.CompletableFuture$UniRelay.tryFire(CompletableFuture.java:1097)
      	at java.base/java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:510)
      	at java.base/java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:2162)
      	at java.base/java.util.concurrent.CompletableFuture$Timeout.run(CompletableFuture.java:2874)
      	at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:539)
      	at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
      	at java.base/java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:304)
      	at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
      	at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
      	at java.base/java.lang.Thread.run(Thread.java:833)
      Caused by: java.util.concurrent.TimeoutException
      	... 7 more
      2023-12-21 02:52:27:094 +0000 [INFO][%TablesAmountCapacityTest_cluster_0%metastorage-watch-executor-0][IndexManager] Creating local index: name=TEST_TABLE_523_PK, id=1052, tableId=1051, token=3622

      So it looks like some part of the create table test_table_523 ... request was somehow retried by the node itself and hence the Table with name 'PUBLIC.TEST_TABLE_523' already exists error.

      The issue was reproduced only once in a few tries. 

       

      Attachments

        Issue Links

          Activity

            People

              ktkalenko@gridgain.com Kirill Tkalenko
              Artukhov Ivan Artiukhov
              Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved:

                Time Tracking

                  Estimated:
                  Original Estimate - Not Specified
                  Not Specified
                  Remaining:
                  Remaining Estimate - 0h
                  0h
                  Logged:
                  Time Spent - 20m
                  20m