This is an automated email from the ASF dual-hosted git repository.

ppa pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/master by this push:
     new 6dcfbd9ee5f IGNITE-16805 Fixed an issue with stopping a node during 
partition map exchange (#9945)
6dcfbd9ee5f is described below

commit 6dcfbd9ee5faa26240f187b2ef23b7874e362038
Author: Pavel Pereslegin <xxt...@gmail.com>
AuthorDate: Wed Apr 13 11:01:01 2022 +0300

    IGNITE-16805 Fixed an issue with stopping a node during partition map 
exchange (#9945)
---
 .../cache/GridCachePartitionExchangeManager.java   |   7 +-
 .../preloader/GridDhtPartitionsExchangeFuture.java |   3 +
 .../GridCacheAbstractNodeRestartSelfTest.java      | 435 ++++++++++-----------
 3 files changed, 203 insertions(+), 242 deletions(-)

diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
index 6a06e8c6483..62d937f537f 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
@@ -1888,10 +1888,11 @@ public class GridCachePartitionExchangeManager<K, V> 
extends GridCacheSharedMana
                 if (initTopVer.compareTo(fut.exchangeId().topologyVersion()) < 
0)
                     continue;
 
-                skipped++;
-
-                if (skipped > 10)
+                // Skip recent exchange futures (ignore futures that have been 
merged into another).
+                if (skipped == 10 || fut.isMerged())
                     fut.cleanUp();
+                else
+                    skipped++;
             }
         }
     }
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index 3e49fbfbf66..ce27be0e652 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
@@ -5520,6 +5520,9 @@ public class GridDhtPartitionsExchangeFuture extends 
GridDhtTopologyFutureAdapte
             delayedLatestMsg = fullMsg;
 
             listen(f -> {
+                if (f.error() != null)
+                    return;
+
                 GridDhtPartitionsFullMessage msg;
 
                 synchronized (this) {
diff --git 
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractNodeRestartSelfTest.java
 
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractNodeRestartSelfTest.java
index 9a62cfcd33a..a2ff1d6cb31 100644
--- 
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractNodeRestartSelfTest.java
+++ 
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractNodeRestartSelfTest.java
@@ -18,10 +18,8 @@
 package org.apache.ignite.internal.processors.cache.distributed;
 
 import java.util.ArrayList;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.LinkedHashMap;
-import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
@@ -39,8 +37,10 @@ import org.apache.ignite.cache.CacheRebalanceMode;
 import org.apache.ignite.cache.eviction.lru.LruEvictionPolicy;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
 import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.GridTestUtils.SF;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
@@ -590,111 +590,94 @@ public abstract class 
GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
 
         startGrids();
 
-        Collection<Thread> threads = new LinkedList<>();
+        GridCompoundFuture<Long, Long> fut = new GridCompoundFuture<>();
 
         try {
             final AtomicInteger putCntr = new AtomicInteger();
-
             final CyclicBarrier barrier = new CyclicBarrier(putThreads + 
restartThreads);
+            final AtomicInteger startedThreadCntr = new AtomicInteger();
 
-            for (int i = 0; i < putThreads; i++) {
-                final int gridIdx = i;
-
-                Thread t = new Thread(new Runnable() {
-                    @Override public void run() {
-                        try {
-                            barrier.await();
+            fut.add(GridTestUtils.runMultiThreadedAsync(() -> {
+                try {
+                    int gridIdx = startedThreadCntr.getAndIncrement();
 
-                            info("Starting put thread: " + gridIdx);
+                    barrier.await();
 
-                            Thread.currentThread().setName("put-worker-" + 
grid(gridIdx).name());
+                    info("Starting put thread: " + gridIdx);
 
-                            IgniteCache<Integer, String> cache = 
grid(gridIdx).cache(CACHE_NAME);
+                    Thread.currentThread().setName("put-worker-" + 
grid(gridIdx).name());
 
-                            while (System.currentTimeMillis() < endTime && 
err.get() == null) {
-                                int key = RAND.nextInt(keyCnt);
+                    IgniteCache<Integer, String> cache = 
grid(gridIdx).cache(CACHE_NAME);
 
-                                try {
-                                    cache.put(key, Integer.toString(key));
-                                }
-                                catch (IgniteException | CacheException 
ignored) {
-                                    // It is ok if primary node leaves grid.
-                                }
+                    while (System.currentTimeMillis() < endTime && err.get() 
== null) {
+                        int key = RAND.nextInt(keyCnt);
 
-                                cache.get(key);
+                        try {
+                            cache.put(key, Integer.toString(key));
+                        }
+                        catch (IgniteException | CacheException ignored) {
+                            // It is ok if primary node leaves grid.
+                        }
 
-                                int c = putCntr.incrementAndGet();
+                        cache.get(key);
 
-                                if (c % LOG_FREQ == 0)
-                                    info(">>> Put iteration [cnt=" + c + ", 
key=" + key + ']');
-                            }
-                        }
-                        catch (Exception e) {
-                            err.compareAndSet(null, e);
+                        int c = putCntr.incrementAndGet();
 
-                            error("Unexpected exception in put-worker.", e);
-                        }
+                        if (c % LOG_FREQ == 0)
+                            info(">>> Put iteration [cnt=" + c + ", key=" + 
key + ']');
                     }
-                }, "put-worker-" + i);
-
-                t.start();
+                }
+                catch (Exception e) {
+                    err.compareAndSet(null, e);
 
-                threads.add(t);
-            }
+                    error("Unexpected exception in put-worker.", e);
+                }
+            }, putThreads, "put-worker"));
 
-            for (int i = 0; i < restartThreads; i++) {
-                final int gridIdx = i + putThreads;
+            GridTestUtils.waitForCondition(() -> startedThreadCntr.get() == 
putThreads, getTestTimeout());
 
-                Thread t = new Thread(new Runnable() {
-                    @Override public void run() {
-                        try {
-                            barrier.await();
+            fut.add(GridTestUtils.runMultiThreadedAsync(() -> {
+                try {
+                    int gridIdx = startedThreadCntr.getAndIncrement();
 
-                            info("Starting restart thread: " + gridIdx);
+                    barrier.await();
 
-                            int cnt = 0;
+                    info("Starting restart thread: " + gridIdx);
 
-                            while (System.currentTimeMillis() < endTime && 
err.get() == null) {
-                                log.info(">>>>>>> Stopping grid " + gridIdx);
+                    int cnt = 0;
 
-                                stopGrid(gridIdx);
+                    while (System.currentTimeMillis() < endTime && err.get() 
== null) {
+                        log.info(">>>>>>> Stopping grid " + gridIdx);
 
-                                log.info(">>>>>>> Starting grid " + gridIdx);
+                        stopGrid(gridIdx);
 
-                                startGrid(gridIdx);
+                        log.info(">>>>>>> Starting grid " + gridIdx);
 
-                                int c = ++cnt;
+                        startGrid(gridIdx);
 
-                                if (c % LOG_FREQ == 0)
-                                    info(">>> Restart iteration: " + c);
-                            }
-                        }
-                        catch (Exception e) {
-                            err.compareAndSet(null, e);
+                        int c = ++cnt;
 
-                            error("Unexpected exception in restart-worker.", 
e);
-                        }
+                        if (c % LOG_FREQ == 0)
+                            info(">>> Restart iteration: " + c);
                     }
-                }, "restart-worker-" + i);
-
-                t.start();
+                }
+                catch (Exception e) {
+                    err.compareAndSet(null, e);
 
-                threads.add(t);
-            }
+                    error("Unexpected exception in restart-worker.", e);
+                }
+            }, restartThreads, "restart-worker"));
 
-            for (Thread t : threads)
-                t.join(2 * duration);
+            fut.markInitialized();
 
-            for (Thread t : threads) {
-                if (t.isAlive())
-                    t.interrupt();
-            }
+            fut.get(getTestTimeout());
 
             if (err.get() != null)
                 throw err.get();
         }
         finally {
-            stopAllGrids();
+            if (!fut.isDone())
+                fut.cancel();
         }
     }
 
@@ -713,147 +696,134 @@ public abstract class 
GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
             return;
 
         final long endTime = System.currentTimeMillis() + duration;
-
         final AtomicReference<Throwable> err = new AtomicReference<>();
 
         startGrids();
 
-        Collection<Thread> threads = new LinkedList<>();
+        GridCompoundFuture<Long, Long> fut = new GridCompoundFuture<>();
 
         try {
             final AtomicInteger txCntr = new AtomicInteger();
-
             final CyclicBarrier barrier = new CyclicBarrier(putThreads + 
restartThreads);
+            final AtomicInteger startedThreadCntr = new AtomicInteger();
 
-            for (int i = 0; i < putThreads; i++) {
-                final int gridIdx = i;
-
-                Thread t = new Thread(new Runnable() {
-                    @Override public void run() {
-                        try {
-                            barrier.await();
+            fut.add(GridTestUtils.runMultiThreadedAsync(() -> {
+                try {
+                    int gridIdx = startedThreadCntr.getAndIncrement();
 
-                            info("Starting put thread: " + gridIdx);
+                    barrier.await();
 
-                            Ignite ignite = grid(gridIdx);
+                    info("Starting put thread: " + gridIdx);
 
-                            Thread.currentThread().setName("put-worker-" + 
ignite.name());
+                    Ignite ignite = grid(gridIdx);
 
-                            UUID locNodeId = ignite.cluster().localNode().id();
+                    Thread.currentThread().setName("put-worker-" + 
ignite.name());
 
-                            IgniteCache<Integer, String> cache = 
ignite.cache(CACHE_NAME).withAllowAtomicOpsInTx();
+                    UUID locNodeId = ignite.cluster().localNode().id();
 
-                            List<Integer> keys = new ArrayList<>(txKeys);
+                    IgniteCache<Integer, String> cache = 
ignite.cache(CACHE_NAME).withAllowAtomicOpsInTx();
 
-                            while (System.currentTimeMillis() < endTime && 
err.get() == null) {
-                                keys.clear();
+                    List<Integer> keys = new ArrayList<>(txKeys);
 
-                                for (int i = 0; i < txKeys; i++)
-                                    keys.add(RAND.nextInt(keyCnt));
+                    while (System.currentTimeMillis() < endTime && err.get() 
== null) {
+                        keys.clear();
 
-                                // Ensure lock order.
-                                Collections.sort(keys);
+                        for (int i = 0; i < txKeys; i++)
+                            keys.add(RAND.nextInt(keyCnt));
 
-                                int c = 0;
+                        // Ensure lock order.
+                        Collections.sort(keys);
 
-                                try {
-                                    IgniteTransactions txs = 
ignite.transactions();
+                        int c = 0;
 
-                                    try (Transaction tx = 
txs.txStart(txConcurrency(), REPEATABLE_READ)) {
-                                        c = txCntr.incrementAndGet();
-
-                                        if (c % LOG_FREQ == 0) {
-                                            info(">>> Tx iteration started 
[cnt=" + c +
-                                                ", keys=" + keys +
-                                                ", locNodeId=" + locNodeId + 
']');
-                                        }
-
-                                        for (int key : keys) {
-                                            int op = cacheOp();
+                        try {
+                            IgniteTransactions txs = ignite.transactions();
 
-                                            if (op == 1)
-                                                cache.put(key, 
Integer.toString(key));
-                                            else if (op == 2)
-                                                cache.remove(key);
-                                            else
-                                                cache.get(key);
-                                        }
-
-                                        tx.commit();
-                                    }
-                                }
-                                catch (IgniteException | CacheException 
ignored) {
-                                    // It is ok if primary node leaves grid.
-                                }
+                            try (Transaction tx = txs.txStart(txConcurrency(), 
REPEATABLE_READ)) {
+                                c = txCntr.incrementAndGet();
 
                                 if (c % LOG_FREQ == 0) {
-                                    info(">>> Tx iteration finished [cnt=" + c 
+
-                                        ", cacheSize=" + cache.localSize() +
+                                    info(">>> Tx iteration started [cnt=" + c +
                                         ", keys=" + keys +
                                         ", locNodeId=" + locNodeId + ']');
                                 }
-                            }
 
-                            info(">>> " + Thread.currentThread().getName() + " 
finished.");
+                                for (int key : keys) {
+                                    int op = cacheOp();
+
+                                    if (op == 1)
+                                        cache.put(key, Integer.toString(key));
+                                    else if (op == 2)
+                                        cache.remove(key);
+                                    else
+                                        cache.get(key);
+                                }
+
+                                tx.commit();
+                            }
+                        }
+                        catch (IgniteException | CacheException ignored) {
+                            // It is ok if primary node leaves grid.
                         }
-                        catch (Exception e) {
-                            err.compareAndSet(null, e);
 
-                            error("Unexpected exception in put-worker.", e);
+                        if (c % LOG_FREQ == 0) {
+                            info(">>> Tx iteration finished [cnt=" + c +
+                                ", cacheSize=" + cache.localSize() +
+                                ", keys=" + keys +
+                                ", locNodeId=" + locNodeId + ']');
                         }
                     }
-                }, "put-worker-" + i);
 
-                t.start();
+                    info(">>> " + Thread.currentThread().getName() + " 
finished.");
+                }
+                catch (Exception e) {
+                    err.compareAndSet(null, e);
 
-                threads.add(t);
-            }
+                    error("Unexpected exception in put-worker.", e);
+                }
+            }, putThreads, "put-worker"));
 
-            for (int i = 0; i < restartThreads; i++) {
-                final int gridIdx = i + putThreads;
+            GridTestUtils.waitForCondition(() -> startedThreadCntr.get() == 
putThreads, getTestTimeout());
 
-                Thread t = new Thread(new Runnable() {
-                    @Override public void run() {
-                        try {
-                            barrier.await();
+            fut.add(GridTestUtils.runMultiThreadedAsync(() -> {
+                try {
+                    int gridIdx = startedThreadCntr.incrementAndGet();
 
-                            info("Starting restart thread: " + gridIdx);
+                    barrier.await();
 
-                            int cnt = 0;
+                    info("Starting restart thread: " + gridIdx);
 
-                            while (System.currentTimeMillis() < endTime && 
err.get() == null) {
-                                stopGrid(getTestIgniteInstanceName(gridIdx), 
false, false);
-                                startGrid(gridIdx);
+                    int cnt = 0;
 
-                                int c = ++cnt;
+                    while (System.currentTimeMillis() < endTime && err.get() 
== null) {
+                        stopGrid(getTestIgniteInstanceName(gridIdx), false, 
false);
+                        startGrid(gridIdx);
 
-                                if (c % LOG_FREQ == 0)
-                                    info(">>> Restart iteration: " + c);
-                            }
+                        int c = ++cnt;
 
-                            info(">>> " + Thread.currentThread().getName() + " 
finished.");
-                        }
-                        catch (Exception e) {
-                            err.compareAndSet(null, e);
-
-                            error("Unexpected exception in restart-worker.", 
e);
-                        }
+                        if (c % LOG_FREQ == 0)
+                            info(">>> Restart iteration: " + c);
                     }
-                }, "restart-worker-" + i);
 
-                t.start();
+                    info(">>> " + Thread.currentThread().getName() + " 
finished.");
+                }
+                catch (Exception e) {
+                    err.compareAndSet(null, e);
 
-                threads.add(t);
-            }
+                    error("Unexpected exception in restart-worker.", e);
+                }
+            }, restartThreads, "restart-worker"));
 
-            for (Thread t : threads)
-                t.join();
+            fut.markInitialized();
+
+            fut.get(getTestTimeout());
 
             if (err.get() != null)
                 throw err.get();
         }
         finally {
-            stopAllGrids();
+            if (!fut.isDone())
+                fut.cancel();
         }
     }
 
@@ -873,129 +843,116 @@ public abstract class 
GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
 
         startGrids();
 
-        Collection<Thread> threads = new LinkedList<>();
+        GridCompoundFuture<Long, Long> fut = new GridCompoundFuture<>();
 
         try {
             final AtomicInteger txCntr = new AtomicInteger();
-
             final CyclicBarrier barrier = new CyclicBarrier(putThreads + 
restartThreads);
-
+            final AtomicInteger startedThreadCntr = new AtomicInteger();
             final int txKeys = 3;
 
-            for (int i = 0; i < putThreads; i++) {
-                final int gridIdx = i;
+            fut.add(GridTestUtils.runMultiThreadedAsync(() -> {
+                try {
+                    int gridIdx = startedThreadCntr.getAndIncrement();
 
-                Thread t = new Thread(new Runnable() {
-                    @Override public void run() {
-                        try {
-                            barrier.await();
-
-                            info("Starting put thread: " + gridIdx);
+                    barrier.await();
 
-                            Ignite ignite = grid(gridIdx);
+                    info("Starting put thread: " + gridIdx);
 
-                            Thread.currentThread().setName("put-worker-" + 
ignite.name());
+                    Ignite ignite = grid(gridIdx);
 
-                            UUID locNodeId = ignite.cluster().localNode().id();
+                    Thread.currentThread().setName("put-worker-" + 
ignite.name());
 
-                            IgniteCache<Integer, String> cache = 
ignite.cache(CACHE_NAME).withAllowAtomicOpsInTx();
+                    UUID locNodeId = ignite.cluster().localNode().id();
 
-                            List<Integer> keys = new ArrayList<>(txKeys);
+                    IgniteCache<Integer, String> cache = 
ignite.cache(CACHE_NAME).withAllowAtomicOpsInTx();
 
-                            while (System.currentTimeMillis() < endTime && 
err.get() == null) {
-                                keys.clear();
+                    List<Integer> keys = new ArrayList<>(txKeys);
 
-                                for (int i = 0; i < txKeys; i++)
-                                    keys.add(RAND.nextInt(keyCnt));
+                    while (System.currentTimeMillis() < endTime && err.get() 
== null) {
+                        keys.clear();
 
-                                // Ensure lock order.
-                                Collections.sort(keys);
+                        for (int i = 0; i < txKeys; i++)
+                            keys.add(RAND.nextInt(keyCnt));
 
-                                int c = 0;
+                        // Ensure lock order.
+                        Collections.sort(keys);
 
-                                try (Transaction tx = 
ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
-                                    c = txCntr.incrementAndGet();
+                        int c = 0;
 
-                                    if (c % LOG_FREQ == 0)
-                                        info(">>> Tx iteration started [cnt=" 
+ c + ", keys=" + keys + ", " +
-                                            "locNodeId=" + locNodeId + ']');
+                        try (Transaction tx = 
ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                            c = txCntr.incrementAndGet();
 
-                                    Map<Integer, String> batch = new 
LinkedHashMap<>();
+                            if (c % LOG_FREQ == 0)
+                                info(">>> Tx iteration started [cnt=" + c + ", 
keys=" + keys + ", " +
+                                    "locNodeId=" + locNodeId + ']');
 
-                                    for (int key : keys)
-                                        batch.put(key, String.valueOf(key));
+                            Map<Integer, String> batch = new LinkedHashMap<>();
 
-                                    cache.putAll(batch);
+                            for (int key : keys)
+                                batch.put(key, String.valueOf(key));
 
-                                    tx.commit();
-                                }
-                                catch (IgniteException | CacheException 
ignored) {
-                                    // It is ok if primary node leaves grid.
-                                }
+                            cache.putAll(batch);
 
-                                if (c % LOG_FREQ == 0) {
-                                    info(">>> Tx iteration finished [cnt=" + c 
+
-                                        ", keys=" + keys + ", " +
-                                        "locNodeId=" + locNodeId + ']');
-                                }
-                            }
+                            tx.commit();
+                        }
+                        catch (IgniteException | CacheException ignored) {
+                            // It is ok if primary node leaves grid.
                         }
-                        catch (Exception e) {
-                            err.compareAndSet(null, e);
 
-                            error("Unexpected exception in put-worker.", e);
+                        if (c % LOG_FREQ == 0) {
+                            info(">>> Tx iteration finished [cnt=" + c +
+                                ", keys=" + keys + ", " +
+                                "locNodeId=" + locNodeId + ']');
                         }
                     }
-                }, "put-worker-" + i);
+                }
+                catch (Exception e) {
+                    err.compareAndSet(null, e);
 
-                t.start();
+                    error("Unexpected exception in put-worker.", e);
+                }
+            }, putThreads, "put-worker"));
 
-                threads.add(t);
-            }
+            GridTestUtils.waitForCondition(() -> startedThreadCntr.get() == 
putThreads, getTestTimeout());
 
-            for (int i = 0; i < restartThreads; i++) {
-                final int gridIdx = i + putThreads;
+            fut.add(GridTestUtils.runMultiThreadedAsync(() -> {
+                try {
+                    int gridIdx = startedThreadCntr.getAndIncrement();
 
-                Thread t = new Thread(new Runnable() {
-                    @Override public void run() {
-                        try {
-                            barrier.await();
+                    barrier.await();
 
-                            info("Starting restart thread: " + gridIdx);
+                    info("Starting restart thread: " + gridIdx);
 
-                            int cnt = 0;
+                    int cnt = 0;
 
-                            while (System.currentTimeMillis() < endTime && 
err.get() == null) {
-                                stopGrid(gridIdx);
-                                startGrid(gridIdx);
+                    while (System.currentTimeMillis() < endTime && err.get() 
== null) {
+                        stopGrid(gridIdx);
+                        startGrid(gridIdx);
 
-                                int c = ++cnt;
+                        int c = ++cnt;
 
-                                if (c % LOG_FREQ == 0)
-                                    info(">>> Restart iteration: " + c);
-                            }
-                        }
-                        catch (Exception e) {
-                            err.compareAndSet(null, e);
-
-                            error("Unexpected exception in restart-worker.", 
e);
-                        }
+                        if (c % LOG_FREQ == 0)
+                            info(">>> Restart iteration: " + c);
                     }
-                }, "restart-worker-" + i);
+                }
+                catch (Exception e) {
+                    err.compareAndSet(null, e);
 
-                t.start();
+                    error("Unexpected exception in restart-worker.", e);
+                }
+            }, restartThreads, "restart-worker"));
 
-                threads.add(t);
-            }
+            fut.markInitialized();
 
-            for (Thread t : threads)
-                t.join();
+            fut.get(getTestTimeout());
 
             if (err.get() != null)
                 throw err.get();
         }
         finally {
-            stopAllGrids();
+            if (!fut.isDone())
+                fut.cancel();
         }
     }
 

Reply via email to