Details
-
Bug
-
Status: Resolved
-
Blocker
-
Resolution: Fixed
-
3.0.0-beta1
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
- blocks
-
IGNITE-20399 .NET: Thin 3.0: TestSchemaUpdateWhileStreaming fails with IncompatibleSchemaException
- Resolved
- depends upon
-
IGNITE-19447 Switch schema validation to CatalogService
- Resolved
- fixes
-
IGNITE-21998 .NET: TestSchemaUpdateWhileStreaming is flaky
- Resolved
- is caused by
-
IGNITE-20044 Validate schema eligibility on each read/write operation in an RW transaction
- Resolved
- is related to
-
IGNITE-19229 Schema validation during tx processing: validators
- Resolved
- relates to
-
IGNITE-20415 Internal IncompatibleSchemaException is thrown from public API
- Resolved
-
IGNITE-20106 Check that client schema version matches server-side schema version
- Resolved
- links to