tkalkirill commented on code in PR #2450:
URL: https://github.com/apache/ignite-3/pull/2450#discussion_r1302491260


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/schema/CheckCatalogVersionOnAppendEntries.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.table.distributed.schema;
+
+import java.nio.ByteBuffer;
+import org.apache.ignite.internal.catalog.CatalogService;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.raft.jraft.Node;
+import org.apache.ignite.raft.jraft.entity.EnumOutter.EntryType;
+import org.apache.ignite.raft.jraft.entity.RaftOutter;
+import org.apache.ignite.raft.jraft.entity.RaftOutter.EntryMeta;
+import org.apache.ignite.raft.jraft.error.RaftError;
+import org.apache.ignite.raft.jraft.rpc.Message;
+import org.apache.ignite.raft.jraft.rpc.RaftRpcFactory;
+import org.apache.ignite.raft.jraft.rpc.RaftServerService;
+import org.apache.ignite.raft.jraft.rpc.RpcRequestClosure;
+import org.apache.ignite.raft.jraft.rpc.RpcRequests.AppendEntriesRequest;
+import org.apache.ignite.raft.jraft.rpc.impl.AppendEntriesRequestInterceptor;
+import org.apache.ignite.raft.jraft.util.Marshaller;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * An {@link AppendEntriesRequestInterceptor} that rejects requests (by 
returning EBUSY error code) if any of the
+ * incoming commands requires catalog version that is not available locally 
yet.
+ */
+public class CheckCatalogVersionOnAppendEntries implements 
AppendEntriesRequestInterceptor {
+    private static final IgniteLogger LOG = 
Loggers.forClass(CheckCatalogVersionOnAppendEntries.class);
+
+    private static final int NO_VERSION_REQUIREMENT = Integer.MIN_VALUE;
+
+    private final CatalogService catalogService;
+
+    public CheckCatalogVersionOnAppendEntries(CatalogService catalogService) {
+        this.catalogService = catalogService;
+    }
+
+    @Override
+    @Nullable
+    public Message intercept(RaftServerService service, AppendEntriesRequest 
request, RpcRequestClosure done) {

Review Comment:
   ```suggestion
       public @Nullable Message intercept(RaftServerService service, 
AppendEntriesRequest request, RpcRequestClosure done) {
   ```



##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/schemasync/ItSchemaSyncAndReplicationTest.java:
##########
@@ -0,0 +1,173 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.schemasync;
+
+import static org.apache.ignite.internal.SessionUtils.executeUpdate;
+import static 
org.apache.ignite.internal.testframework.IgniteTestUtils.waitForCondition;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.notNullValue;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BooleanSupplier;
+import java.util.logging.LogRecord;
+import java.util.logging.Logger;
+import org.apache.ignite.internal.ClusterPerTestIntegrationTest;
+import org.apache.ignite.internal.ReplicationGroupsUtils;
+import org.apache.ignite.internal.app.IgniteImpl;
+import org.apache.ignite.internal.metastorage.server.raft.MetastorageGroupId;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.storage.RowId;
+import org.apache.ignite.internal.table.TableImpl;
+import 
org.apache.ignite.internal.table.distributed.schema.CheckCatalogVersionOnAppendEntries;
+import org.apache.ignite.internal.test.WatchListenerInhibitor;
+import org.apache.ignite.internal.testframework.jul.NoOpHandler;
+import org.apache.ignite.table.KeyValueView;
+import org.apache.ignite.table.Tuple;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Tests about interaction between Schema Synchronization and Replication.
+ */
+@SuppressWarnings("resource")
+class ItSchemaSyncAndReplicationTest extends ClusterPerTestIntegrationTest {
+    @Override
+    protected int initialNodes() {
+        return 3;
+    }
+
+    /**
+     * The replication mechanism must not replicate commands for which schemas 
are not yet available on the node
+     * to which replication happens (in Raft, it means that followers/learners 
cannot receive commands that they
+     * cannot execute without waiting for schemas). This method tests this 
scenario.
+     */
+    @Test
+    void laggingSchemasPreventPartitionDataReplication() throws Exception {
+        createTestTableWith3Replicas();
+
+        final int notInhibitedNodeIndex = 0;
+        transferLeadershipsTo(notInhibitedNodeIndex);
+
+        IgniteImpl nodeToInhibitMetaStorage = cluster.node(1);
+
+        WatchListenerInhibitor listenerInhibitor = 
WatchListenerInhibitor.metastorageEventsInhibitor(nodeToInhibitMetaStorage);
+        listenerInhibitor.startInhibit();
+
+        try {
+            CountDownLatch rejectionTriggered = 
rejectionDueToMetadataLagTriggered();
+
+            updateTableSchemaAt(notInhibitedNodeIndex);
+            putToTableAt(notInhibitedNodeIndex);
+
+            assertTrue(rejectionTriggered.await(10, TimeUnit.SECONDS), "Did 
not see rejections due to lagging metadata");
+
+            assertTrue(solePartitionIsEmpty(nodeToInhibitMetaStorage), 
"Something was written to the partition");
+
+            listenerInhibitor.stopInhibit();
+
+            assertTrue(
+                    waitForCondition(() -> 
!solePartitionIsEmpty(nodeToInhibitMetaStorage), 10_000),
+                    "Nothing was written to partition even after inhibiting 
was cancelled"
+            );
+        } finally {
+            listenerInhibitor.stopInhibit();
+        }
+    }
+
+    private void createTestTableWith3Replicas() throws InterruptedException {
+        String zoneSql = "create zone test_zone with partitions=1, replicas=3";
+        String sql = "create table test (key int primary key, value 
varchar(20))"
+                + " with primary_zone='TEST_ZONE'";
+
+        cluster.doInSession(0, session -> {
+            executeUpdate(zoneSql, session);
+            executeUpdate(sql, session);
+        });
+
+        waitForTableToStart();
+    }
+
+    private void waitForTableToStart() throws InterruptedException {
+        // TODO: IGNITE-18733 - remove this wait because when a table creation 
query is executed, the table must be fully ready.
+
+        BooleanSupplier tableStarted = () -> {
+            int numberOfStartedRaftNodes = cluster.runningNodes()
+                    .map(ReplicationGroupsUtils::tablePartitionIds)
+                    .mapToInt(List::size)
+                    .sum();
+            return numberOfStartedRaftNodes == 3;
+        };
+
+        assertTrue(waitForCondition(tableStarted, 10_000), "Did not see all 
table RAFT nodes started");
+    }
+
+    private void transferLeadershipsTo(int nodeIndex) throws 
InterruptedException {
+        cluster.transferLeadershipTo(nodeIndex, MetastorageGroupId.INSTANCE);
+        cluster.transferLeadershipTo(nodeIndex, cluster.solePartitionId());
+    }
+
+    private static CountDownLatch rejectionDueToMetadataLagTriggered() {
+        Logger interceptorLogger = 
Logger.getLogger(CheckCatalogVersionOnAppendEntries.class.getName());
+
+        CountDownLatch rejectionTriggered = new CountDownLatch(1);

Review Comment:
   Ease of use, instead of `assertTrue(rejectionTriggered.await(10, 
TimeUnit.SECONDS), "Did not see rejections due to lagging metadata");` you can 
use `assertThat(future, willCompleteSuccessfully());`, up to u.
   



##########
modules/raft/src/main/java/org/apache/ignite/internal/raft/server/impl/JraftServerImpl.java:
##########
@@ -122,10 +125,12 @@ public class JraftServerImpl implements RaftServer {
     private ExecutorService requestExecutor;
 
     /** Marshaller for RAFT commands. */
-    private final Marshaller commandsMarshaller;
+    private final Marshaller defaultCommandsMarshaller;
 
     /** Raft service event interceptor. */
-    private RaftServiceEventInterceptor serviceEventInterceptor;
+    private final RaftServiceEventInterceptor serviceEventInterceptor;
+
+    private AppendEntriesRequestInterceptor appendEntriesRequestInterceptor = 
new NullAppendEntriesRequestInterceptor();

Review Comment:
   Let's point it out that it's not thread safe.



##########
modules/raft/src/main/java/org/apache/ignite/internal/raft/server/impl/JraftServerImpl.java:
##########
@@ -122,10 +125,12 @@ public class JraftServerImpl implements RaftServer {
     private ExecutorService requestExecutor;
 
     /** Marshaller for RAFT commands. */
-    private final Marshaller commandsMarshaller;
+    private final Marshaller defaultCommandsMarshaller;

Review Comment:
   Please fix it.



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java:
##########
@@ -1158,25 +1169,31 @@ private CompletableFuture<Object> 
finishTransaction(List<TablePartitionId> aggre
         HybridTimestamp currentTimestamp = hybridClock.now();
         HybridTimestamp commitTimestamp = commit ? currentTimestamp : null;
 
-        FinishTxCommandBuilder finishTxCmdBldr = MSG_FACTORY.finishTxCommand()
-                .txId(txId)
-                .commit(commit)
-                .safeTimeLong(currentTimestamp.longValue())
-                .tablePartitionIds(
-                        aggregatedGroupIds.stream()
-                                
.map(PartitionReplicaListener::tablePartitionId)
-                                .collect(toList())
-                );
-
-        if (commit) {
-            finishTxCmdBldr.commitTimestampLong(commitTimestamp.longValue());
-        }
+        return catalogVersionFor(currentTimestamp)
+                .thenApply(catalogVersion -> {

Review Comment:
   wait



##########
modules/runner/src/testFixtures/java/org/apache/ignite/internal/Cluster.java:
##########
@@ -523,6 +531,74 @@ public void removeNetworkPartitionOf(int nodeIndex) {
         LOG.info("Reanimated node " + nodeIndex + " by removing an artificial 
network partition");
     }
 
+    /**
+     * Transfers leadsership over a replication group to a node identified by 
the given index.
+     *
+     * @param nodeIndex Node index of the new leader.
+     * @param groupId ID of the replication group.
+     * @throws InterruptedException If interrupted while waiting.
+     */
+    public void transferLeadershipTo(int nodeIndex, ReplicationGroupId 
groupId) throws InterruptedException {
+        String nodeConsistentId = node(nodeIndex).node().name();
+
+        int maxAttempts = 3;
+
+        for (int attempt = 1; attempt <= maxAttempts; attempt++) {
+            boolean transferred = tryTransferLeadershipTo(nodeConsistentId, 
groupId);
+
+            if (transferred) {
+                break;
+            }
+
+            if (attempt < maxAttempts) {
+                LOG.info("Did not transfer leadership after " + attempt + " 
attempts, going to retry...");
+            } else {
+                fail("Did not transfer leadership in time after " + 
maxAttempts + " attempts");

Review Comment:
   =)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to