IGNITE-27460 Add schema compatibility validation for full commands#7598
IGNITE-27460 Add schema compatibility validation for full commands#7598rpuch wants to merge 11 commits intoapache:mainfrom
Conversation
fdf9a30 to
b65f713
Compare
There was a problem hiding this comment.
Pull request overview
This pull request implements schema compatibility validation for full (1PC) transaction commands in the Raft replication layer. The approach validates commands after safe time assignment but before they are added to the Raft log on the leader, preventing inconsistencies that could arise from forward-incompatible schema changes.
Changes:
- Introduces a
SafeTimeValidatorextension point in Raft that validates commands before replication, with support for both temporary retry (EBUSY) and permanent rejection (EREJECTED_BY_USER_LOGIC) - Implements
PartitionSafeTimeValidatorto validate full update commands against schema compatibility rules, ensuring commit timestamps are valid for the schema versions used - Updates
PartitionReplicaListenerto handle validation failures by converting EREJECTED_BY_USER_LOGIC errors toIncompatibleSchemaVersionException - Adds comprehensive integration tests verifying that schema version consistency is maintained across nodes during concurrent schema changes and writes
Reviewed changes
Copilot reviewed 44 out of 44 changed files in this pull request and generated 4 comments.
Show a summary per file
| File | Description |
|---|---|
| modules/raft/src/main/java/org/apache/ignite/raft/jraft/option/SafeTimeValidator.java | New interface for safe time validation in Raft |
| modules/raft/src/main/java/org/apache/ignite/raft/jraft/option/SafeTimeValidationResult.java | New class representing validation results (valid, retry, or rejected) |
| modules/raft/src/main/java/org/apache/ignite/raft/jraft/option/PermissiveSafeTimeValidator.java | Default no-op validator implementation |
| modules/raft/src/main/java/org/apache/ignite/raft/jraft/core/NodeImpl.java | Integration of validation in Raft leader's executeApplyingTasks |
| modules/raft/src/main/java/org/apache/ignite/raft/jraft/error/RaftError.java | New EREJECTED_BY_USER_LOGIC error code for permanent rejections |
| modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/PartitionSafeTimeValidator.java | Partition-specific validator implementation |
| modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java | Error handling for validation failures |
| modules/partition-replicator/src/main/java/org/apache/ignite/internal/partition/replicator/network/command/UpdateCommandBase.java | New base interface for update commands |
| modules/partition-replicator/src/main/java/org/apache/ignite/internal/partition/replicator/schemacompat/CompatValidationResult.java | Added validationFailedMessage() method |
| modules/runner/src/integrationTest/java/org/apache/ignite/internal/schemasync/ItSchemaForwardCompatibilityConsistencyTest.java | New integration test verifying schema consistency |
| modules/runner/src/integrationTest/java/org/apache/ignite/internal/schemasync/ItBlockedSchemaSyncAndRaftCommandExecutionTest.java | Test verifying node stop behavior with blocked schema sync |
Comments suppressed due to low confidence (1)
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/schema/MetadataSufficiency.java:44
- The JavaDoc for isMetadataAvailableForTimestamp incorrectly says "Determines whether the local Catalog version is sufficient" when it should say something like "Determines whether the local schema metadata is sufficient for the given timestamp".
💡 Add Copilot custom instructions for smarter, more guided reviews. Learn how to get started.
...n/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java
Show resolved
Hide resolved
...ain/java/org/apache/ignite/internal/partition/replicator/raft/ZonePartitionRaftListener.java
Show resolved
Hide resolved
...va/org/apache/ignite/internal/schemasync/ItBlockedSchemaSyncAndRaftCommandExecutionTest.java
Outdated
Show resolved
Hide resolved
modules/raft/src/main/java/org/apache/ignite/raft/jraft/error/RaftError.java
Outdated
Show resolved
Hide resolved
* Introduce a mechanism to validate command+safeTime in Raft - this happens after we assign a safeTime to a command, but before it gets saved to log on the leader (and hence before it gets replicated) - this allows to request a retry of the same command by the Raft client (if the failure is temporary, like insufficient schemas information on the node) or instruct it to fail the command if it will never become valid * Introduce full commands validation wrt schema compatibility - do the validation in the Raft extension point introduced above (as safeTime becomes commitTs for full updates) - if the node lacks schema information, fail the validation requesting a retry from the Raft client - if the commitTs is invalid, return the result to Replica listener - in Replica listener, handle the failure by retrying the replica request (using updated schema)
b65f713 to
02b9d0b
Compare
...n/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java
Outdated
Show resolved
Hide resolved
modules/raft/src/main/java/org/apache/ignite/raft/jraft/core/NodeImpl.java
Show resolved
Hide resolved
...ava/org/apache/ignite/internal/partition/replicator/schemacompat/CompatValidationResult.java
Show resolved
Hide resolved
.../main/java/org/apache/ignite/internal/table/distributed/raft/PartitionSafeTimeValidator.java
Outdated
Show resolved
Hide resolved
.../main/java/org/apache/ignite/internal/table/distributed/raft/PartitionSafeTimeValidator.java
Outdated
Show resolved
Hide resolved
...e/src/main/java/org/apache/ignite/internal/table/distributed/schema/MetadataSufficiency.java
Show resolved
Hide resolved
...e/src/main/java/org/apache/ignite/internal/table/distributed/schema/MetadataSufficiency.java
Outdated
Show resolved
Hide resolved
.../org/apache/ignite/internal/table/distributed/schema/CheckCatalogVersionOnActionRequest.java
Show resolved
Hide resolved
.../org/apache/ignite/internal/table/distributed/schema/CheckCatalogVersionOnAppendEntries.java
Outdated
Show resolved
Hide resolved
modules/raft/src/main/java/org/apache/ignite/raft/jraft/core/NodeImpl.java
Show resolved
Hide resolved
...n/java/org/apache/ignite/internal/partition/replicator/network/command/UpdateAllCommand.java
Show resolved
Hide resolved
...tFixtures/java/org/apache/ignite/internal/testframework/asserts/CompletableFutureAssert.java
Outdated
Show resolved
Hide resolved
...rc/integrationTest/java/org/apache/ignite/internal/rebalance/ItRebalanceDistributedTest.java
Outdated
Show resolved
Hide resolved
...va/org/apache/ignite/internal/schemasync/ItBlockedSchemaSyncAndRaftCommandExecutionTest.java
Outdated
Show resolved
Hide resolved
...va/org/apache/ignite/internal/schemasync/ItBlockedSchemaSyncAndRaftCommandExecutionTest.java
Outdated
Show resolved
Hide resolved
...va/org/apache/ignite/internal/schemasync/ItBlockedSchemaSyncAndRaftCommandExecutionTest.java
Show resolved
Hide resolved
...va/org/apache/ignite/internal/schemasync/ItBlockedSchemaSyncAndRaftCommandExecutionTest.java
Show resolved
Hide resolved
...va/org/apache/ignite/internal/schemasync/ItBlockedSchemaSyncAndRaftCommandExecutionTest.java
Show resolved
Hide resolved
.../main/java/org/apache/ignite/internal/table/distributed/raft/PartitionSafeTimeValidator.java
Outdated
Show resolved
Hide resolved
...e/src/main/java/org/apache/ignite/internal/table/distributed/schema/MetadataSufficiency.java
Show resolved
Hide resolved
| * Returns error message corresponding to validation failure. Should only be called for a failed validation result, otherwise an | ||
| * assertion error may be thrown. | ||
| */ | ||
| public String validationFailedMessage() { |
There was a problem hiding this comment.
Should we instead provide the exception itself:
public Exception validationException()
This implies better incapsulation.
There was a problem hiding this comment.
Different exceptions can be constructed with this error message. Also, the Raft safe time validator doesn't need the exception itself, just the error message to construct a response message. So I think error message is more suitable here
https://issues.apache.org/jira/browse/IGNITE-27460
What I tried before
In this PR (approach number 3), the commands are unconditional as before, and we make the decision 'to replicate the command or not' on just one node (on the leader) once.