Details
-
Bug
-
Status: Resolved
-
Blocker
-
Resolution: Fixed
-
3.0.0-beta1
Description
TestSchemaUpdateWhileStreaming fails due to ALTER TABLE happening between an implicit tx start and commit
Apache.Ignite.IgniteException : Table schema was updated after the transaction was started [table=93, startSchema=1, operationSchema=2] ----> Apache.Ignite.IgniteException : org.apache.ignite.internal.table.distributed.replicator.IncompatibleSchemaException: IGN-TX-12 TraceId:db7e4055-c764-46fb-a141-d17a61d8bedc Table schema was updated after the transaction was started [table=93, startSchema=1, operationSchema=2] at org.apache.ignite.internal.table.distributed.replicator.SchemaCompatValidator.failIfSchemaChangedAfterTxStart(SchemaCompatValidator.java:183) at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.failIfSchemaChangedSinceTxStart(PartitionReplicaListener.java:2693) at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.lambda$validateAtTimestampAndBuildUpdateAllCommand$168(PartitionReplicaListener.java:2766) at java.base/java.util.concurrent.CompletableFuture.uniApplyNow(CompletableFuture.java:680) at java.base/java.util.concurrent.CompletableFuture.uniApplyStage(CompletableFuture.java:658) at java.base/java.util.concurrent.CompletableFuture.thenApply(CompletableFuture.java:2094) at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.validateAtTimestampAndBuildUpdateAllCommand(PartitionReplicaListener.java:2765) at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.validateAtTimestampAndBuildUpdateAllCommand(PartitionReplicaListener.java:2714) at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.lambda$processMultiEntryAction$93(PartitionReplicaListener.java:1775) 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.processMultiEntryAction(PartitionReplicaListener.java:1761) at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.lambda$processRequest$4(PartitionReplicaListener.java:379) at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.appendTxCommand(PartitionReplicaListener.java:1505) at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.processRequest(PartitionReplicaListener.java:379) at org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener.lambda$invoke$0(PartitionReplicaListener.java:355) 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.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.metastorage.server.time.ClusterTimeImpl.updateSafeTime(ClusterTimeImpl.java:146) at org.apache.ignite.internal.metastorage.impl.MetaStorageManagerImpl.onSafeTimeAdvanced(MetaStorageManagerImpl.java:849) at org.apache.ignite.internal.metastorage.impl.MetaStorageManagerImpl$1.onSafeTimeAdvanced(MetaStorageManagerImpl.java:456) at org.apache.ignite.internal.metastorage.server.WatchProcessor.lambda$advanceSafeTime$7(WatchProcessor.java:269) at java.base/java.util.concurrent.CompletableFuture$UniRun.tryFire(CompletableFuture.java:783) at java.base/java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:478) 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) at Apache.Ignite.Internal.ClientFailoverSocket.DoOutInOpAndGetSocketAsync(ClientOp clientOp, Transaction tx, PooledArrayBuffer request, PreferredNode preferredNode, IRetryPolicy retryPolicyOverride) in /opt/buildagent/work/b8d4df1365f1f1e5/modules/platforms/dotnet/Apache.Ignite/Internal/ClientFailoverSocket.cs:line 195 at Apache.Ignite.Internal.Table.RecordView`1.DoOutInOpAsync(ClientOp clientOp, Transaction tx, PooledArrayBuffer request, PreferredNode preferredNode, IRetryPolicy retryPolicyOverride) in /opt/buildagent/work/b8d4df1365f1f1e5/modules/platforms/dotnet/Apache.Ignite/Internal/Table/RecordView.cs:line 400 at Apache.Ignite.Internal.Table.RecordView`1.<StreamDataAsync>b__29_0(PooledArrayBuffer batch, String preferredNode, IRetryPolicy retryPolicy) in /opt/buildagent/work/b8d4df1365f1f1e5/modules/platforms/dotnet/Apache.Ignite/Internal/Table/RecordView.cs:line 304 at Apache.Ignite.Internal.Table.DataStreamer.<>c__DisplayClass1_0`1.<<StreamDataAsync>g__SendAndDisposeBufAsync|5>d.MoveNext() in /opt/buildagent/work/b8d4df1365f1f1e5/modules/platforms/dotnet/Apache.Ignite/Internal/Table/DataStreamer.cs:line 300 --- End of stack trace from previous location --- at Apache.Ignite.Internal.Table.DataStreamer.<>c__DisplayClass1_0`1.<<StreamDataAsync>g__SendAsync|4>d.MoveNext() in /opt/buildagent/work/b8d4df1365f1f1e5/modules/platforms/dotnet/Apache.Ignite/Internal/Table/DataStreamer.cs:line 231 --- End of stack trace from previous location --- at Apache.Ignite.Internal.Table.DataStreamer.StreamDataAsync[T](IAsyncEnumerable`1 data, Func`4 sender, RecordSerializer`1 writer, Func`2 schemaProvider, Func`1 partitionAssignmentProvider, DataStreamerOptions options, CancellationToken cancellationToken) in /opt/buildagent/work/b8d4df1365f1f1e5/modules/platforms/dotnet/Apache.Ignite/Internal/Table/DataStreamer.cs:line 103 at Apache.Ignite.Internal.Table.DataStreamer.StreamDataAsync[T](IAsyncEnumerable`1 data, Func`4 sender, RecordSerializer`1 writer, Func`2 schemaProvider, Func`1 partitionAssignmentProvider, DataStreamerOptions options, CancellationToken cancellationToken) in /opt/buildagent/work/b8d4df1365f1f1e5/modules/platforms/dotnet/Apache.Ignite/Internal/Table/DataStreamer.cs:line 94 at Apache.Ignite.Internal.Table.RecordView`1.StreamDataAsync(IAsyncEnumerable`1 data, DataStreamerOptions options, CancellationToken cancellationToken) in /opt/buildagent/work/b8d4df1365f1f1e5/modules/platforms/dotnet/Apache.Ignite/Internal/Table/RecordView.cs:line 300 at Apache.Ignite.Tests.Table.SchemaSynchronizationTest.TestSchemaUpdateWhileStreaming(Boolean insertNewColumn) in /opt/buildagent/work/b8d4df1365f1f1e5/modules/platforms/dotnet/Apache.Ignite.Tests/Table/SchemaSynchronizationTest.cs:line 348
Attachments
Issue Links
- is blocked by
-
IGNITE-20416 IncompatibleSchemaException thrown when schema change is compatible
- Resolved
- relates to
-
IGNITE-20415 Internal IncompatibleSchemaException is thrown from public API
- Resolved
- links to