You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by GitBox <gi...@apache.org> on 2021/09/06 13:11:22 UTC

[GitHub] [ignite] vmalin commented on a change in pull request #9362: IGNITE-9386 Fix cofused results of control.sh --tx when limit is set to a small number

vmalin commented on a change in pull request #9362:
URL: https://github.com/apache/ignite/pull/9362#discussion_r702832727



##########
File path: modules/core/src/test/java/org/apache/ignite/internal/visor/tx/VisorTxTaskExecutionTest.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.visor.tx;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxFinishRequest;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.visor.VisorTaskArgument;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.transactions.Transaction;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+import static org.apache.ignite.internal.TestRecordingCommunicationSpi.spi;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+
+/**
+ * VisorTxTask integration test
+ */
+public class VisorTxTaskExecutionTest extends GridCommonAbstractTest {
+    /** Cache name. */
+    protected static final String CACHE_NAME = "test";
+
+    /** Server node count. */
+    private static final int GRID_CNT = 3;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        if (!igniteInstanceName.startsWith("client")) {
+            CacheConfiguration ccfg = new CacheConfiguration(CACHE_NAME);
+
+            ccfg.setAtomicityMode(TRANSACTIONAL);
+            ccfg.setBackups(GRID_CNT - 1);
+            ccfg.setWriteSynchronizationMode(FULL_SYNC);
+            ccfg.setOnheapCacheEnabled(false);
+
+            cfg.setCacheConfiguration(ccfg);
+        }
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        final IgniteEx crd = startGrid(0);
+
+        startGridsMultiThreaded(1, GRID_CNT - 1);
+
+        crd.cluster().active(true);

Review comment:
       _active(true)_ is deprecated

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/visor/tx/VisorTxTaskExecutionTest.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.visor.tx;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxFinishRequest;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.visor.VisorTaskArgument;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.transactions.Transaction;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+import static org.apache.ignite.internal.TestRecordingCommunicationSpi.spi;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+
+/**
+ * VisorTxTask integration test
+ */
+public class VisorTxTaskExecutionTest extends GridCommonAbstractTest {
+    /** Cache name. */
+    protected static final String CACHE_NAME = "test";
+
+    /** Server node count. */
+    private static final int GRID_CNT = 3;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        if (!igniteInstanceName.startsWith("client")) {
+            CacheConfiguration ccfg = new CacheConfiguration(CACHE_NAME);
+
+            ccfg.setAtomicityMode(TRANSACTIONAL);
+            ccfg.setBackups(GRID_CNT - 1);
+            ccfg.setWriteSynchronizationMode(FULL_SYNC);
+            ccfg.setOnheapCacheEnabled(false);
+
+            cfg.setCacheConfiguration(ccfg);
+        }
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        final IgniteEx crd = startGrid(0);
+
+        startGridsMultiThreaded(1, GRID_CNT - 1);
+
+        crd.cluster().active(true);
+
+        awaitPartitionMapExchange();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();

Review comment:
       It's better to call _super.afterTest()_ at the end of the current _afterTest()_. The idea is to first call more specific ending actions and then more general. _beforeTest()_ works vice versa. 

##########
File path: modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/TxCommands.java
##########
@@ -138,8 +139,12 @@ else if (args.getOperation() == VisorTxOperation.KILL)
                     ", consistentId=" + key.consistentId() +
                     "]");
 
-                for (VisorTxInfo info : entry.getValue().getInfos())
-                    logger.info(info.toUserString());
+                for (VisorTxInfo info : entry.getValue().getInfos()) {
+                    if (Objects.equals(info.getXid(), info.getNearXid()))
+                        logger.info(info.toUserString());
+                    else
+                        logger.info(info.toUserString() + " [Near node may be missed]");

Review comment:
       Near node **might** be miss**ing**. Or better "Near node might have left topology." Left **the** topology is grammatically correct, but _left toology_ is used in many places including logs.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxTask.java
##########
@@ -74,6 +73,9 @@
     /** */
     private static final long serialVersionUID = 0L;
 
+    /** */
+    private static final int DEFAULT_REDUCE_LIMIT = 5;

Review comment:
       The logic looks a bit flawed to me. You use different values for map and reduce limits if the limit isn't set explicitly. However, if it is, you use the same value. I think default values should be equal. Otherwise, we will do some computations that will be discarded at the end(either in _map_ or in _reduce_ methods)

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/visor/tx/VisorTxTaskExecutionTest.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.visor.tx;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxFinishRequest;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.visor.VisorTaskArgument;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.transactions.Transaction;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+import static org.apache.ignite.internal.TestRecordingCommunicationSpi.spi;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+
+/**
+ * VisorTxTask integration test
+ */
+public class VisorTxTaskExecutionTest extends GridCommonAbstractTest {
+    /** Cache name. */
+    protected static final String CACHE_NAME = "test";
+
+    /** Server node count. */
+    private static final int GRID_CNT = 3;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        if (!igniteInstanceName.startsWith("client")) {
+            CacheConfiguration ccfg = new CacheConfiguration(CACHE_NAME);
+
+            ccfg.setAtomicityMode(TRANSACTIONAL);
+            ccfg.setBackups(GRID_CNT - 1);
+            ccfg.setWriteSynchronizationMode(FULL_SYNC);
+            ccfg.setOnheapCacheEnabled(false);
+
+            cfg.setCacheConfiguration(ccfg);
+        }
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        final IgniteEx crd = startGrid(0);
+
+        startGridsMultiThreaded(1, GRID_CNT - 1);
+
+        crd.cluster().active(true);
+
+        awaitPartitionMapExchange();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @return Started client.
+     * @throws Exception If f nodeailed.
+     */
+    private Ignite startClient() throws Exception {
+        Ignite client = startClientGrid("client");
+
+        assertTrue(client.configuration().isClientMode());
+
+        assertNotNull(client.cache(CACHE_NAME));
+
+        return client;
+    }
+
+
+    /** Test limit parameter for VisorTxTask */
+    @Test
+    public void testVisorTxTaskLimitParam() throws Exception {
+        final int txCnt = 50;
+        final long latchTimeoutSeconds = 60;
+        final int testLimit = 10;
+
+        final Ignite client = startClient();
+
+        final List<Integer> keys = primaryKeys(grid(0).cache(CACHE_NAME), txCnt);
+
+        CountDownLatch txLatch = new CountDownLatch(txCnt);
+        CountDownLatch commitLatch = new CountDownLatch(1);
+
+        List<IgniteInternalFuture> futures = new ArrayList<>(txCnt);
+
+        for (int i = 0; i < txCnt; i++) {
+
+            final int f = i;
+
+            futures.add(runAsync(new Runnable() {

Review comment:
       No need to explicitly use _new Runnable()_. () -> construction will also work

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxTask.java
##########
@@ -217,7 +224,10 @@ private VisorTxJob(VisorTxTaskArg arg, boolean debug) {
 
             List<VisorTxInfo> infos = new ArrayList<>();
 
-            int limit = arg.getLimit() == null ? DEFAULT_LIMIT : arg.getLimit();
+            int perNodelimit = DEFAULT_MAP_LIMIT;
+

Review comment:
       may be shortened:
   int perNodeLimit = arg.getLimit == null ? DEFAULT_MAP_LIMIT : max(arg.getLimit, DEFAULT_MAP_LIMIT);

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxTask.java
##########
@@ -325,15 +335,15 @@ else if (locTx instanceof GridDhtTxRemote) {
 
                 TxVerboseInfo verboseInfo = arg.verboseMode() ? createVerboseInfo(ignite, locTx) : null;
 
+                if (arg.getOperation() == VisorTxOperation.KILL)

Review comment:
       Is there any reason to move this string up?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxTask.java
##########
@@ -120,7 +122,11 @@
 
     /** {@inheritDoc} */
     @Nullable @Override protected Map<ClusterNode, VisorTxTaskResult> reduce0(List<ComputeJobResult> results) throws IgniteException {
-        Map<ClusterNode, VisorTxTaskResult> mapRes = new TreeMap<>();
+        int limit = taskArg.getLimit() == null ? DEFAULT_REDUCE_LIMIT : taskArg.getLimit();
+
+        if (limit <= 0) return Collections.emptyMap();

Review comment:
       _return_ statement should occupy a separate line

##########
File path: modules/core/src/test/java/org/apache/ignite/testsuites/IgniteVisorTaskTestSuite.java
##########
@@ -0,0 +1,34 @@
+/*
+ * 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.testsuites;
+
+import org.apache.ignite.internal.visor.tx.VisorTxTaskExecutionTest;
+import org.apache.ignite.internal.visor.tx.VisorTxTaskTest;
+import org.junit.runner.RunWith;
+import org.junit.runners.Suite;
+
+/**
+ * Test suite for Visor task execution.
+ */
+@RunWith(Suite.class)
+@Suite.SuiteClasses({
+    VisorTxTaskTest.class,
+    VisorTxTaskExecutionTest.class
+})
+public class IgniteVisorTaskTestSuite {

Review comment:
       I can't see the new tests in the _Run all_ used to get a visa. Seems that something more should be done to register a new suite

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/visor/tx/VisorTxTaskExecutionTest.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.visor.tx;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxFinishRequest;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.visor.VisorTaskArgument;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.transactions.Transaction;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+import static org.apache.ignite.internal.TestRecordingCommunicationSpi.spi;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+
+/**
+ * VisorTxTask integration test
+ */
+public class VisorTxTaskExecutionTest extends GridCommonAbstractTest {
+    /** Cache name. */
+    protected static final String CACHE_NAME = "test";
+
+    /** Server node count. */
+    private static final int GRID_CNT = 3;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        if (!igniteInstanceName.startsWith("client")) {
+            CacheConfiguration ccfg = new CacheConfiguration(CACHE_NAME);
+
+            ccfg.setAtomicityMode(TRANSACTIONAL);
+            ccfg.setBackups(GRID_CNT - 1);
+            ccfg.setWriteSynchronizationMode(FULL_SYNC);
+            ccfg.setOnheapCacheEnabled(false);
+
+            cfg.setCacheConfiguration(ccfg);
+        }
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        final IgniteEx crd = startGrid(0);
+
+        startGridsMultiThreaded(1, GRID_CNT - 1);
+
+        crd.cluster().active(true);
+
+        awaitPartitionMapExchange();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @return Started client.
+     * @throws Exception If f nodeailed.
+     */
+    private Ignite startClient() throws Exception {
+        Ignite client = startClientGrid("client");
+
+        assertTrue(client.configuration().isClientMode());
+
+        assertNotNull(client.cache(CACHE_NAME));
+
+        return client;
+    }
+
+
+    /** Test limit parameter for VisorTxTask */
+    @Test
+    public void testVisorTxTaskLimitParam() throws Exception {
+        final int txCnt = 50;
+        final long latchTimeoutSeconds = 60;
+        final int testLimit = 10;
+
+        final Ignite client = startClient();
+
+        final List<Integer> keys = primaryKeys(grid(0).cache(CACHE_NAME), txCnt);
+
+        CountDownLatch txLatch = new CountDownLatch(txCnt);
+        CountDownLatch commitLatch = new CountDownLatch(1);
+
+        List<IgniteInternalFuture> futures = new ArrayList<>(txCnt);
+
+        for (int i = 0; i < txCnt; i++) {
+
+            final int f = i;
+
+            futures.add(runAsync(new Runnable() {
+                @Override public void run() {
+                    try (Transaction tx = client.transactions().txStart()) {
+                        client.cache(CACHE_NAME).put(keys.get(f), 0);
+
+                        txLatch.countDown();
+
+                        U.await(commitLatch, latchTimeoutSeconds, TimeUnit.SECONDS);
+
+                        tx.commit();
+                    }
+                    catch (Exception e) {
+                        if (!(e.getCause() instanceof NodeStoppingException))
+                            log.error("Error while transaction executing.", e);
+
+                        //NodeStoppingException expected.
+                    }
+                }
+            }));
+        }
+
+        spi(client).blockMessages((node, msg) -> msg instanceof GridNearTxFinishRequest);
+
+        U.awaitQuiet(txLatch);
+
+        commitLatch.countDown();
+
+        VisorTxTaskArg arg =
+            new VisorTxTaskArg(VisorTxOperation.INFO, testLimit, null, null, null, null, null, null, null, null, null);
+
+        Map<ClusterNode, VisorTxTaskResult> res = client.compute(client.cluster().forPredicate(F.alwaysTrue())).
+            execute(new VisorTxTask(), new VisorTaskArgument<>(client.cluster().localNode().id(), arg, false));
+
+        //All transactions are in PREPARED state.
+        for (Map.Entry<ClusterNode, VisorTxTaskResult> entry : res.entrySet()) {
+            if (entry.getValue().getInfos().isEmpty())
+                fail("Every node should have transaction info.");
+
+            assertEquals(testLimit, entry.getValue().getInfos().size());
+        }
+
+        arg = new VisorTxTaskArg(VisorTxOperation.KILL, null, null, null, null, null, null, null, null, null, null);
+
+        client.compute(client.cluster().forPredicate(F.alwaysTrue()))

Review comment:
       The result of this compute operation should also be checked.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxTask.java
##########
@@ -120,7 +122,11 @@
 
     /** {@inheritDoc} */
     @Nullable @Override protected Map<ClusterNode, VisorTxTaskResult> reduce0(List<ComputeJobResult> results) throws IgniteException {
-        Map<ClusterNode, VisorTxTaskResult> mapRes = new TreeMap<>();
+        int limit = taskArg.getLimit() == null ? DEFAULT_REDUCE_LIMIT : taskArg.getLimit();
+
+        if (limit <= 0) return Collections.emptyMap();
+
+        Map<ClusterNode, VisorTxTaskResult> mapRes = new HashMap<>();

Review comment:
       You changed the data structure here. If there is no good reason for this I strongly advise to leave it as it was.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxTask.java
##########
@@ -217,7 +224,10 @@ private VisorTxJob(VisorTxTaskArg arg, boolean debug) {
 
             List<VisorTxInfo> infos = new ArrayList<>();
 
-            int limit = arg.getLimit() == null ? DEFAULT_LIMIT : arg.getLimit();
+            int perNodelimit = DEFAULT_MAP_LIMIT;

Review comment:
       perNode**L**imit

##########
File path: modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/TxCommands.java
##########
@@ -138,8 +139,12 @@ else if (args.getOperation() == VisorTxOperation.KILL)
                     ", consistentId=" + key.consistentId() +
                     "]");
 
-                for (VisorTxInfo info : entry.getValue().getInfos())
-                    logger.info(info.toUserString());
+                for (VisorTxInfo info : entry.getValue().getInfos()) {
+                    if (Objects.equals(info.getXid(), info.getNearXid()))
+                        logger.info(info.toUserString());
+                    else
+                        logger.info(info.toUserString() + " [Near node may be missed]");
+                }

Review comment:
       Also looks like that a remark about missing node is correct only if the mode is non-verbose. In verbose mode we don't remove transactions that meet this condition.




-- 
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: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org