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

IncompatibleSchemaException thrown when schema change is compatible

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Blocker
    • Resolution: Fixed
    • 3.0.0-beta1
    • 3.0
    • sql

    Description

      The following code performs compatible schema change (add column with a default value), but causes IncompatibleSchemaException (add to ItSqlSynchronousApiTest and run):

          @Test
          public void schemaMigration() {
              IgniteSql sql = igniteSql();
              Session ses = sql.createSession();
      
              checkDdl(true, ses, "CREATE TABLE TEST(ID INT PRIMARY KEY, VAL0 INT)");
              var view = CLUSTER_NODES.get(0).tables().table("TEST").recordView();
      
              var upsertFut = CompletableFuture.runAsync(() -> {
                  for (int i = 0; i < 1000; i++) {
                      view.upsert(null, Tuple.create().set("ID", i).set("VAL0", i));
                  }
              });
      
              checkDdl(true, ses, "ALTER TABLE TEST ADD COLUMN VAL1 INT DEFAULT -1");
      
              upsertFut.join();
          }
      

      If we perform schema update before, after, or between upserts, then there is no error. Only when thy happen concurrently.

      Result:

      java.util.concurrent.CompletionException: org.apache.ignite.internal.table.distributed.replicator.IncompatibleSchemaException: IGN-TX-12 TraceId:52bbae8c-4706-4394-8bd3-30bac5747da5 Table schema was updated after the transaction was started [table=1, startSchema=1, operationSchema=2]
      	at java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:314) ~[?:?]
      	at java.util.concurrent.CompletableFuture.uniApplyNow(CompletableFuture.java:683) ~[?:?]
      	at java.util.concurrent.CompletableFuture.uniApplyStage(CompletableFuture.java:658) ~[?:?]
      	at java.util.concurrent.CompletableFuture.thenApply(CompletableFuture.java:2094) ~[?:?]
      	at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.validateAtTimestampAndBuildUpdateCommand(PartitionReplicaListener.java:2643) ~[ignite-table-3.0.0-SNAPSHOT.jar:?]
      	at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.validateAtTimestampAndBuildUpdateCommand(PartitionReplicaListener.java:2586) ~[ignite-table-3.0.0-SNAPSHOT.jar:?]
      	at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.lambda$processSingleEntryAction$109(PartitionReplicaListener.java:2058) ~[ignite-table-3.0.0-SNAPSHOT.jar:?]
      	at java.util.concurrent.CompletableFuture.uniComposeStage(CompletableFuture.java:1106) ~[?:?]
      	at java.util.concurrent.CompletableFuture.thenCompose(CompletableFuture.java:2235) ~[?:?]
      	at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.lambda$processSingleEntryAction$112(PartitionReplicaListener.java:2058) ~[ignite-table-3.0.0-SNAPSHOT.jar:?]
      	at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.continueResolvingByPk(PartitionReplicaListener.java:1448) ~[ignite-table-3.0.0-SNAPSHOT.jar:?]
      	at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.lambda$resolveRowByPk$68(PartitionReplicaListener.java:1428) ~[ignite-table-3.0.0-SNAPSHOT.jar:?]
      	at java.util.concurrent.CompletableFuture.uniComposeStage(CompletableFuture.java:1106) ~[?:?]
      	at java.util.concurrent.CompletableFuture.thenCompose(CompletableFuture.java:2235) ~[?:?]
      	at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.resolveRowByPk(PartitionReplicaListener.java:1419) ~[ignite-table-3.0.0-SNAPSHOT.jar:?]
      	at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.processSingleEntryAction(PartitionReplicaListener.java:2048) ~[ignite-table-3.0.0-SNAPSHOT.jar:?]
      	at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.lambda$processRequest$2(PartitionReplicaListener.java:363) ~[ignite-table-3.0.0-SNAPSHOT.jar:?]
      	at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.appendTxCommand(PartitionReplicaListener.java:1494) ~[ignite-table-3.0.0-SNAPSHOT.jar:?]
      	at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.processRequest(PartitionReplicaListener.java:363) ~[ignite-table-3.0.0-SNAPSHOT.jar:?]
      	at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.lambda$invoke$0(PartitionReplicaListener.java:347) ~[ignite-table-3.0.0-SNAPSHOT.jar:?]
      	at java.util.concurrent.CompletableFuture$UniCompose.tryFire(CompletableFuture.java:1072) ~[?:?]
      	at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:506) ~[?:?]
      	at java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:2073) ~[?:?]
      	at org.apache.ignite.internal.util.PendingComparableValuesTracker.lambda$completeWaitersOnUpdate$0(PendingComparableValuesTracker.java:169) ~[ignite-core-3.0.0-SNAPSHOT.jar:?]
      	at java.util.concurrent.ConcurrentMap.forEach(ConcurrentMap.java:122) ~[?:?]
      	at org.apache.ignite.internal.util.PendingComparableValuesTracker.completeWaitersOnUpdate(PendingComparableValuesTracker.java:169) ~[ignite-core-3.0.0-SNAPSHOT.jar:?]
      	at org.apache.ignite.internal.util.PendingComparableValuesTracker.update(PendingComparableValuesTracker.java:103) ~[ignite-core-3.0.0-SNAPSHOT.jar:?]
      	at org.apache.ignite.internal.metastorage.server.time.ClusterTimeImpl.updateSafeTime(ClusterTimeImpl.java:146) ~[ignite-metastorage-3.0.0-SNAPSHOT.jar:?]
      	at org.apache.ignite.internal.metastorage.impl.MetaStorageManagerImpl.onSafeTimeAdvanced(MetaStorageManagerImpl.java:849) ~[ignite-metastorage-3.0.0-SNAPSHOT.jar:?]
      	at org.apache.ignite.internal.metastorage.impl.MetaStorageManagerImpl$1.onSafeTimeAdvanced(MetaStorageManagerImpl.java:456) ~[ignite-metastorage-3.0.0-SNAPSHOT.jar:?]
      	at org.apache.ignite.internal.metastorage.server.WatchProcessor.lambda$advanceSafeTime$7(WatchProcessor.java:269) ~[ignite-metastorage-3.0.0-SNAPSHOT.jar:?]
      	at java.util.concurrent.CompletableFuture$UniRun.tryFire(CompletableFuture.java:783) ~[?:?]
      	at java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:478) ~[?:?]
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) ~[?:?]
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) ~[?:?]
      	at java.lang.Thread.run(Thread.java:829) ~[?:?]
      Caused by: org.apache.ignite.internal.table.distributed.replicator.IncompatibleSchemaException: Table schema was updated after the transaction was started [table=1, startSchema=1, operationSchema=2]
      	at org.apache.ignite.internal.table.distributed.replicator.SchemaCompatValidator.failIfSchemaChangedAfterTxStart(SchemaCompatValidator.java:183) ~[ignite-table-3.0.0-SNAPSHOT.jar:?]
      	at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.failIfSchemaChangedSinceTxStart(PartitionReplicaListener.java:2682) ~[ignite-table-3.0.0-SNAPSHOT.jar:?]
      	at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.lambda$validateAtTimestampAndBuildUpdateCommand$163(PartitionReplicaListener.java:2644) ~[ignite-table-3.0.0-SN
      

      Attachments

        Issue Links

          Activity

            People

              rpuch Roman Puchkovskiy
              ptupitsyn Pavel Tupitsyn
              Aleksandr Polovtsev Aleksandr Polovtsev
              Votes:
              0 Vote for this issue
              Watchers:
              3 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 - 1h 10m
                  1h 10m