You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2018/04/10 08:59:29 UTC
[01/12] ignite git commit: IGNITE-7222 Added ZooKeeper discovery SPI
Repository: ignite
Updated Branches:
refs/heads/master a0a187bf0 -> a64b941df
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/test/java/org/apache/zookeeper/ZkTestClientCnxnSocketNIO.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/test/java/org/apache/zookeeper/ZkTestClientCnxnSocketNIO.java b/modules/zookeeper/src/test/java/org/apache/zookeeper/ZkTestClientCnxnSocketNIO.java
new file mode 100644
index 0000000..7892b5e
--- /dev/null
+++ b/modules/zookeeper/src/test/java/org/apache/zookeeper/ZkTestClientCnxnSocketNIO.java
@@ -0,0 +1,137 @@
+/*
+ * 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.zookeeper;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.channels.SelectionKey;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.logger.java.JavaLogger;
+import org.apache.ignite.testframework.GridTestUtils;
+
+/**
+ *
+ */
+public class ZkTestClientCnxnSocketNIO extends ClientCnxnSocketNIO {
+ /** */
+ public static final IgniteLogger log = new JavaLogger().getLogger(ZkTestClientCnxnSocketNIO.class);
+
+ /** */
+ public static volatile boolean DEBUG = false;
+
+ /** */
+ public volatile CountDownLatch blockConnectLatch;
+
+ /** */
+ public static ConcurrentHashMap<String, ZkTestClientCnxnSocketNIO> clients = new ConcurrentHashMap<>();
+
+ /** */
+ private final String nodeName;
+
+ /**
+ *
+ */
+ public static void reset() {
+ clients.clear();
+ }
+
+ /**
+ * @param node Node.
+ * @return ZK client.
+ */
+ public static ZkTestClientCnxnSocketNIO forNode(Ignite node) {
+ return clients.get(node.name());
+ }
+
+ /**
+ * @param instanceName Ignite instance name.
+ * @return ZK client.
+ */
+ public static ZkTestClientCnxnSocketNIO forNode(String instanceName) {
+ return clients.get(instanceName);
+ }
+
+ /**
+ * @throws IOException If failed.
+ */
+ public ZkTestClientCnxnSocketNIO() throws IOException {
+ super();
+
+ String threadName = Thread.currentThread().getName();
+
+ nodeName = threadName.substring(threadName.indexOf('-') + 1);
+
+ if (DEBUG)
+ log.info("ZkTestClientCnxnSocketNIO created for node: " + nodeName);
+ }
+
+ /** {@inheritDoc} */
+ @Override void connect(InetSocketAddress addr) throws IOException {
+ CountDownLatch blockConnect = this.blockConnectLatch;
+
+ if (DEBUG)
+ log.info("ZkTestClientCnxnSocketNIO connect [node=" + nodeName + ", addr=" + addr + ']');
+
+ if (blockConnect != null && blockConnect.getCount() > 0) {
+ try {
+ log.info("ZkTestClientCnxnSocketNIO block connect");
+
+ blockConnect.await(60, TimeUnit.SECONDS);
+
+ log.info("ZkTestClientCnxnSocketNIO finish block connect");
+ }
+ catch (Exception e) {
+ log.error("Error in ZkTestClientCnxnSocketNIO: " + e, e);
+ }
+ }
+
+ super.connect(addr);
+
+ clients.put(nodeName, this);
+ }
+
+ /**
+ *
+ */
+ public void allowConnect() {
+ assert blockConnectLatch != null && blockConnectLatch.getCount() == 1 : blockConnectLatch;
+
+ log.info("ZkTestClientCnxnSocketNIO allowConnect [node=" + nodeName + ']');
+
+ blockConnectLatch.countDown();
+ }
+
+ /**
+ * @param blockConnect {@code True} to block client reconnect.
+ * @throws Exception If failed.
+ */
+ public void closeSocket(boolean blockConnect) throws Exception {
+ if (blockConnect)
+ blockConnectLatch = new CountDownLatch(1);
+
+ log.info("ZkTestClientCnxnSocketNIO closeSocket [node=" + nodeName + ", block=" + blockConnect + ']');
+
+ SelectionKey k = GridTestUtils.getFieldValue(this, ClientCnxnSocketNIO.class, "sockKey");
+
+ k.channel().close();
+ }
+}
[12/12] ignite git commit: IGNITE-7222 Added ZooKeeper discovery SPI
Posted by ag...@apache.org.
IGNITE-7222 Added ZooKeeper discovery SPI
Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/a64b941d
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/a64b941d
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/a64b941d
Branch: refs/heads/master
Commit: a64b941df0a4d3bfc3a2dab32f85c371c1a509be
Parents: a0a187b
Author: Semyon Boikov <sb...@apache.org>
Authored: Tue Apr 10 11:37:39 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Apr 10 11:57:40 2018 +0300
----------------------------------------------------------------------
.../jdbc2/JdbcAbstractDmlStatementSelfTest.java | 6 +-
.../CommunicationFailureContext.java | 62 +
.../CommunicationFailureResolver.java | 28 +
.../DefaultCommunicationFailureResolver.java | 305 ++
.../configuration/IgniteConfiguration.java | 22 +
.../org/apache/ignite/internal/GridTopic.java | 3 +
.../apache/ignite/internal/IgniteKernal.java | 5 +-
.../org/apache/ignite/internal/IgnitionEx.java | 3 +
.../internal/managers/GridManagerAdapter.java | 8 +
.../managers/communication/GridIoManager.java | 4 +-
.../communication/GridIoMessageFactory.java | 12 +
.../discovery/CustomMessageWrapper.java | 5 +
.../internal/managers/discovery/DiscoCache.java | 8 +
.../discovery/DiscoveryCustomMessage.java | 10 +-
.../DiscoveryMessageResultsCollector.java | 222 +
.../discovery/GridDiscoveryManager.java | 128 +-
.../managers/discovery/IgniteClusterNode.java | 69 +
.../managers/discovery/IgniteDiscoverySpi.java | 67 +
.../IgniteDiscoverySpiInternalListener.java | 42 +
.../authentication/UserAcceptedMessage.java | 5 +
.../authentication/UserProposedMessage.java | 5 +
.../cache/CacheAffinityChangeMessage.java | 5 +
.../cache/CacheAffinitySharedManager.java | 26 +-
.../cache/CacheStatisticsModeChangeMessage.java | 5 +
.../ClientCacheChangeDiscoveryMessage.java | 5 +
.../ClientCacheChangeDummyDiscoveryMessage.java | 5 +
.../cache/DynamicCacheChangeBatch.java | 5 +
.../processors/cache/GridCacheAdapter.java | 3 +-
.../GridCachePartitionExchangeManager.java | 16 +-
.../processors/cache/GridCacheProcessor.java | 4 +-
.../processors/cache/GridCacheUtils.java | 6 +-
.../processors/cache/WalStateFinishMessage.java | 5 +
.../cache/WalStateProposeMessage.java | 5 +
.../cache/binary/BinaryMetadataTransport.java | 24 +-
.../binary/MetadataUpdateAcceptedMessage.java | 5 +
.../binary/MetadataUpdateProposedMessage.java | 5 +
.../dht/GridClientPartitionTopology.java | 39 +
.../distributed/dht/GridDhtCacheAdapter.java | 4 +
.../dht/GridDhtPartitionTopology.java | 6 +
.../dht/GridDhtPartitionTopologyImpl.java | 39 +
.../GridDhtPartitionsExchangeFuture.java | 26 +-
.../cluster/ChangeGlobalStateFinishMessage.java | 5 +
.../cluster/ChangeGlobalStateMessage.java | 5 +
.../cluster/ClusterMetricsUpdateMessage.java | 158 +
.../processors/cluster/ClusterNodeMetrics.java | 62 +
.../processors/cluster/ClusterProcessor.java | 249 +-
.../continuous/AbstractContinuousMessage.java | 5 +
.../continuous/ContinuousRoutineInfo.java | 100 +
.../ContinuousRoutineStartResultMessage.java | 206 +
.../ContinuousRoutinesCommonDiscoveryData.java | 45 +
.../continuous/ContinuousRoutinesInfo.java | 132 +
...tinuousRoutinesJoiningNodeDiscoveryData.java | 45 +
.../continuous/GridContinuousProcessor.java | 862 +++-
.../continuous/StartRequestDataV2.java | 164 +
.../StartRoutineDiscoveryMessageV2.java | 77 +
.../StopRoutineAckDiscoveryMessage.java | 5 +
.../datastreamer/DataStreamerImpl.java | 27 +-
.../marshaller/MappingAcceptedMessage.java | 5 +
.../marshaller/MappingProposedMessage.java | 5 +
.../message/SchemaFinishDiscoveryMessage.java | 5 +
.../message/SchemaProposeDiscoveryMessage.java | 5 +
.../ignite/internal/util/nio/GridNioServer.java | 18 +-
.../org/apache/ignite/spi/IgniteSpiAdapter.java | 10 +
.../org/apache/ignite/spi/IgniteSpiContext.java | 11 +
.../communication/tcp/TcpCommunicationSpi.java | 190 +-
.../tcp/internal/ConnectionKey.java | 117 +
.../TcpCommunicationConnectionCheckFuture.java | 519 ++
...pCommunicationNodeConnectionCheckFuture.java | 30 +
.../discovery/DiscoverySpiCustomMessage.java | 15 +-
...DiscoverySpiMutableCustomMessageSupport.java | 40 +
.../spi/discovery/tcp/TcpDiscoverySpi.java | 55 +-
.../tcp/internal/TcpDiscoveryNode.java | 32 +-
.../resources/META-INF/classnames.properties | 2 +
...unctionExcludeNeighborsAbstractSelfTest.java | 8 +-
.../failure/FailureHandlerTriggeredTest.java | 4 +
.../internal/ClusterGroupHostsSelfTest.java | 3 +
.../ignite/internal/ClusterGroupSelfTest.java | 2 +
.../internal/ClusterNodeMetricsUpdateTest.java | 173 +
.../internal/DiscoverySpiTestListener.java | 162 +
.../ignite/internal/GridDiscoverySelfTest.java | 14 +-
.../GridJobMasterLeaveAwareSelfTest.java | 2 +
.../internal/GridJobStealingSelfTest.java | 2 +
.../internal/GridSameVmStartupSelfTest.java | 19 +-
.../apache/ignite/internal/GridSelfTest.java | 2 +
.../IgniteClientReconnectAbstractTest.java | 53 +-
.../IgniteClientReconnectApiExceptionTest.java | 21 +-
.../IgniteClientReconnectAtomicsTest.java | 30 +-
.../IgniteClientReconnectCacheTest.java | 49 +-
.../IgniteClientReconnectCollectionsTest.java | 14 +-
.../IgniteClientReconnectComputeTest.java | 6 +-
...eClientReconnectContinuousProcessorTest.java | 13 +-
...IgniteClientReconnectDiscoveryStateTest.java | 22 +-
...niteClientReconnectFailoverAbstractTest.java | 12 +-
.../IgniteClientReconnectServicesTest.java | 8 +-
.../internal/IgniteClientReconnectStopTest.java | 12 +-
.../IgniteClientReconnectStreamerTest.java | 4 +-
.../ignite/internal/IgniteClientRejoinTest.java | 3 +
.../GridDiscoveryManagerAliveCacheSelfTest.java | 16 +-
.../GridAffinityProcessorAbstractSelfTest.java | 4 +-
.../CacheMetricsForClusterGroupSelfTest.java | 12 +-
.../cache/GridCacheAbstractSelfTest.java | 2 +
.../cache/IgniteCacheNearLockValueSelfTest.java | 4 +-
.../IgniteCacheP2pUnmarshallingErrorTest.java | 11 +
.../IgniteClusterActivateDeactivateTest.java | 65 +
.../IgniteDaemonNodeMarshallerCacheTest.java | 3 +-
.../binary/BinaryMetadataUpdatesFlowTest.java | 12 +-
...ntNodeBinaryObjectMetadataMultinodeTest.java | 2 +-
.../GridCacheQueueClientDisconnectTest.java | 10 +
.../IgniteClientDataStructuresAbstractTest.java | 3 +-
.../CacheLateAffinityAssignmentTest.java | 127 +-
.../GridCacheNodeFailureAbstractTest.java | 5 +-
.../distributed/IgniteCache150ClientsTest.java | 2 +
.../distributed/IgniteCacheManyClientsTest.java | 44 +-
.../IgniteOptimisticTxSuspendResumeTest.java | 2 +
...ridCacheDhtPreloadMultiThreadedSelfTest.java | 4 +
.../dht/GridCacheDhtPreloadSelfTest.java | 2 +
.../dht/TxRecoveryStoreEnabledTest.java | 15 +-
...titionedExplicitLockNodeFailureSelfTest.java | 3 +-
.../ClientReconnectContinuousQueryTest.java | 19 +-
...yRemoteFilterMissingInClassPathSelfTest.java | 23 +-
...CacheContinuousQueryClientReconnectTest.java | 3 +
.../CacheVersionedEntryAbstractTest.java | 33 +-
.../continuous/GridEventConsumeSelfTest.java | 34 +-
.../service/ClosureServiceClientsNodesTest.java | 19 +-
.../internal/util/GridTestClockTimer.java | 9 +
.../GridMarshallerMappingConsistencyTest.java | 4 +
.../ignite/messaging/GridMessagingSelfTest.java | 126 +-
.../GridTcpCommunicationSpiAbstractTest.java | 71 +
.../FilterDataForClientNodeDiscoveryTest.java | 5 +
.../testframework/GridSpiTestContext.java | 10 +
.../config/GridTestProperties.java | 9 +
.../testframework/junits/GridAbstractTest.java | 129 +-
.../junits/multijvm/IgniteNodeRunner.java | 2 +
.../testsuites/IgniteComputeGridTestSuite.java | 2 +
...niteCacheDistributedQueryCancelSelfTest.java | 2 +-
.../DynamicIndexAbstractBasicSelfTest.java | 5 +-
.../GridJtaTransactionManagerSelfTest.java | 21 +-
.../GridPartitionedCacheJtaFactorySelfTest.java | 19 +-
.../org/apache/ignite/spark/IgniteRDD.scala | 9 +-
.../ignite/internal/GridFactorySelfTest.java | 3 +-
.../p2p/GridP2PUserVersionChangeSelfTest.java | 5 +-
modules/yardstick/pom-standalone.xml | 6 +
modules/yardstick/pom.xml | 6 +
modules/zookeeper/pom.xml | 40 +
.../spi/discovery/zk/ZookeeperDiscoverySpi.java | 557 ++
.../zk/internal/ZkAbstractCallabck.java | 83 +
.../zk/internal/ZkAbstractChildrenCallback.java | 61 +
.../zk/internal/ZkAbstractWatcher.java | 55 +
.../discovery/zk/internal/ZkAliveNodeData.java | 40 +
.../zk/internal/ZkBulkJoinContext.java | 50 +
.../discovery/zk/internal/ZkClusterNodes.java | 103 +
.../internal/ZkCommunicationErrorNodeState.java | 46 +
.../ZkCommunicationErrorProcessFuture.java | 411 ++
...kCommunicationErrorResolveFinishMessage.java | 69 +
.../ZkCommunicationErrorResolveResult.java | 45 +
...ZkCommunicationErrorResolveStartMessage.java | 61 +
.../internal/ZkCommunicationFailureContext.java | 188 +
.../zk/internal/ZkDiscoveryCustomEventData.java | 89 +
.../zk/internal/ZkDiscoveryEventData.java | 165 +
.../zk/internal/ZkDiscoveryEventsData.java | 121 +
.../internal/ZkDiscoveryNodeFailEventData.java | 55 +
.../internal/ZkDiscoveryNodeJoinEventData.java | 60 +
.../ZkDistributedCollectDataFuture.java | 250 +
.../zk/internal/ZkForceNodeFailMessage.java | 65 +
.../discovery/zk/internal/ZkIgnitePaths.java | 307 ++
.../zk/internal/ZkInternalJoinErrorMessage.java | 44 +
.../zk/internal/ZkInternalMessage.java | 27 +
.../zk/internal/ZkJoinEventDataForJoined.java | 83 +
.../zk/internal/ZkJoinedNodeEvtData.java | 79 +
.../zk/internal/ZkJoiningNodeData.java | 87 +
.../zk/internal/ZkNoServersMessage.java | 50 +
.../zk/internal/ZkNodeValidateResult.java | 43 +
.../spi/discovery/zk/internal/ZkRunnable.java | 51 +
.../discovery/zk/internal/ZkRuntimeState.java | 135 +
.../discovery/zk/internal/ZkTimeoutObject.java | 54 +
.../discovery/zk/internal/ZookeeperClient.java | 1219 +++++
.../ZookeeperClientFailedException.java | 40 +
.../zk/internal/ZookeeperClusterNode.java | 362 ++
.../zk/internal/ZookeeperDiscoveryImpl.java | 4464 ++++++++++++++++
.../java/org/apache/ZookeeperNodeStart.java | 46 +
...CacheEntryListenerWithZkDiscoAtomicTest.java | 32 +
.../ZookeeperDiscoverySpiAbstractTestSuite.java | 118 +
.../zk/ZookeeperDiscoverySpiTestSuite1.java | 44 +
.../zk/ZookeeperDiscoverySpiTestSuite2.java | 94 +
...ZookeeperDiscoverySuitePreprocessorTest.java | 101 +
.../zk/internal/ZookeeperClientTest.java | 495 ++
...okeeperDiscoverySpiSaslAuthAbstractTest.java | 247 +
...ZookeeperDiscoverySpiSaslFailedAuthTest.java | 44 +
...eeperDiscoverySpiSaslSuccessfulAuthTest.java | 48 +
.../zk/internal/ZookeeperDiscoverySpiTest.java | 4847 ++++++++++++++++++
.../zookeeper/ZkTestClientCnxnSocketNIO.java | 137 +
191 files changed, 21158 insertions(+), 777 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java
index f4c0ca3..0a055a9 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java
@@ -138,8 +138,10 @@ public abstract class JdbcAbstractDmlStatementSelfTest extends GridCommonAbstrac
@Override protected void afterTest() throws Exception {
((IgniteEx)ignite(0)).context().cache().dynamicDestroyCache(DEFAULT_CACHE_NAME, true, true, false);
- conn.close();
- assertTrue(conn.isClosed());
+ if (conn != null) {
+ conn.close();
+ assertTrue(conn.isClosed());
+ }
cleanUpWorkingDir();
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/configuration/CommunicationFailureContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CommunicationFailureContext.java b/modules/core/src/main/java/org/apache/ignite/configuration/CommunicationFailureContext.java
new file mode 100644
index 0000000..a32d38c
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CommunicationFailureContext.java
@@ -0,0 +1,62 @@
+/*
+ * 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.configuration;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.spi.communication.CommunicationSpi;
+
+/**
+ * Communication Failure Context.
+ */
+public interface CommunicationFailureContext {
+ /**
+ * @return Current topology snapshot.
+ */
+ public List<ClusterNode> topologySnapshot();
+
+ /**
+ * @param node1 First node.
+ * @param node2 Second node.
+ * @return {@code True} if {@link CommunicationSpi} is able to establish connection from first node to second node.
+ */
+ public boolean connectionAvailable(ClusterNode node1, ClusterNode node2);
+
+ /**
+ * @return Currently started caches.
+ */
+ public Map<String, CacheConfiguration<?, ?>> startedCaches();
+
+ /**
+ * @param cacheName Cache name.
+ * @return Cache partitions affinity assignment.
+ */
+ public List<List<ClusterNode>> cacheAffinity(String cacheName);
+
+ /**
+ * @param cacheName Cache name.
+ * @return Cache partitions owners.
+ */
+ public List<List<ClusterNode>> cachePartitionOwners(String cacheName);
+
+ /**
+ * @param node Node to kill.
+ */
+ public void killNode(ClusterNode node);
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/configuration/CommunicationFailureResolver.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CommunicationFailureResolver.java b/modules/core/src/main/java/org/apache/ignite/configuration/CommunicationFailureResolver.java
new file mode 100644
index 0000000..a4d92f3
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CommunicationFailureResolver.java
@@ -0,0 +1,28 @@
+/*
+ * 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.configuration;
+
+/**
+ * Communication Failure Resolver.
+ */
+public interface CommunicationFailureResolver {
+ /**
+ * @param ctx Context.
+ */
+ public void resolve(CommunicationFailureContext ctx);
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/configuration/DefaultCommunicationFailureResolver.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/DefaultCommunicationFailureResolver.java b/modules/core/src/main/java/org/apache/ignite/configuration/DefaultCommunicationFailureResolver.java
new file mode 100644
index 0000000..a4c6da9
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/DefaultCommunicationFailureResolver.java
@@ -0,0 +1,305 @@
+/*
+ * 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.configuration;
+
+import java.util.BitSet;
+import java.util.List;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.resources.LoggerResource;
+
+/**
+ * Default Communication Failure Resolver.
+ */
+public class DefaultCommunicationFailureResolver implements CommunicationFailureResolver {
+ /** */
+ @LoggerResource
+ private IgniteLogger log;
+
+ /** {@inheritDoc} */
+ @Override public void resolve(CommunicationFailureContext ctx) {
+ ClusterGraph graph = new ClusterGraph(log, ctx);
+
+ ClusterSearch cluster = graph.findLargestIndependentCluster();
+
+ List<ClusterNode> nodes = ctx.topologySnapshot();
+
+ assert nodes.size() > 0;
+ assert cluster != null;
+
+ if (graph.checkFullyConnected(cluster.nodesBitSet)) {
+ assert cluster.nodeCnt <= nodes.size();
+
+ if (cluster.nodeCnt < nodes.size()) {
+ if (log.isInfoEnabled()) {
+ log.info("Communication problem resolver found fully connected independent cluster [" +
+ "clusterSrvCnt=" + cluster.srvCnt +
+ ", clusterTotalNodes=" + cluster.nodeCnt +
+ ", totalAliveNodes=" + nodes.size() + "]");
+ }
+
+ for (int i = 0; i < nodes.size(); i++) {
+ if (!cluster.nodesBitSet.get(i))
+ ctx.killNode(nodes.get(i));
+ }
+ }
+ else
+ U.warn(log, "All alive nodes are fully connected, this should be resolved automatically.");
+ }
+ else {
+ if (log.isInfoEnabled()) {
+ log.info("Communication problem resolver failed to find fully connected independent cluster.");
+ }
+ }
+ }
+
+ /**
+ * @param cluster Cluster nodes mask.
+ * @param nodes Nodes.
+ * @param limit IDs limit.
+ * @return Cluster node IDs string.
+ */
+ private static String clusterNodeIds(BitSet cluster, List<ClusterNode> nodes, int limit) {
+ int startIdx = 0;
+
+ StringBuilder builder = new StringBuilder();
+
+ int cnt = 0;
+
+ for (;;) {
+ int idx = cluster.nextSetBit(startIdx);
+
+ if (idx == -1)
+ break;
+
+ startIdx = idx + 1;
+
+ if (builder.length() == 0) {
+ builder.append('[');
+ }
+ else
+ builder.append(", ");
+
+ builder.append(nodes.get(idx).id());
+
+ if (cnt++ > limit)
+ builder.append(", ...");
+ }
+
+ builder.append(']');
+
+ return builder.toString();
+ }
+
+ /**
+ *
+ */
+ private static class ClusterSearch {
+ /** */
+ int srvCnt;
+
+ /** */
+ int nodeCnt;
+
+ /** */
+ final BitSet nodesBitSet;
+
+ /**
+ * @param nodes Total nodes.
+ */
+ ClusterSearch(int nodes) {
+ nodesBitSet = new BitSet(nodes);
+ }
+ }
+
+ /**
+ *
+ */
+ private static class ClusterGraph {
+ /** */
+ private final static int WORD_IDX_SHIFT = 6;
+
+ /** */
+ private final IgniteLogger log;
+
+ /** */
+ private final int nodeCnt;
+
+ /** */
+ private final long[] visitBitSet;
+
+ /** */
+ private final CommunicationFailureContext ctx;
+
+ /** */
+ private final List<ClusterNode> nodes;
+
+ /**
+ * @param log Logger.
+ * @param ctx Context.
+ */
+ ClusterGraph(IgniteLogger log, CommunicationFailureContext ctx) {
+ this.log = log;
+ this.ctx = ctx;
+
+ nodes = ctx.topologySnapshot();
+
+ nodeCnt = nodes.size();
+
+ assert nodeCnt > 0;
+
+ visitBitSet = initBitSet(nodeCnt);
+ }
+
+ /**
+ * @param bitIndex Bit index.
+ * @return Word index containing bit with given index.
+ */
+ private static int wordIndex(int bitIndex) {
+ return bitIndex >> WORD_IDX_SHIFT;
+ }
+
+ /**
+ * @param bitCnt Number of bits.
+ * @return Bit set words.
+ */
+ static long[] initBitSet(int bitCnt) {
+ return new long[wordIndex(bitCnt - 1) + 1];
+ }
+
+ /**
+ * @return Cluster nodes bit set.
+ */
+ ClusterSearch findLargestIndependentCluster() {
+ ClusterSearch maxCluster = null;
+
+ for (int i = 0; i < nodeCnt; i++) {
+ if (getBit(visitBitSet, i))
+ continue;
+
+ ClusterSearch cluster = new ClusterSearch(nodeCnt);
+
+ search(cluster, i);
+
+ if (log.isInfoEnabled()) {
+ log.info("Communication problem resolver found cluster [srvCnt=" + cluster.srvCnt +
+ ", totalNodeCnt=" + cluster.nodeCnt +
+ ", nodeIds=" + clusterNodeIds(cluster.nodesBitSet, nodes, 1000) + "]");
+ }
+
+ if (maxCluster == null || cluster.srvCnt > maxCluster.srvCnt)
+ maxCluster = cluster;
+ }
+
+ return maxCluster;
+ }
+
+ /**
+ * @param cluster Cluster nodes bit set.
+ * @return {@code True} if all cluster nodes are able to connect to each other.
+ */
+ boolean checkFullyConnected(BitSet cluster) {
+ int startIdx = 0;
+
+ int clusterNodes = cluster.cardinality();
+
+ for (;;) {
+ int idx = cluster.nextSetBit(startIdx);
+
+ if (idx == -1)
+ break;
+
+ ClusterNode node1 = nodes.get(idx);
+
+ for (int i = 0; i < clusterNodes; i++) {
+ if (!cluster.get(i) || i == idx)
+ continue;
+
+ ClusterNode node2 = nodes.get(i);
+
+ if (cluster.get(i) && !ctx.connectionAvailable(node1, node2))
+ return false;
+ }
+
+ startIdx = idx + 1;
+ }
+
+ return true;
+ }
+
+ /**
+ * @param cluster Current cluster bit set.
+ * @param idx Node index.
+ */
+ void search(ClusterSearch cluster, int idx) {
+ assert !getBit(visitBitSet, idx);
+
+ setBit(visitBitSet, idx);
+
+ cluster.nodesBitSet.set(idx);
+ cluster.nodeCnt++;
+
+ ClusterNode node1 = nodes.get(idx);
+
+ if (!CU.clientNode(node1))
+ cluster.srvCnt++;
+
+ for (int i = 0; i < nodeCnt; i++) {
+ if (i == idx || getBit(visitBitSet, i))
+ continue;
+
+ ClusterNode node2 = nodes.get(i);
+
+ boolean connected = ctx.connectionAvailable(node1, node2) ||
+ ctx.connectionAvailable(node2, node1);
+
+ if (connected)
+ search(cluster, i);
+ }
+ }
+
+ /**
+ * @param words Bit set words.
+ * @param bitIndex Bit index.
+ */
+ static void setBit(long words[], int bitIndex) {
+ int wordIndex = wordIndex(bitIndex);
+
+ words[wordIndex] |= (1L << bitIndex);
+ }
+
+ /**
+ * @param words Bit set words.
+ * @param bitIndex Bit index.
+ * @return Bit value.
+ */
+ static boolean getBit(long[] words, int bitIndex) {
+ int wordIndex = wordIndex(bitIndex);
+
+ return (words[wordIndex] & (1L << bitIndex)) != 0;
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return S.toString(DefaultCommunicationFailureResolver.class, this);
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
index add3880..cc3ea10 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
@@ -493,6 +493,9 @@ public class IgniteConfiguration {
/** Failure handler. */
private FailureHandler failureHnd;
+ /** Communication failure resolver */
+ private CommunicationFailureResolver commFailureRslvr;
+
/**
* Creates valid grid configuration with all default values.
*/
@@ -520,6 +523,8 @@ public class IgniteConfiguration {
loadBalancingSpi = cfg.getLoadBalancingSpi();
indexingSpi = cfg.getIndexingSpi();
+ commFailureRslvr = cfg.getCommunicationFailureResolver();
+
/*
* Order alphabetically for maintenance purposes.
*/
@@ -607,6 +612,23 @@ public class IgniteConfiguration {
}
/**
+ * @return Communication failure resovler.
+ */
+ public CommunicationFailureResolver getCommunicationFailureResolver() {
+ return commFailureRslvr;
+ }
+
+ /**
+ * @param commFailureRslvr Communication failure resovler.
+ * @return {@code this} instance.
+ */
+ public IgniteConfiguration setCommunicationFailureResolver(CommunicationFailureResolver commFailureRslvr) {
+ this.commFailureRslvr = commFailureRslvr;
+
+ return this;
+ }
+
+ /**
* Gets optional grid name. Returns {@code null} if non-default grid name was not
* provided.
* <p>The name only works locally and has no effect on topology</p>
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java b/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java
index 4932e67..1227e8c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java
@@ -121,6 +121,9 @@ public enum GridTopic {
TOPIC_WAL,
/** */
+ TOPIC_METRICS,
+
+ /** */
TOPIC_AUTH;
/** Enum values. */
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index 8bc46fd..0b102e5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -1298,7 +1298,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
ackStart(rtBean);
if (!isDaemon())
- ctx.discovery().ackTopology(localNode().order());
+ ctx.discovery().ackTopology(ctx.discovery().localJoin().joinTopologyVersion().topologyVersion());
}
/**
@@ -2623,6 +2623,9 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
objs.add(cfg.getGridLogger());
objs.add(cfg.getMBeanServer());
+ if (cfg.getCommunicationFailureResolver() != null)
+ objs.add(cfg.getCommunicationFailureResolver());
+
return objs;
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index 4708dd3..417ba1e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -75,6 +75,7 @@ import org.apache.ignite.failure.FailureContext;
import org.apache.ignite.failure.FailureType;
import org.apache.ignite.internal.binary.BinaryMarshaller;
import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager;
import org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor;
import org.apache.ignite.internal.processors.igfs.IgfsThreadFactory;
import org.apache.ignite.internal.processors.igfs.IgfsUtils;
@@ -2243,6 +2244,8 @@ public class IgnitionEx {
initializeDefaultSpi(myCfg);
+ GridDiscoveryManager.initCommunicationErrorResolveConfiguration(myCfg);
+
initializeDefaultCacheConfiguration(myCfg);
ExecutorConfiguration[] execCfgs = myCfg.getExecutorConfiguration();
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java
index 74f5a10..b0756cf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java
@@ -618,6 +618,14 @@ public abstract class GridManagerAdapter<T extends IgniteSpi> implements GridMan
return ctx.nodeAttributes();
}
+ @Override public boolean communicationFailureResolveSupported() {
+ return ctx.discovery().communicationErrorResolveSupported();
+ }
+
+ @Override public void resolveCommunicationFailure(ClusterNode node, Exception err) {
+ ctx.discovery().resolveCommunicationError(node, err);
+ }
+
/**
* @param e Exception to handle.
* @return GridSpiException Converted exception.
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
index d5cdd2d..8d9a700 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
@@ -298,9 +298,9 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
@Override public MessageReader reader(UUID rmtNodeId, MessageFactory msgFactory)
throws IgniteCheckedException {
- assert rmtNodeId != null;
- return new DirectMessageReader(msgFactory, U.directProtocolVersion(ctx, rmtNodeId));
+ return new DirectMessageReader(msgFactory,
+ rmtNodeId != null ? U.directProtocolVersion(ctx, rmtNodeId) : GridIoManager.DIRECT_PROTO_VER);
}
};
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
index a0fc2f8..5616fd0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
@@ -123,6 +123,8 @@ import org.apache.ignite.internal.processors.cache.transactions.TxLocksResponse;
import org.apache.ignite.internal.processors.cache.version.GridCacheRawVersionedEntry;
import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
import org.apache.ignite.internal.processors.cache.version.GridCacheVersionEx;
+import org.apache.ignite.internal.processors.cluster.ClusterMetricsUpdateMessage;
+import org.apache.ignite.internal.processors.continuous.ContinuousRoutineStartResultMessage;
import org.apache.ignite.internal.processors.continuous.GridContinuousMessage;
import org.apache.ignite.internal.processors.datastreamer.DataStreamerEntry;
import org.apache.ignite.internal.processors.datastreamer.DataStreamerRequest;
@@ -909,6 +911,16 @@ public class GridIoMessageFactory implements MessageFactory {
break;
+ case 133:
+ msg = new ClusterMetricsUpdateMessage();
+
+ break;
+
+ case 134:
+ msg = new ContinuousRoutineStartResultMessage();
+
+ break;
+
// [-3..119] [124..129] [-23..-27] [-36..-55]- this
// [120..123] - DR
// [-4..-22, -30..-35] - SQL
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/CustomMessageWrapper.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/CustomMessageWrapper.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/CustomMessageWrapper.java
index 4268886..4b6b7a2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/CustomMessageWrapper.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/CustomMessageWrapper.java
@@ -49,6 +49,11 @@ public class CustomMessageWrapper implements DiscoverySpiCustomMessage {
return delegate.isMutable();
}
+ /** {@inheritDoc} */
+ @Override public boolean stopProcess() {
+ return delegate.stopProcess();
+ }
+
/**
* @return Delegate.
*/
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoCache.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoCache.java
index c21698f..fef44fa 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoCache.java
@@ -312,6 +312,14 @@ public class DiscoCache {
}
/**
+ * @param nodeId Node ID.
+ * @return {@code True} if node is in alives list.
+ */
+ public boolean alive(UUID nodeId) {
+ return alives.contains(nodeId);
+ }
+
+ /**
* Gets all nodes that have cache with given name.
*
* @param cacheName Cache name.
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryCustomMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryCustomMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryCustomMessage.java
index c708c62..6ed2096 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryCustomMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryCustomMessage.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.managers.discovery;
import java.io.Serializable;
import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeAddFinishedMessage;
import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeAddedMessage;
import org.jetbrains.annotations.Nullable;
@@ -87,11 +88,18 @@ public interface DiscoveryCustomMessage extends Serializable {
@Nullable public DiscoveryCustomMessage ackMessage();
/**
- * @return {@code true} if message can be modified during listener notification. Changes will be send to next nodes.
+ * @return {@code True} if message can be modified during listener notification. Changes will be sent to next nodes.
*/
public boolean isMutable();
/**
+ * See {@link DiscoverySpiCustomMessage#stopProcess()}.
+ *
+ * @return {@code True} if message should not be sent to others nodes after it was processed on coordinator.
+ */
+ public boolean stopProcess();
+
+ /**
* Creates new discovery cache if message caused topology version change.
*
* @param mgr Discovery manager.
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryMessageResultsCollector.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryMessageResultsCollector.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryMessageResultsCollector.java
new file mode 100644
index 0000000..43be952
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryMessageResultsCollector.java
@@ -0,0 +1,222 @@
+/*
+ * 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.managers.discovery;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ *
+ */
+public abstract class DiscoveryMessageResultsCollector<M, R> {
+ /** */
+ private final Map<UUID, NodeMessage<M>> rcvd = new HashMap<>();
+
+ /** */
+ private int leftMsgs;
+
+ /** */
+ protected DiscoCache discoCache;
+
+ /** */
+ protected final GridKernalContext ctx;
+
+ /**
+ * @param ctx Context.
+ */
+ protected DiscoveryMessageResultsCollector(GridKernalContext ctx) {
+ this.ctx = ctx;
+ }
+
+ /**
+ * @param rcvd Received messages.
+ * @return Result.
+ */
+ protected abstract R createResult(Map<UUID, NodeMessage<M>> rcvd);
+
+ /**
+ * @param r Result.
+ */
+ protected abstract void onResultsCollected(R r);
+
+ /**
+ * @param discoCache Discovery state when discovery message was received.
+ * @param node Node.
+ * @return {@code True} if need wait for result from given node.
+ */
+ protected abstract boolean waitForNode(DiscoCache discoCache, ClusterNode node);
+
+ /**
+ * @param discoCache Discovery state.
+ */
+ public final void init(DiscoCache discoCache) {
+ assert discoCache != null;
+
+ R res = null;
+
+ synchronized (this) {
+ assert this.discoCache == null;
+ assert leftMsgs == 0 : leftMsgs;
+
+ this.discoCache = discoCache;
+
+ for (ClusterNode node : discoCache.allNodes()) {
+ if (ctx.discovery().alive(node) && waitForNode(discoCache, node) && !rcvd.containsKey(node.id())) {
+ rcvd.put(node.id(), new NodeMessage<>((M)null));
+
+ leftMsgs++;
+ }
+ }
+
+ if (leftMsgs == 0)
+ res = createResult(rcvd);
+ }
+
+ if (res != null)
+ onResultsCollected(res);
+ }
+
+ /**
+ * @param nodeId Node ID.
+ * @param msg Message.
+ */
+ public final void onMessage(UUID nodeId, M msg) {
+ R res = null;
+
+ synchronized (this) {
+ if (allReceived())
+ return;
+
+ NodeMessage<M> expMsg = rcvd.get(nodeId);
+
+ if (expMsg == null)
+ rcvd.put(nodeId, new NodeMessage<>(msg));
+ else if (expMsg.set(msg)) {
+ assert leftMsgs > 0;
+
+ leftMsgs--;
+
+ if (allReceived())
+ res = createResult(rcvd);
+ }
+ }
+
+ if (res != null)
+ onResultsCollected(res);
+ }
+
+ /**
+ * @param nodeId Failed node ID.
+ */
+ public final void onNodeFail(UUID nodeId) {
+ R res = null;
+
+ synchronized (this) {
+ if (allReceived())
+ return;
+
+ NodeMessage expMsg = rcvd.get(nodeId);
+
+ if (expMsg != null && expMsg.onNodeFailed()) {
+ assert leftMsgs > 0 : leftMsgs;
+
+ leftMsgs--;
+
+ if (allReceived())
+ res = createResult(rcvd);
+ }
+ }
+
+ if (res != null)
+ onResultsCollected(res);
+ }
+
+ /**
+ * @return {@code True} if expected messages are initialized and all message are received.
+ */
+ private boolean allReceived() {
+ return discoCache != null && leftMsgs == 0;
+ }
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return S.toString(DiscoveryMessageResultsCollector.class, this);
+ }
+
+ /**
+ *
+ */
+ protected static class NodeMessage<M> {
+ /** */
+ boolean nodeFailed;
+
+ /** */
+ M msg;
+
+ /**
+ * @param msg Message.
+ */
+ NodeMessage(M msg) {
+ this.msg = msg;
+ }
+
+ /**
+ * @return Message or {@code null} if node failed.
+ */
+ @Nullable public M message() {
+ return msg;
+ }
+
+ /**
+ * @return {@code True} if node result was not set before.
+ */
+ boolean onNodeFailed() {
+ if (nodeFailed || msg != null)
+ return false;
+
+ nodeFailed = true;
+
+ return true;
+ }
+
+ /**
+ * @param msg Received message.
+ * @return {@code True} if node result was not set before.
+ */
+ boolean set(M msg) {
+ assert msg != null;
+
+ if (this.msg != null)
+ return false;
+
+ this.msg = msg;
+
+ return !nodeFailed;
+ }
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return S.toString(NodeMessage.class, this);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index 2e814d4..4c5690e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -54,8 +54,11 @@ import org.apache.ignite.cluster.BaselineNode;
import org.apache.ignite.cluster.ClusterMetrics;
import org.apache.ignite.cluster.ClusterNode;
import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.CommunicationFailureResolver;
import org.apache.ignite.configuration.DataRegionConfiguration;
import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.DefaultCommunicationFailureResolver;
+import org.apache.ignite.configuration.IgniteConfiguration;
import org.apache.ignite.events.DiscoveryEvent;
import org.apache.ignite.events.Event;
import org.apache.ignite.failure.FailureContext;
@@ -112,6 +115,8 @@ import org.apache.ignite.lang.IgniteUuid;
import org.apache.ignite.plugin.security.SecurityCredentials;
import org.apache.ignite.plugin.segmentation.SegmentationPolicy;
import org.apache.ignite.spi.IgniteSpiException;
+import org.apache.ignite.spi.communication.CommunicationSpi;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
import org.apache.ignite.spi.discovery.DiscoveryDataBag;
import org.apache.ignite.spi.discovery.DiscoveryDataBag.JoiningNodeDiscoveryData;
import org.apache.ignite.spi.discovery.DiscoveryMetricsProvider;
@@ -120,10 +125,10 @@ import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
import org.apache.ignite.spi.discovery.DiscoverySpiDataExchange;
import org.apache.ignite.spi.discovery.DiscoverySpiHistorySupport;
import org.apache.ignite.spi.discovery.DiscoverySpiListener;
+import org.apache.ignite.spi.discovery.DiscoverySpiMutableCustomMessageSupport;
import org.apache.ignite.spi.discovery.DiscoverySpiNodeAuthenticator;
import org.apache.ignite.spi.discovery.DiscoverySpiOrderSupport;
import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
-import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode;
import org.apache.ignite.thread.IgniteThread;
import org.jetbrains.annotations.NotNull;
import org.jetbrains.annotations.Nullable;
@@ -478,7 +483,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
/** {@inheritDoc} */
@Override protected void onKernalStart0() throws IgniteCheckedException {
- if (Boolean.TRUE.equals(ctx.config().isClientMode()) && !getSpi().isClientMode())
+ if ((getSpi() instanceof TcpDiscoverySpi) && Boolean.TRUE.equals(ctx.config().isClientMode()) && !getSpi().isClientMode())
ctx.performance().add("Enable client mode for TcpDiscoverySpi " +
"(set TcpDiscoverySpi.forceServerMode to false)");
}
@@ -551,6 +556,9 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
});
}
+ if (ctx.config().getCommunicationFailureResolver() != null)
+ ctx.resource().injectGeneric(ctx.config().getCommunicationFailureResolver());
+
spi.setListener(new DiscoverySpiListener() {
private long gridStartTime;
@@ -559,8 +567,8 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
for (IgniteInClosure<ClusterNode> lsnr : locNodeInitLsnrs)
lsnr.apply(locNode);
- if (locNode instanceof TcpDiscoveryNode) {
- final TcpDiscoveryNode node = (TcpDiscoveryNode)locNode;
+ if (locNode instanceof IgniteClusterNode) {
+ final IgniteClusterNode node = (IgniteClusterNode)locNode;
if (consistentId != null)
node.setConsistentId(consistentId);
@@ -1052,7 +1060,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
/**
* @return Metrics provider.
*/
- private DiscoveryMetricsProvider createMetricsProvider() {
+ public DiscoveryMetricsProvider createMetricsProvider() {
return new DiscoveryMetricsProvider() {
/** */
private final long startTime = U.currentTimeMillis();
@@ -1679,13 +1687,15 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
return getSpi().pingNode(nodeId);
}
catch (IgniteException e) {
- if (e.hasCause(IgniteClientDisconnectedCheckedException.class)) {
+ if (e.hasCause(IgniteClientDisconnectedCheckedException.class, IgniteClientDisconnectedException.class)) {
IgniteFuture<?> reconnectFut = ctx.cluster().clientReconnectFuture();
throw new IgniteClientDisconnectedCheckedException(reconnectFut, e.getMessage());
}
- throw e;
+ LT.warn(log, "Ping failed with error [node=" + nodeId + ", err=" + e + ']');
+
+ return true;
}
finally {
busyLock.leaveBusy();
@@ -2025,7 +2035,16 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
Map<Long, Collection<ClusterNode>> snapshots = topHist;
- return snapshots.get(topVer);
+ Collection<ClusterNode> nodes = snapshots.get(topVer);
+
+ if (nodes == null) {
+ DiscoCache cache = discoCacheHist.get(new AffinityTopologyVersion(topVer, 0));
+
+ if (cache != null)
+ nodes = cache.allNodes();
+ }
+
+ return nodes;
}
/**
@@ -2158,6 +2177,19 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
}
/**
+ * @param discoCache
+ * @param node
+ */
+ public void metricsUpdateEvent(DiscoCache discoCache, ClusterNode node) {
+ discoWrk.addEvent(EVT_NODE_METRICS_UPDATED,
+ discoCache.version(),
+ node,
+ discoCache,
+ discoCache.nodeMap.values(),
+ null);
+ }
+
+ /**
* Gets first grid node start time, see {@link DiscoverySpi#getGridStartTime()}.
*
* @return Start time of the first grid node.
@@ -2211,8 +2243,9 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
public boolean reconnectSupported() {
DiscoverySpi spi = getSpi();
- return ctx.discovery().localNode().isClient() && (spi instanceof TcpDiscoverySpi) &&
- !(((TcpDiscoverySpi) spi).isClientReconnectDisabled());
+ return ctx.discovery().localNode().isClient() &&
+ (spi instanceof IgniteDiscoverySpi) &&
+ ((IgniteDiscoverySpi)spi).clientReconnectSupported();
}
/**
@@ -2225,7 +2258,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
DiscoverySpi discoverySpi = getSpi();
- ((TcpDiscoverySpi)discoverySpi).reconnect();
+ ((IgniteDiscoverySpi)discoverySpi).clientReconnect();
}
/**
@@ -2379,6 +2412,76 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
cacheNodes.add(rich);
}
+ /**
+ * @param cfg Configuration.
+ * @throws IgniteCheckedException If configuration is not valid.
+ */
+ public static void initCommunicationErrorResolveConfiguration(IgniteConfiguration cfg) throws IgniteCheckedException {
+ CommunicationFailureResolver rslvr = cfg.getCommunicationFailureResolver();
+ CommunicationSpi commSpi = cfg.getCommunicationSpi();
+ DiscoverySpi discoverySpi = cfg.getDiscoverySpi();
+
+ if (rslvr != null) {
+ if (!supportsCommunicationErrorResolve(commSpi))
+ throw new IgniteCheckedException("CommunicationFailureResolver is configured, but CommunicationSpi does not support communication" +
+ "problem resolve: " + commSpi.getClass().getName());
+
+ if (!supportsCommunicationErrorResolve(discoverySpi))
+ throw new IgniteCheckedException("CommunicationFailureResolver is configured, but DiscoverySpi does not support communication" +
+ "problem resolve: " + discoverySpi.getClass().getName());
+ }
+ else {
+ if (supportsCommunicationErrorResolve(commSpi) && supportsCommunicationErrorResolve(discoverySpi))
+ cfg.setCommunicationFailureResolver(new DefaultCommunicationFailureResolver());
+ }
+ }
+
+ /**
+ * @param spi Discovery SPI.
+ * @return {@code True} if SPI supports communication error resolve.
+ */
+ private static boolean supportsCommunicationErrorResolve(DiscoverySpi spi) {
+ return spi instanceof IgniteDiscoverySpi && ((IgniteDiscoverySpi)spi).supportsCommunicationFailureResolve();
+ }
+
+ /**
+ * @param spi Discovery SPI.
+ * @return {@code True} if SPI supports communication error resolve.
+ */
+ private static boolean supportsCommunicationErrorResolve(CommunicationSpi spi) {
+ return spi instanceof TcpCommunicationSpi;
+ }
+
+ /**
+ * @return {@code True} if communication error resolve is supported.
+ */
+ public boolean communicationErrorResolveSupported() {
+ return ctx.config().getCommunicationFailureResolver() != null;
+ }
+
+ /**
+ * @return {@code True} if configured {@link DiscoverySpi} supports mutable custom messages.
+ */
+ public boolean mutableCustomMessages() {
+ DiscoverySpiMutableCustomMessageSupport ann = U.getAnnotation(ctx.config().getDiscoverySpi().getClass(),
+ DiscoverySpiMutableCustomMessageSupport.class);
+
+ return ann != null && ann.value();
+ }
+
+ /**
+ * @param node Problem node.
+ * @param err Error.
+ */
+ public void resolveCommunicationError(ClusterNode node, Exception err) {
+ DiscoverySpi spi = getSpi();
+
+ if (!supportsCommunicationErrorResolve(spi) || !supportsCommunicationErrorResolve(ctx.config().getCommunicationSpi()))
+ throw new UnsupportedOperationException();
+
+ ((IgniteDiscoverySpi)spi).resolveCommunicationFailure(node, err);
+ }
+
/** Worker for network segment checks. */
private class SegmentCheckWorker extends GridWorker {
/** */
@@ -2587,6 +2690,9 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
AffinityTopologyVersion topVer = evt.get2();
+ if (type == EVT_NODE_METRICS_UPDATED && topVer.compareTo(discoCache.version()) < 0)
+ return;
+
ClusterNode node = evt.get3();
boolean isDaemon = node.isDaemon();
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/IgniteClusterNode.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/IgniteClusterNode.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/IgniteClusterNode.java
new file mode 100644
index 0000000..cbc706a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/IgniteClusterNode.java
@@ -0,0 +1,69 @@
+/*
+ * 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.managers.discovery;
+
+import java.io.Serializable;
+import java.util.Map;
+import org.apache.ignite.cache.CacheMetrics;
+import org.apache.ignite.cluster.ClusterMetrics;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.IgniteConfiguration;
+
+/**
+ *
+ */
+public interface IgniteClusterNode extends ClusterNode {
+ /**
+ * Sets consistent globally unique node ID which survives node restarts.
+ *
+ * @param consistentId Consistent globally unique node ID.
+ */
+ public void setConsistentId(Serializable consistentId);
+
+ /**
+ * Sets node metrics.
+ *
+ * @param metrics Node metrics.
+ */
+ public void setMetrics(ClusterMetrics metrics);
+
+ /**
+ * Gets collections of cache metrics for this node. Note that node cache metrics are constantly updated
+ * and provide up to date information about caches.
+ * <p>
+ * Cache metrics are updated with some delay which is directly related to metrics update
+ * frequency. For example, by default the update will happen every {@code 2} seconds.
+ *
+ * @return Runtime metrics snapshots for this node.
+ */
+ public Map<Integer, CacheMetrics> cacheMetrics();
+
+ /**
+ * Sets node cache metrics.
+ *
+ * @param cacheMetrics Cache metrics.
+ */
+ public void setCacheMetrics(Map<Integer, CacheMetrics> cacheMetrics);
+
+ /**
+ * Whether this node is cache client (see {@link IgniteConfiguration#isClientMode()}).
+ *
+ * @return {@code True if client}.
+ */
+ public boolean isCacheClient();
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/IgniteDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/IgniteDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/IgniteDiscoverySpi.java
new file mode 100644
index 0000000..9aa5d14
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/IgniteDiscoverySpi.java
@@ -0,0 +1,67 @@
+/*
+ * 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.managers.discovery;
+
+import java.util.UUID;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.spi.discovery.DiscoverySpi;
+
+/**
+ *
+ */
+public interface IgniteDiscoverySpi extends DiscoverySpi {
+ /**
+ * @param nodeId Node ID.
+ * @return {@code True} if node joining or already joined topology.
+ */
+ public boolean knownNode(UUID nodeId);
+
+ /**
+ *
+ * @return {@code True} if SPI supports client reconnect.
+ */
+ public boolean clientReconnectSupported();
+
+ /**
+ *
+ */
+ public void clientReconnect();
+
+ /**
+ * For TESTING only.
+ */
+ public void simulateNodeFailure();
+
+ /**
+ * For TESTING only.
+ *
+ * @param lsnr Listener.
+ */
+ public void setInternalListener(IgniteDiscoverySpiInternalListener lsnr);
+
+ /**
+ * @return {@code True} if supports communication error resolve.
+ */
+ public boolean supportsCommunicationFailureResolve();
+
+ /**
+ * @param node Problem node.
+ * @param err Connection error.
+ */
+ public void resolveCommunicationFailure(ClusterNode node, Exception err);
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/IgniteDiscoverySpiInternalListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/IgniteDiscoverySpiInternalListener.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/IgniteDiscoverySpiInternalListener.java
new file mode 100644
index 0000000..24405f8
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/IgniteDiscoverySpiInternalListener.java
@@ -0,0 +1,42 @@
+/*
+ * 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.managers.discovery;
+
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.spi.discovery.DiscoverySpi;
+import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
+
+/**
+ * For TESTING only.
+ */
+public interface IgniteDiscoverySpiInternalListener {
+ /**
+ * @param locNode Local node.
+ * @param log Log.
+ */
+ public void beforeJoin(ClusterNode locNode, IgniteLogger log);
+
+ /**
+ * @param spi SPI instance.
+ * @param log Logger.
+ * @param msg Custom message.
+ * @return {@code False} to cancel event send.
+ */
+ public boolean beforeSendCustomEvent(DiscoverySpi spi, IgniteLogger log, DiscoverySpiCustomMessage msg);
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/UserAcceptedMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/UserAcceptedMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/UserAcceptedMessage.java
index ef87a44..2e2aed9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/UserAcceptedMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/UserAcceptedMessage.java
@@ -72,6 +72,11 @@ public class UserAcceptedMessage implements DiscoveryCustomMessage {
}
/** {@inheritDoc} */
+ @Override public boolean stopProcess() {
+ return false;
+ }
+
+ /** {@inheritDoc} */
@Nullable @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr,
AffinityTopologyVersion topVer, DiscoCache discoCache) {
throw new UnsupportedOperationException();
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/UserProposedMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/UserProposedMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/UserProposedMessage.java
index 1a0be8e..19f9e82 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/UserProposedMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/UserProposedMessage.java
@@ -72,6 +72,11 @@ public class UserProposedMessage implements DiscoveryServerOnlyCustomMessage {
}
/** {@inheritDoc} */
+ @Override public boolean stopProcess() {
+ return false;
+ }
+
+ /** {@inheritDoc} */
@Nullable @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr,
AffinityTopologyVersion topVer, DiscoCache discoCache) {
throw new UnsupportedOperationException();
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinityChangeMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinityChangeMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinityChangeMessage.java
index fe1014c..937a889 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinityChangeMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinityChangeMessage.java
@@ -156,6 +156,11 @@ public class CacheAffinityChangeMessage implements DiscoveryCustomMessage {
}
/** {@inheritDoc} */
+ @Override public boolean stopProcess() {
+ return false;
+ }
+
+ /** {@inheritDoc} */
@Nullable @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr,
AffinityTopologyVersion topVer, DiscoCache discoCache) {
return discoCache.copy(topVer, null);
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
index 6691b13..92b8d3e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
@@ -1310,20 +1310,17 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
* @param fut Current exchange future.
* @param msg Message finish message.
* @param resTopVer Result topology version.
- * @throws IgniteCheckedException If failed.
*/
public void onLocalJoin(final GridDhtPartitionsExchangeFuture fut,
GridDhtPartitionsFullMessage msg,
- final AffinityTopologyVersion resTopVer)
- throws IgniteCheckedException {
+ final AffinityTopologyVersion resTopVer) {
final Set<Integer> affReq = fut.context().groupsAffinityRequestOnJoin();
final Map<Long, ClusterNode> nodesByOrder = new HashMap<>();
final Map<Integer, CacheGroupAffinityMessage> joinedNodeAff = msg.joinedNodeAffinity();
- assert !F.isEmpty(joinedNodeAff) : msg;
- assert joinedNodeAff.size() >= affReq.size();
+ assert F.isEmpty(affReq) || (!F.isEmpty(joinedNodeAff) && joinedNodeAff.size() >= affReq.size()) : msg;
forAllCacheGroups(false, new IgniteInClosureX<GridAffinityAssignmentCache>() {
@Override public void applyx(GridAffinityAssignmentCache aff) throws IgniteCheckedException {
@@ -1333,7 +1330,7 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
assert grp != null;
- if (affReq.contains(aff.groupId())) {
+ if (affReq != null && affReq.contains(aff.groupId())) {
assert AffinityTopologyVersion.NONE.equals(aff.lastVersion());
CacheGroupAffinityMessage affMsg = joinedNodeAff.get(aff.groupId());
@@ -2282,6 +2279,23 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
}
/**
+ * @return All registered cache groups.
+ */
+ public Map<Integer, DynamicCacheDescriptor> caches() {
+ return caches.registeredCaches;
+ }
+
+ /**
+ * @param grpId Cache group ID
+ * @return Cache affinity cache.
+ */
+ @Nullable public GridAffinityAssignmentCache groupAffinity(int grpId) {
+ CacheGroupHolder grpHolder = grpHolders.get(grpId);
+
+ return grpHolder != null ? grpHolder.affinity() : null;
+ }
+
+ /**
*
*/
public void dumpDebugInfo() {
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStatisticsModeChangeMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStatisticsModeChangeMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStatisticsModeChangeMessage.java
index 40bcfaf..e33256f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStatisticsModeChangeMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStatisticsModeChangeMessage.java
@@ -101,6 +101,11 @@ public class CacheStatisticsModeChangeMessage implements DiscoveryCustomMessage
}
/** {@inheritDoc} */
+ @Override public boolean stopProcess() {
+ return false;
+ }
+
+ /** {@inheritDoc} */
@Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr, AffinityTopologyVersion topVer,
DiscoCache discoCache) {
throw new UnsupportedOperationException();
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClientCacheChangeDiscoveryMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClientCacheChangeDiscoveryMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClientCacheChangeDiscoveryMessage.java
index e35d80e..ae76c95 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClientCacheChangeDiscoveryMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClientCacheChangeDiscoveryMessage.java
@@ -173,6 +173,11 @@ public class ClientCacheChangeDiscoveryMessage implements DiscoveryCustomMessage
}
/** {@inheritDoc} */
+ @Override public boolean stopProcess() {
+ return false;
+ }
+
+ /** {@inheritDoc} */
@Nullable @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr,
AffinityTopologyVersion topVer, DiscoCache discoCache) {
throw new UnsupportedOperationException();
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClientCacheChangeDummyDiscoveryMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClientCacheChangeDummyDiscoveryMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClientCacheChangeDummyDiscoveryMessage.java
index 6ed3ecc..4ce0c87 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClientCacheChangeDummyDiscoveryMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClientCacheChangeDummyDiscoveryMessage.java
@@ -105,6 +105,11 @@ public class ClientCacheChangeDummyDiscoveryMessage implements DiscoveryCustomMe
}
/** {@inheritDoc} */
+ @Override public boolean stopProcess() {
+ return false;
+ }
+
+ /** {@inheritDoc} */
@Nullable @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr,
AffinityTopologyVersion topVer, DiscoCache discoCache) {
throw new UnsupportedOperationException();
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
index 83459a5..d85e29b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
@@ -77,6 +77,11 @@ public class DynamicCacheChangeBatch implements DiscoveryCustomMessage {
}
/** {@inheritDoc} */
+ @Override public boolean stopProcess() {
+ return false;
+ }
+
+ /** {@inheritDoc} */
@Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr, AffinityTopologyVersion topVer,
DiscoCache discoCache) {
return mgr.createDiscoCacheOnCacheChange(topVer, discoCache);
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 55357ff..c2d0f42 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -82,6 +82,7 @@ import org.apache.ignite.internal.NodeStoppingException;
import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
import org.apache.ignite.internal.cluster.ClusterTopologyServerNotFoundException;
import org.apache.ignite.internal.cluster.IgniteClusterEx;
+import org.apache.ignite.internal.managers.discovery.IgniteClusterNode;
import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
import org.apache.ignite.internal.processors.cache.affinity.GridCacheAffinityImpl;
import org.apache.ignite.internal.processors.cache.distributed.IgniteExternalizableExpiryPolicy;
@@ -3234,7 +3235,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
List<CacheMetrics> metrics = new ArrayList<>(grp.nodes().size());
for (ClusterNode node : grp.nodes()) {
- Map<Integer, CacheMetrics> nodeCacheMetrics = ((TcpDiscoveryNode)node).cacheMetrics();
+ Map<Integer, CacheMetrics> nodeCacheMetrics = ((IgniteClusterNode)node).cacheMetrics();
if (nodeCacheMetrics != null) {
CacheMetrics e = nodeCacheMetrics.get(context().cacheId());
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
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 a30a24a..77ffce3 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
@@ -427,7 +427,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
if (evt.type() != EVT_DISCOVERY_CUSTOM_EVT) {
assert evt.type() != EVT_NODE_JOINED || n.isLocal() || n.order() > loc.order() :
"Node joined with smaller-than-local " +
- "order [newOrder=" + n.order() + ", locOrder=" + loc.order() + ']';
+ "order [newOrder=" + n.order() + ", locOrder=" + loc.order() + ", evt=" + evt + ']';
exchId = exchangeId(n.id(), affinityTopologyVersion(evt), evt);
@@ -570,12 +570,6 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
for (ClusterNode n : cctx.discovery().remoteNodes())
cctx.versions().onReceived(n.id(), n.metrics().getLastDataVersion());
- ClusterNode loc = cctx.localNode();
-
- long startTime = loc.metrics().getStartTime();
-
- assert startTime > 0;
-
DiscoveryLocalJoinData locJoin = cctx.discovery().localJoin();
GridDhtPartitionsExchangeFuture fut = null;
@@ -758,6 +752,14 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
/**
* @param grpId Cache group ID.
+ * @return Topology.
+ */
+ @Nullable public GridDhtPartitionTopology clientTopologyIfExists(int grpId) {
+ return clientTops.get(grpId);
+ }
+
+ /**
+ * @param grpId Cache group ID.
* @param discoCache Discovery data cache.
* @return Topology.
*/
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index a3f7c94..7edac73 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -3363,7 +3363,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
* @return Validation result or {@code null} in case of success.
*/
@Nullable private IgniteNodeValidationResult validateHashIdResolvers(ClusterNode node) {
- if (!node.isClient()) {
+ if (!CU.clientNode(node)) {
for (DynamicCacheDescriptor desc : cacheDescriptors().values()) {
CacheConfiguration cfg = desc.cacheConfiguration();
@@ -3372,7 +3372,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
Object nodeHashObj = aff.resolveNodeHash(node);
- for (ClusterNode topNode : ctx.discovery().allNodes()) {
+ for (ClusterNode topNode : ctx.discovery().aliveServerNodes()) {
Object topNodeHashObj = aff.resolveNodeHash(topNode);
if (nodeHashObj.hashCode() == topNodeHashObj.hashCode()) {
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index 83ce2ba..a5169d2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
@@ -65,6 +65,7 @@ import org.apache.ignite.internal.IgniteNodeAttributes;
import org.apache.ignite.internal.cluster.ClusterGroupEmptyCheckedException;
import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
import org.apache.ignite.internal.cluster.ClusterTopologyServerNotFoundException;
+import org.apache.ignite.internal.managers.discovery.IgniteClusterNode;
import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
import org.apache.ignite.internal.processors.cache.distributed.GridDistributedLockCancelledException;
import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheAdapter;
@@ -97,7 +98,6 @@ import org.apache.ignite.lang.IgnitePredicate;
import org.apache.ignite.lang.IgniteReducer;
import org.apache.ignite.lifecycle.LifecycleAware;
import org.apache.ignite.plugin.CachePluginConfiguration;
-import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode;
import org.apache.ignite.transactions.Transaction;
import org.apache.ignite.transactions.TransactionConcurrency;
import org.apache.ignite.transactions.TransactionIsolation;
@@ -1348,8 +1348,8 @@ public class GridCacheUtils {
* @return {@code True} if given node is client node (has flag {@link IgniteConfiguration#isClientMode()} set).
*/
public static boolean clientNode(ClusterNode node) {
- if (node instanceof TcpDiscoveryNode)
- return ((TcpDiscoveryNode)node).isCacheClient();
+ if (node instanceof IgniteClusterNode)
+ return ((IgniteClusterNode)node).isCacheClient();
else
return clientNodeDirect(node);
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateFinishMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateFinishMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateFinishMessage.java
index 57f25d0..4afa403 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateFinishMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateFinishMessage.java
@@ -67,6 +67,11 @@ public class WalStateFinishMessage extends WalStateAbstractMessage {
}
/** {@inheritDoc} */
+ @Override public boolean stopProcess() {
+ return false;
+ }
+
+ /** {@inheritDoc} */
@Override public String toString() {
return S.toString(WalStateFinishMessage.class, this, "super", super.toString());
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateProposeMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateProposeMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateProposeMessage.java
index 747fd6a..b9d96fc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateProposeMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateProposeMessage.java
@@ -98,6 +98,11 @@ public class WalStateProposeMessage extends WalStateAbstractMessage {
}
/** {@inheritDoc} */
+ @Override public boolean stopProcess() {
+ return false;
+ }
+
+ /** {@inheritDoc} */
@Override public String toString() {
return S.toString(WalStateProposeMessage.class, this, "super", super.toString());
}
[09/12] ignite git commit: IGNITE-7222 Added ZooKeeper discovery SPI
Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpCommunicationConnectionCheckFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpCommunicationConnectionCheckFuture.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpCommunicationConnectionCheckFuture.java
new file mode 100644
index 0000000..c42fa57
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpCommunicationConnectionCheckFuture.java
@@ -0,0 +1,519 @@
+/*
+ * 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.spi.communication.tcp.internal;
+
+import java.net.InetSocketAddress;
+import java.nio.channels.SocketChannel;
+import java.util.BitSet;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.events.DiscoveryEvent;
+import org.apache.ignite.events.Event;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
+import org.apache.ignite.internal.util.GridLeanMap;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.nio.GridNioServer;
+import org.apache.ignite.internal.util.nio.GridNioSession;
+import org.apache.ignite.internal.util.nio.GridNioSessionMetaKey;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.spi.IgniteSpiTimeoutObject;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
+import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
+
+/**
+ * Tcp Communication Connection Check Future.
+ */
+public class TcpCommunicationConnectionCheckFuture extends GridFutureAdapter<BitSet> implements IgniteSpiTimeoutObject, GridLocalEventListener {
+ /** Session future. */
+ public static final int SES_FUT_META = GridNioSessionMetaKey.nextUniqueKey();
+
+ /** */
+ private static final AtomicIntegerFieldUpdater<SingleAddressConnectFuture> connFutDoneUpdater =
+ AtomicIntegerFieldUpdater.newUpdater(SingleAddressConnectFuture.class, "done");
+
+ /** */
+ private static final AtomicIntegerFieldUpdater<MultipleAddressesConnectFuture> connResCntUpdater =
+ AtomicIntegerFieldUpdater.newUpdater(MultipleAddressesConnectFuture.class, "resCnt");
+
+ /** */
+ private final AtomicInteger resCntr = new AtomicInteger();
+
+ /** */
+ private final List<ClusterNode> nodes;
+
+ /** */
+ private volatile ConnectFuture[] futs;
+
+ /** */
+ private final GridNioServer nioSrvr;
+
+ /** */
+ private final TcpCommunicationSpi spi;
+
+ /** */
+ private final IgniteUuid timeoutObjId = IgniteUuid.randomUuid();
+
+ /** */
+ private final BitSet resBitSet;
+
+ /** */
+ private long endTime;
+
+ /** */
+ private final IgniteLogger log;
+
+ /**
+ * @param spi SPI instance.
+ * @param log Logger.
+ * @param nioSrvr NIO server.
+ * @param nodes Nodes to check.
+ */
+ public TcpCommunicationConnectionCheckFuture(TcpCommunicationSpi spi,
+ IgniteLogger log,
+ GridNioServer nioSrvr,
+ List<ClusterNode> nodes)
+ {
+ this.spi = spi;
+ this.log = log;
+ this.nioSrvr = nioSrvr;
+ this.nodes = nodes;
+
+ resBitSet = new BitSet(nodes.size());
+ }
+
+ /**
+ * @param timeout Connect timeout.
+ */
+ public void init(long timeout) {
+ ConnectFuture[] futs = new ConnectFuture[nodes.size()];
+
+ UUID locId = spi.getSpiContext().localNode().id();
+
+ for (int i = 0; i < nodes.size(); i++) {
+ ClusterNode node = nodes.get(i);
+
+ if (!node.id().equals(locId)) {
+ if (spi.getSpiContext().node(node.id()) == null) {
+ receivedConnectionStatus(i, false);
+
+ continue;
+ }
+
+ Collection<InetSocketAddress> addrs;
+
+ try {
+ addrs = spi.nodeAddresses(node, false);
+ }
+ catch (Exception e) {
+ U.error(log, "Failed to get node addresses: " + node, e);
+
+ receivedConnectionStatus(i, false);
+
+ continue;
+ }
+
+ if (addrs.size() == 1) {
+ SingleAddressConnectFuture fut = new SingleAddressConnectFuture(i);
+
+ fut.init(addrs.iterator().next(), node.id());
+
+ futs[i] = fut;
+ }
+ else {
+ MultipleAddressesConnectFuture fut = new MultipleAddressesConnectFuture(i);
+
+ fut.init(addrs, node.id());
+
+ futs[i] = fut;
+ }
+ }
+ else
+ receivedConnectionStatus(i, true);
+ }
+
+ this.futs = futs;
+
+ spi.getSpiContext().addLocalEventListener(this, EVT_NODE_LEFT, EVT_NODE_FAILED);
+
+ if (!isDone()) {
+ endTime = System.currentTimeMillis() + timeout;
+
+ spi.getSpiContext().addTimeoutObject(this);
+ }
+ }
+
+ /**
+ * @param idx Node index.
+ * @param res Success flag.
+ */
+ private void receivedConnectionStatus(int idx, boolean res) {
+ assert resCntr.get() < nodes.size();
+
+ synchronized (resBitSet) {
+ resBitSet.set(idx, res);
+ }
+
+ if (resCntr.incrementAndGet() == nodes.size())
+ onDone(resBitSet);
+ }
+
+ /**
+ * @param nodeIdx Node index.
+ * @return Node ID.
+ */
+ private UUID nodeId(int nodeIdx) {
+ return nodes.get(nodeIdx).id();
+ }
+
+ /** {@inheritDoc} */
+ @Override public IgniteUuid id() {
+ return timeoutObjId;
+ }
+
+ /** {@inheritDoc} */
+ @Override public long endTime() {
+ return endTime;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void onEvent(Event evt) {
+ if (isDone())
+ return;
+
+ assert evt instanceof DiscoveryEvent : evt;
+ assert evt.type() == EVT_NODE_LEFT || evt.type() == EVT_NODE_FAILED ;
+
+ UUID nodeId = ((DiscoveryEvent)evt).eventNode().id();
+
+ for (int i = 0; i < nodes.size(); i++) {
+ if (nodes.get(i).id().equals(nodeId)) {
+ ConnectFuture fut = futs[i];
+
+ if (fut != null)
+ fut.onNodeFailed();
+
+ return;
+ }
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override public void onTimeout() {
+ if (isDone())
+ return;
+
+ ConnectFuture[] futs = this.futs;
+
+ for (int i = 0; i < futs.length; i++) {
+ ConnectFuture fut = futs[i];
+
+ if (fut != null)
+ fut.onTimeout();
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean onDone(@Nullable BitSet res, @Nullable Throwable err) {
+ if (super.onDone(res, err)) {
+ spi.getSpiContext().removeTimeoutObject(this);
+
+ spi.getSpiContext().removeLocalEventListener(this);
+
+ return true;
+ }
+
+ return false;
+ }
+
+ /**
+ *
+ */
+ private interface ConnectFuture {
+ /**
+ *
+ */
+ void onTimeout();
+
+ /**
+ *
+ */
+ void onNodeFailed();
+ }
+
+ /**
+ *
+ */
+ private class SingleAddressConnectFuture implements TcpCommunicationNodeConnectionCheckFuture, ConnectFuture {
+ /** */
+ final int nodeIdx;
+
+ /** */
+ volatile int done;
+
+ /** */
+ Map<Integer, Object> sesMeta;
+
+ /** */
+ private SocketChannel ch;
+
+ /**
+ * @param nodeIdx Node index.
+ */
+ SingleAddressConnectFuture(int nodeIdx) {
+ this.nodeIdx = nodeIdx;
+ }
+
+ /**
+ * @param addr Node address.
+ * @param rmtNodeId Id of node to open connection check session with.
+ */
+ public void init(InetSocketAddress addr, UUID rmtNodeId) {
+ boolean connect;
+
+ try {
+ ch = SocketChannel.open();
+
+ ch.configureBlocking(false);
+
+ ch.socket().setTcpNoDelay(true);
+ ch.socket().setKeepAlive(false);
+
+ connect = ch.connect(addr);
+ }
+ catch (Exception e) {
+ finish(false);
+
+ return;
+ }
+
+ if (!connect) {
+ sesMeta = new GridLeanMap<>(3);
+
+ // Set dummy key to identify connection-check outgoing connection.
+ sesMeta.put(TcpCommunicationSpi.CONN_IDX_META, new ConnectionKey(rmtNodeId, -1, -1, true));
+ sesMeta.put(SES_FUT_META, this);
+
+ nioSrvr.createSession(ch, sesMeta, true, new IgniteInClosure<IgniteInternalFuture<GridNioSession>>() {
+ @Override public void apply(IgniteInternalFuture<GridNioSession> fut) {
+ if (fut.error() != null)
+ finish(false);
+ }
+ });
+ }
+ }
+
+ /**
+ *
+ */
+ @SuppressWarnings("unchecked")
+ void cancel() {
+ if (finish(false))
+ nioSrvr.cancelConnect(ch, sesMeta);
+ }
+
+ /** {@inheritDoc} */
+ public void onTimeout() {
+ cancel();
+ }
+
+ /** {@inheritDoc} */
+ public void onConnected(UUID rmtNodeId) {
+ finish(nodeId(nodeIdx).equals(rmtNodeId));
+ }
+
+ /** {@inheritDoc} */
+ @Override public void onNodeFailed() {
+ cancel();
+ }
+
+ /**
+ * @param res Result.
+ * @return {@code True} if result was set by this call.
+ */
+ public boolean finish(boolean res) {
+ if (connFutDoneUpdater.compareAndSet(this, 0, 1)) {
+ onStatusReceived(res);
+
+ return true;
+ }
+
+ return false;
+ }
+
+ /**
+ * @param res Result.
+ */
+ void onStatusReceived(boolean res) {
+ receivedConnectionStatus(nodeIdx, res);
+ }
+ }
+
+ /**
+ *
+ */
+ private class MultipleAddressesConnectFuture implements ConnectFuture {
+ /** */
+ volatile int resCnt;
+
+ /** */
+ volatile SingleAddressConnectFuture[] futs;
+
+ /** */
+ final int nodeIdx;
+
+ /**
+ * @param nodeIdx Node index.
+ */
+ MultipleAddressesConnectFuture(int nodeIdx) {
+ this.nodeIdx = nodeIdx;
+
+ }
+
+ /** {@inheritDoc} */
+ @Override public void onNodeFailed() {
+ SingleAddressConnectFuture[] futs = this.futs;
+
+ for (int i = 0; i < futs.length; i++) {
+ ConnectFuture fut = futs[i];
+
+ if (fut != null)
+ fut.onNodeFailed();
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override public void onTimeout() {
+ SingleAddressConnectFuture[] futs = this.futs;
+
+ for (int i = 0; i < futs.length; i++) {
+ ConnectFuture fut = futs[i];
+
+ if (fut != null)
+ fut.onTimeout();
+ }
+ }
+
+ /**
+ * @param addrs Node addresses.
+ * @param rmtNodeId Id of node to open connection check session with.
+ */
+ void init(Collection<InetSocketAddress> addrs, UUID rmtNodeId) {
+ SingleAddressConnectFuture[] futs = new SingleAddressConnectFuture[addrs.size()];
+
+ for (int i = 0; i < addrs.size(); i++) {
+ SingleAddressConnectFuture fut = new SingleAddressConnectFuture(nodeIdx) {
+ @Override void onStatusReceived(boolean res) {
+ receivedAddressStatus(res);
+ }
+ };
+
+ futs[i] = fut;
+ }
+
+ this.futs = futs;
+
+ int idx = 0;
+
+ for (InetSocketAddress addr : addrs) {
+ futs[idx++].init(addr, rmtNodeId);
+
+ if (resCnt == Integer.MAX_VALUE)
+ return;
+ }
+
+ // Close race.
+ if (done())
+ cancelFutures();
+ }
+
+ /**
+ * @return {@code True}
+ */
+ private boolean done() {
+ int resCnt0 = resCnt;
+
+ return resCnt0 == Integer.MAX_VALUE || resCnt0 == futs.length;
+ }
+
+ /**
+ *
+ */
+ private void cancelFutures() {
+ SingleAddressConnectFuture[] futs = this.futs;
+
+ if (futs != null) {
+ for (int i = 0; i < futs.length; i++) {
+ SingleAddressConnectFuture fut = futs[i];
+
+ fut.cancel();
+ }
+ }
+ }
+
+ /**
+ * @param res Result.
+ */
+ void receivedAddressStatus(boolean res) {
+ if (res) {
+ for (;;) {
+ int resCnt0 = resCnt;
+
+ if (resCnt0 == Integer.MAX_VALUE)
+ return;
+
+ if (connResCntUpdater.compareAndSet(this, resCnt0, Integer.MAX_VALUE)) {
+ receivedConnectionStatus(nodeIdx, true);
+
+ cancelFutures(); // Cancel others connects if they are still in progress.
+
+ return;
+ }
+ }
+ }
+ else {
+ for (;;) {
+ int resCnt0 = resCnt;
+
+ if (resCnt0 == Integer.MAX_VALUE)
+ return;
+
+ int resCnt1 = resCnt0 + 1;
+
+ if (connResCntUpdater.compareAndSet(this, resCnt0, resCnt1)) {
+ if (resCnt1 == futs.length)
+ receivedConnectionStatus(nodeIdx, false);
+
+ return;
+ }
+ }
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpCommunicationNodeConnectionCheckFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpCommunicationNodeConnectionCheckFuture.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpCommunicationNodeConnectionCheckFuture.java
new file mode 100644
index 0000000..cbf27b5
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpCommunicationNodeConnectionCheckFuture.java
@@ -0,0 +1,30 @@
+/*
+ * 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.spi.communication.tcp.internal;
+
+import java.util.UUID;
+
+/**
+ * Tcp Communication Node Connection Check Future.
+ */
+public interface TcpCommunicationNodeConnectionCheckFuture {
+ /**
+ * @param nodeId Remote node ID.
+ */
+ public void onConnected(UUID nodeId);
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiCustomMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiCustomMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiCustomMessage.java
index a0f9b75..f26ad33 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiCustomMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiCustomMessage.java
@@ -30,12 +30,23 @@ import org.jetbrains.annotations.Nullable;
*/
public interface DiscoverySpiCustomMessage extends Serializable {
/**
- * Called when message passed the ring.
+ * Called when custom message has been handled by all nodes.
+ *
+ * @return Ack message or {@code null} if ack is not required.
*/
@Nullable public DiscoverySpiCustomMessage ackMessage();
/**
- * @return {@code true} if message can be modified during listener notification. Changes will be send to next nodes.
+ * @return {@code True} if message can be modified during listener notification. Changes will be send to next nodes.
*/
public boolean isMutable();
+
+ /**
+ * Called on discovery coordinator node after listener is notified. If returns {@code true}
+ * then message is not passed to others nodes, if after this method {@link #ackMessage()} returns non-null ack
+ * message, it is sent to all nodes.
+ *
+ * @return {@code True} if message should not be sent to all nodes.
+ */
+ public boolean stopProcess();
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiMutableCustomMessageSupport.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiMutableCustomMessageSupport.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiMutableCustomMessageSupport.java
new file mode 100644
index 0000000..37aa323
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiMutableCustomMessageSupport.java
@@ -0,0 +1,40 @@
+/*
+ * 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.spi.discovery;
+
+import java.lang.annotation.Documented;
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Inherited;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * This annotation is for all implementations of {@link DiscoverySpi} that support
+ * topology mutable {@link DiscoverySpiCustomMessage}s.
+ */
+@Documented
+@Inherited
+@Retention(RetentionPolicy.RUNTIME)
+@Target({ElementType.TYPE})
+public @interface DiscoverySpiMutableCustomMessageSupport {
+ /**
+ * @return Whether or not target SPI supports mutable {@link DiscoverySpiCustomMessage}s.
+ */
+ public boolean value();
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
index 2d9a314..f0a5186 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
@@ -55,6 +55,8 @@ import org.apache.ignite.cluster.ClusterNode;
import org.apache.ignite.configuration.AddressResolver;
import org.apache.ignite.configuration.IgniteConfiguration;
import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi;
+import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpiInternalListener;
import org.apache.ignite.internal.IgniteKernal;
import org.apache.ignite.internal.util.tostring.GridToStringExclude;
import org.apache.ignite.internal.util.typedef.F;
@@ -88,6 +90,7 @@ import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
import org.apache.ignite.spi.discovery.DiscoverySpiDataExchange;
import org.apache.ignite.spi.discovery.DiscoverySpiHistorySupport;
import org.apache.ignite.spi.discovery.DiscoverySpiListener;
+import org.apache.ignite.spi.discovery.DiscoverySpiMutableCustomMessageSupport;
import org.apache.ignite.spi.discovery.DiscoverySpiNodeAuthenticator;
import org.apache.ignite.spi.discovery.DiscoverySpiOrderSupport;
import org.apache.ignite.spi.discovery.tcp.internal.DiscoveryDataPacket;
@@ -103,6 +106,7 @@ import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryAuthFailedMessag
import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryCheckFailedMessage;
import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryDuplicateIdMessage;
import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryEnsureDelivery;
+import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryJoinRequestMessage;
import org.jetbrains.annotations.Nullable;
import static org.apache.ignite.IgniteSystemProperties.IGNITE_CONSISTENT_ID_BY_HOST_WITHOUT_PORT;
@@ -223,7 +227,8 @@ import static org.apache.ignite.IgniteSystemProperties.getBoolean;
@IgniteSpiMultipleInstancesSupport(true)
@DiscoverySpiOrderSupport(true)
@DiscoverySpiHistorySupport(true)
-public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi {
+@DiscoverySpiMutableCustomMessageSupport(true)
+public class TcpDiscoverySpi extends IgniteSpiAdapter implements IgniteDiscoverySpi {
/** Node attribute that is mapped to node's external addresses (value is <tt>disc.tcp.ext-addrs</tt>). */
public static final String ATTR_EXT_ADDRS = "disc.tcp.ext-addrs";
@@ -409,6 +414,9 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi {
/** */
protected IgniteSpiContext spiCtx;
+ /** */
+ private IgniteDiscoverySpiInternalListener internalLsnr;
+
/**
* Gets current SPI state.
*
@@ -473,6 +481,13 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi {
/** {@inheritDoc} */
@Override public void sendCustomEvent(DiscoverySpiCustomMessage msg) throws IgniteException {
+ IgniteDiscoverySpiInternalListener internalLsnr = this.internalLsnr;
+
+ if (internalLsnr != null) {
+ if (!internalLsnr.beforeSendCustomEvent(this, log, msg))
+ return;
+ }
+
impl.sendCustomEvent(msg);
}
@@ -1559,6 +1574,9 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi {
OutputStream out,
TcpDiscoveryAbstractMessage msg,
long timeout) throws IOException, IgniteCheckedException {
+ if (internalLsnr != null && msg instanceof TcpDiscoveryJoinRequestMessage)
+ internalLsnr.beforeJoin(locNode, log);
+
assert sock != null;
assert msg != null;
assert out != null;
@@ -2118,15 +2136,31 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi {
return ignite().configuration().getSslContextFactory() != null;
}
- /**
- * Force reconnect to cluster.
- *
- * @throws IgniteSpiException If failed.
- */
- public void reconnect() throws IgniteSpiException {
+ /** {@inheritDoc} */
+ public void clientReconnect() throws IgniteSpiException {
impl.reconnect();
}
+ /** {@inheritDoc} */
+ @Override public boolean knownNode(UUID nodeId) {
+ return getNode0(nodeId) != null;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean clientReconnectSupported() {
+ return !clientReconnectDisabled;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean supportsCommunicationFailureResolve() {
+ return false;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void resolveCommunicationFailure(ClusterNode node, Exception err) {
+ throw new UnsupportedOperationException();
+ }
+
/**
* <strong>FOR TEST ONLY!!!</strong>
*/
@@ -2148,6 +2182,11 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi {
sndMsgLsnrs.add(lsnr);
}
+ /** {@inheritDoc} */
+ @Override public void setInternalListener(IgniteDiscoverySpiInternalListener lsnr) {
+ this.internalLsnr = lsnr;
+ }
+
/**
* <strong>FOR TEST ONLY!!!</strong>
*/
@@ -2185,7 +2224,7 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi {
* <p>
* This method is intended for test purposes only.
*/
- protected void simulateNodeFailure() {
+ public void simulateNodeFailure() {
impl.simulateNodeFailure();
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java
index 01534f7..55fe4e6 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java
@@ -33,9 +33,9 @@ import java.util.UUID;
import org.apache.ignite.cache.CacheMetrics;
import org.apache.ignite.cluster.ClusterMetrics;
import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.configuration.IgniteConfiguration;
import org.apache.ignite.internal.ClusterMetricsSnapshot;
import org.apache.ignite.internal.IgniteNodeAttributes;
+import org.apache.ignite.internal.managers.discovery.IgniteClusterNode;
import org.apache.ignite.internal.util.lang.GridMetadataAwareAdapter;
import org.apache.ignite.internal.util.tostring.GridToStringExclude;
import org.apache.ignite.internal.util.tostring.GridToStringInclude;
@@ -58,7 +58,7 @@ import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_NODE_CONSISTE
* <strong>This class is not intended for public use</strong> and has been made
* <tt>public</tt> due to certain limitations of Java technology.
*/
-public class TcpDiscoveryNode extends GridMetadataAwareAdapter implements ClusterNode,
+public class TcpDiscoveryNode extends GridMetadataAwareAdapter implements IgniteClusterNode,
Comparable<TcpDiscoveryNode>, Externalizable {
/** */
private static final long serialVersionUID = 0L;
@@ -291,26 +291,14 @@ public class TcpDiscoveryNode extends GridMetadataAwareAdapter implements Cluste
return metrics;
}
- /**
- * Sets node metrics.
- *
- * @param metrics Node metrics.
- */
+ /** {@inheritDoc} */
public void setMetrics(ClusterMetrics metrics) {
assert metrics != null;
this.metrics = metrics;
}
- /**
- * Gets collections of cache metrics for this node. Note that node cache metrics are constantly updated
- * and provide up to date information about caches.
- * <p>
- * Cache metrics are updated with some delay which is directly related to metrics update
- * frequency. For example, by default the update will happen every {@code 2} seconds.
- *
- * @return Runtime metrics snapshots for this node.
- */
+ /** {@inheritDoc} */
public Map<Integer, CacheMetrics> cacheMetrics() {
if (metricsProvider != null) {
Map<Integer, CacheMetrics> cacheMetrics0 = metricsProvider.cacheMetrics();
@@ -323,11 +311,7 @@ public class TcpDiscoveryNode extends GridMetadataAwareAdapter implements Cluste
return cacheMetrics;
}
- /**
- * Sets node cache metrics.
- *
- * @param cacheMetrics Cache metrics.
- */
+ /** {@inheritDoc} */
public void setCacheMetrics(Map<Integer, CacheMetrics> cacheMetrics) {
this.cacheMetrics = cacheMetrics != null ? cacheMetrics : Collections.<Integer, CacheMetrics>emptyMap();
}
@@ -544,11 +528,7 @@ public class TcpDiscoveryNode extends GridMetadataAwareAdapter implements Cluste
return node;
}
- /**
- * Whether this node is cache client (see {@link IgniteConfiguration#isClientMode()}).
- *
- * @return {@code True if client}.
- */
+ /** {@inheritDoc} */
public boolean isCacheClient() {
if (!cacheCliInit) {
cacheCli = CU.clientNodeDirect(this);
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/resources/META-INF/classnames.properties
----------------------------------------------------------------------
diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties
index f0f143d..6dc3d85 100644
--- a/modules/core/src/main/resources/META-INF/classnames.properties
+++ b/modules/core/src/main/resources/META-INF/classnames.properties
@@ -832,6 +832,7 @@ org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPar
org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloader$1
org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloader$2
org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloaderAssignments
+org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgniteDhtDemandedPartitionsMap
org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgniteDhtPartitionCountersMap
org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgniteDhtPartitionCountersMap2
org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgniteDhtPartitionHistorySuppliersMap
@@ -1129,6 +1130,7 @@ org.apache.ignite.internal.processors.closure.GridClosureProcessor$T8
org.apache.ignite.internal.processors.closure.GridClosureProcessor$T9
org.apache.ignite.internal.processors.closure.GridClosureProcessor$TaskNoReduceAdapter
org.apache.ignite.internal.processors.closure.GridPeerDeployAwareTaskAdapter
+org.apache.ignite.internal.processors.cluster.ClusterNodeMetrics
org.apache.ignite.internal.processors.cluster.BaselineTopology
org.apache.ignite.internal.processors.cluster.BaselineTopologyHistory
org.apache.ignite.internal.processors.cluster.BaselineTopologyHistoryItem
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityFunctionExcludeNeighborsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityFunctionExcludeNeighborsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityFunctionExcludeNeighborsAbstractSelfTest.java
index 900d4f5..eee47c7 100644
--- a/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityFunctionExcludeNeighborsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityFunctionExcludeNeighborsAbstractSelfTest.java
@@ -32,7 +32,6 @@ import org.apache.ignite.internal.IgniteNodeAttributes;
import org.apache.ignite.internal.util.typedef.internal.U;
import org.apache.ignite.lang.IgniteProductVersion;
import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
-import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode;
import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
@@ -124,12 +123,9 @@ public abstract class AffinityFunctionExcludeNeighborsAbstractSelfTest extends G
Affinity<Object> aff = g.affinity(DEFAULT_CACHE_NAME);
- List<TcpDiscoveryNode> top = new ArrayList<>();
+ List<ClusterNode> top = new ArrayList<>(g.cluster().nodes());
- for (ClusterNode node : g.cluster().nodes())
- top.add((TcpDiscoveryNode) node);
-
- Collections.sort(top);
+ Collections.sort((List)top);
assertEquals(grids, top.size());
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/failure/FailureHandlerTriggeredTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/failure/FailureHandlerTriggeredTest.java b/modules/core/src/test/java/org/apache/ignite/failure/FailureHandlerTriggeredTest.java
index 4e4d75a..5eca7d6 100644
--- a/modules/core/src/test/java/org/apache/ignite/failure/FailureHandlerTriggeredTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/failure/FailureHandlerTriggeredTest.java
@@ -120,6 +120,10 @@ public class FailureHandlerTriggeredTest extends GridCommonAbstractTest {
@Override public boolean isMutable() {
return false;
}
+
+ @Override public boolean stopProcess() {
+ return false;
+ }
});
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/ClusterGroupHostsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/ClusterGroupHostsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/ClusterGroupHostsSelfTest.java
index 2328c84..141f4af 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/ClusterGroupHostsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/ClusterGroupHostsSelfTest.java
@@ -61,6 +61,9 @@ public class ClusterGroupHostsSelfTest extends GridCommonAbstractTest {
* @throws Exception If failed.
*/
public void testForHosts() throws Exception {
+ if (!tcpDiscovery())
+ return;
+
Ignite ignite = grid();
assertEquals(1, ignite.cluster().forHost("h_1").nodes().size());
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/ClusterGroupSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/ClusterGroupSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/ClusterGroupSelfTest.java
index 9df561a..99006d1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/ClusterGroupSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/ClusterGroupSelfTest.java
@@ -68,6 +68,8 @@ public class ClusterGroupSelfTest extends ClusterGroupAbstractTest {
if (i == 0)
ignite = g;
}
+
+ waitForTopology(NODES_CNT);
}
finally {
Ignition.setClientMode(false);
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/ClusterNodeMetricsUpdateTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/ClusterNodeMetricsUpdateTest.java b/modules/core/src/test/java/org/apache/ignite/internal/ClusterNodeMetricsUpdateTest.java
new file mode 100644
index 0000000..6e6b4a4
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/ClusterNodeMetricsUpdateTest.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;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import junit.framework.AssertionFailedError;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCompute;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.cluster.ClusterGroup;
+import org.apache.ignite.cluster.ClusterMetrics;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.lang.GridAbsPredicate;
+import org.apache.ignite.lang.IgniteCallable;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ *
+ */
+public class ClusterNodeMetricsUpdateTest extends GridCommonAbstractTest {
+ /** */
+ private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+ /** */
+ private boolean client;
+
+ /** {@inheritDoc} */
+ @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+ IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+ ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
+
+ cfg.setMetricsUpdateFrequency(500);
+
+ cfg.setClientMode(client);
+
+ return cfg;
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testMetrics() throws Exception {
+ int NODES = 6;
+
+ Ignite srv0 = startGridsMultiThreaded(NODES / 2);
+
+ client = true;
+
+ startGridsMultiThreaded(NODES / 2, NODES / 2);
+
+ Map<UUID, Integer> expJobs = new HashMap<>();
+
+ for (int i = 0; i < NODES; i++)
+ expJobs.put(nodeId(i), 0);
+
+ checkMetrics(NODES, expJobs);
+
+ for (int i = 0; i < NODES; i++) {
+ UUID nodeId = nodeId(i);
+
+ IgniteCompute c = srv0.compute(srv0.cluster().forNodeId(nodeId(i)));
+
+ c.call(new DummyCallable(null));
+
+ expJobs.put(nodeId, 1);
+ }
+ }
+
+ /**
+ * @param expNodes Expected nodes.
+ * @param expJobs Expected jobs number per node.
+ */
+ private void checkMetrics0(int expNodes, Map<UUID, Integer> expJobs) {
+ List<Ignite> nodes = Ignition.allGrids();
+
+ assertEquals(expNodes, nodes.size());
+ assertEquals(expNodes, expJobs.size());
+
+ int totalJobs = 0;
+
+ for (Integer c : expJobs.values())
+ totalJobs += c;
+
+ for (final Ignite ignite : nodes) {
+ ClusterMetrics m = ignite.cluster().metrics();
+
+ assertEquals(expNodes, m.getTotalNodes());
+ assertEquals(totalJobs, m.getTotalExecutedJobs());
+
+ for (Map.Entry<UUID, Integer> e : expJobs.entrySet()) {
+ UUID nodeId = e.getKey();
+
+ ClusterGroup g = ignite.cluster().forNodeId(nodeId);
+
+ ClusterMetrics nodeM = g.metrics();
+
+ assertEquals(e.getValue(), (Integer)nodeM.getTotalExecutedJobs());
+ }
+ }
+ }
+
+ /**
+ * @param expNodes Expected nodes.
+ * @param expJobs Expected jobs number per node.
+ * @throws Exception If failed.
+ */
+ private void checkMetrics(final int expNodes, final Map<UUID, Integer> expJobs) throws Exception {
+ GridTestUtils.waitForCondition(new GridAbsPredicate() {
+ @Override public boolean apply() {
+ try {
+ checkMetrics0(expNodes, expJobs);
+ }
+ catch (AssertionFailedError e) {
+ return false;
+ }
+
+ return true;
+ }
+ }, 5000);
+
+ checkMetrics0(expNodes, expJobs);
+ }
+
+ /**
+ *
+ */
+ private static class DummyCallable implements IgniteCallable<Object> {
+ /** */
+ private byte[] data;
+
+ /**
+ * @param data Data.
+ */
+ DummyCallable(byte[] data) {
+ this.data = data;
+ }
+
+ /** {@inheritDoc} */
+ @Override public Object call() throws Exception {
+ return data;
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override protected void afterTest() throws Exception {
+ super.afterTest();
+
+ stopAllGrids();
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/DiscoverySpiTestListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/DiscoverySpiTestListener.java b/modules/core/src/test/java/org/apache/ignite/internal/DiscoverySpiTestListener.java
new file mode 100644
index 0000000..46d9edc
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/DiscoverySpiTestListener.java
@@ -0,0 +1,162 @@
+/*
+ * 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;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
+import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpiInternalListener;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.discovery.DiscoverySpi;
+import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
+import org.apache.ignite.testframework.GridTestUtils;
+
+/**
+ * Test callback for discovery SPI.
+ * <p>
+ * Allows block/delay node join and custom event sending.
+ */
+public class DiscoverySpiTestListener implements IgniteDiscoverySpiInternalListener {
+ /** */
+ private volatile CountDownLatch joinLatch;
+
+ /** */
+ private Set<Class<?>> blockCustomEvtCls;
+
+ /** */
+ private final Object mux = new Object();
+
+ /** */
+ private List<DiscoverySpiCustomMessage> blockedMsgs = new ArrayList<>();
+
+ /** */
+ private volatile DiscoverySpi spi;
+
+ /** */
+ private volatile IgniteLogger log;
+
+ /**
+ *
+ */
+ public void startBlockJoin() {
+ joinLatch = new CountDownLatch(1);
+ }
+
+ /**
+ *
+ */
+ public void stopBlockJoin() {
+ joinLatch.countDown();
+ }
+
+ /** {@inheritDoc} */
+ @Override public void beforeJoin(ClusterNode locNode, IgniteLogger log) {
+ try {
+ CountDownLatch writeLatch0 = joinLatch;
+
+ if (writeLatch0 != null) {
+ log.info("Block join");
+
+ U.await(writeLatch0);
+ }
+ }
+ catch (Exception e) {
+ throw new IgniteException(e);
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean beforeSendCustomEvent(DiscoverySpi spi, IgniteLogger log, DiscoverySpiCustomMessage msg) {
+ this.spi = spi;
+ this.log = log;
+
+ synchronized (mux) {
+ if (blockCustomEvtCls != null) {
+ DiscoveryCustomMessage msg0 = GridTestUtils.getFieldValue(msg, "delegate");
+
+ if (blockCustomEvtCls.contains(msg0.getClass())) {
+ log.info("Block custom message: " + msg0);
+
+ blockedMsgs.add(msg);
+
+ mux.notifyAll();
+
+ return false;
+ }
+ }
+ }
+
+ return true;
+ }
+
+ /**
+ * @param blockCustomEvtCls Event class to block.
+ */
+ public void blockCustomEvent(Class<?> cls0, Class<?> ... blockCustomEvtCls) {
+ synchronized (mux) {
+ assert blockedMsgs.isEmpty() : blockedMsgs;
+
+ this.blockCustomEvtCls = new HashSet<>();
+
+ this.blockCustomEvtCls.add(cls0);
+
+ Collections.addAll(this.blockCustomEvtCls, blockCustomEvtCls);
+ }
+ }
+
+ /**
+ * @throws InterruptedException If interrupted.
+ */
+ public void waitCustomEvent() throws InterruptedException {
+ synchronized (mux) {
+ while (blockedMsgs.isEmpty())
+ mux.wait();
+ }
+ }
+
+ /**
+ *
+ */
+ public void stopBlockCustomEvents() {
+ if (spi == null)
+ return;
+
+ List<DiscoverySpiCustomMessage> msgs;
+
+ synchronized (this) {
+ msgs = new ArrayList<>(blockedMsgs);
+
+ blockCustomEvtCls = null;
+
+ blockedMsgs.clear();
+ }
+
+ for (DiscoverySpiCustomMessage msg : msgs) {
+ log.info("Resend blocked message: " + msg);
+
+ spi.sendCustomEvent(msg);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoverySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoverySelfTest.java
index e6b678b..883d677 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoverySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoverySelfTest.java
@@ -49,6 +49,7 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
import org.jetbrains.annotations.Nullable;
import static java.util.concurrent.TimeUnit.MINUTES;
+import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
import static org.apache.ignite.events.EventType.EVT_NODE_JOINED;
import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
import static org.apache.ignite.lang.IgniteProductVersion.fromString;
@@ -158,10 +159,10 @@ public class GridDiscoverySelfTest extends GridCommonAbstractTest {
final AtomicInteger cnt = new AtomicInteger();
- /** Joined nodes counter. */
+ // Joined nodes counter.
final CountDownLatch joinedCnt = new CountDownLatch(NODES_CNT);
- /** Left nodes counter. */
+ // Left nodes counter.
final CountDownLatch leftCnt = new CountDownLatch(NODES_CNT);
IgnitePredicate<Event> lsnr = new IgnitePredicate<Event>() {
@@ -171,7 +172,7 @@ public class GridDiscoverySelfTest extends GridCommonAbstractTest {
joinedCnt.countDown();
}
- else if (EVT_NODE_LEFT == evt.type()) {
+ else if (EVT_NODE_LEFT == evt.type() || EVT_NODE_FAILED == evt.type()) {
int i = cnt.decrementAndGet();
assert i >= 0;
@@ -185,7 +186,10 @@ public class GridDiscoverySelfTest extends GridCommonAbstractTest {
}
};
- ignite.events().localListen(lsnr, EVT_NODE_LEFT, EVT_NODE_JOINED);
+ int[] evts = tcpDiscovery() ? new int[]{EVT_NODE_LEFT, EVT_NODE_JOINED} :
+ new int[]{EVT_NODE_LEFT, EVT_NODE_FAILED, EVT_NODE_JOINED};
+
+ ignite.events().localListen(lsnr, evts);
try {
for (int i = 0; i < NODES_CNT; i++)
@@ -242,6 +246,8 @@ public class GridDiscoverySelfTest extends GridCommonAbstractTest {
for (int i = 0; i < NODES_CNT; i++)
stopGrid(i);
+ waitForTopology(1);
+
final long topVer = discoMgr.topologyVersion();
assert topVer == topVer0 + NODES_CNT * 2 : "Unexpected topology version: " + topVer;
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/GridJobMasterLeaveAwareSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridJobMasterLeaveAwareSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridJobMasterLeaveAwareSelfTest.java
index cd6b2c0..a8be541 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridJobMasterLeaveAwareSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridJobMasterLeaveAwareSelfTest.java
@@ -259,6 +259,8 @@ public class GridJobMasterLeaveAwareSelfTest extends GridCommonAbstractTest {
// Now we stop master grid.
stopGrid(lastGridIdx, true);
+ waitForTopology(GRID_CNT - 1);
+
// Release communication SPI wait latches. As master node is stopped, job worker will receive and exception.
for (int i = 0; i < lastGridIdx; i++)
((CommunicationSpi)grid(i).configuration().getCommunicationSpi()).releaseWaitLatch();
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/GridJobStealingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridJobStealingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridJobStealingSelfTest.java
index f3a19aa..6824d51 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridJobStealingSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridJobStealingSelfTest.java
@@ -187,6 +187,8 @@ public class GridJobStealingSelfTest extends GridCommonAbstractTest {
public void testProjectionPredicateInternalStealing() throws Exception {
final Ignite ignite3 = startGrid(3);
+ waitForTopology(3);
+
final UUID node1 = ignite1.cluster().localNode().id();
final UUID node3 = ignite3.cluster().localNode().id();
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/GridSameVmStartupSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridSameVmStartupSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridSameVmStartupSelfTest.java
index 66e9cf4..a04c38e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridSameVmStartupSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridSameVmStartupSelfTest.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal;
import java.util.Collection;
import java.util.UUID;
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.events.DiscoveryEvent;
@@ -75,8 +76,10 @@ public class GridSameVmStartupSelfTest extends GridCommonAbstractTest {
ignite2.events().localListen(new IgnitePredicate<Event>() {
@Override public boolean apply(Event evt) {
- assert evt.type() != EVT_NODE_FAILED :
- "Node1 did not exit gracefully.";
+ boolean tcpDiscovery = tcpDiscovery();
+
+ if (tcpDiscovery)
+ assert evt.type() != EVT_NODE_FAILED : "Node1 did not exit gracefully.";
if (evt instanceof DiscoveryEvent) {
// Local node can send METRICS_UPDATED event.
@@ -86,8 +89,14 @@ public class GridSameVmStartupSelfTest extends GridCommonAbstractTest {
((DiscoveryEvent) evt).eventNode().id() + ", expected=" + grid1LocNodeId +
", type=" + evt.type() + ']';
- if (evt.type() == EVT_NODE_LEFT)
- latch.countDown();
+ if (tcpDiscovery) {
+ if (evt.type() == EVT_NODE_LEFT)
+ latch.countDown();
+ }
+ else {
+ if (evt.type() == EVT_NODE_LEFT || evt.type() == EVT_NODE_FAILED)
+ latch.countDown();
+ }
}
return true;
@@ -96,7 +105,7 @@ public class GridSameVmStartupSelfTest extends GridCommonAbstractTest {
stopGrid(1);
- latch.await();
+ assertTrue(latch.await(10, TimeUnit.SECONDS));
Collection<ClusterNode> top2 = ignite2.cluster().forRemotes().nodes();
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/GridSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridSelfTest.java
index 7e368cb..f71ffb0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridSelfTest.java
@@ -45,6 +45,8 @@ public class GridSelfTest extends ClusterGroupAbstractTest {
for (int i = 0; i < NODES_CNT; i++)
startGrid(i);
+
+ waitForTopology(NODES_CNT);
}
/** {@inheritDoc} */
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAbstractTest.java
index fa9cc35..e68ea13 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAbstractTest.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal;
import java.io.IOException;
import java.io.OutputStream;
import java.net.Socket;
+import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
@@ -38,6 +39,7 @@ import org.apache.ignite.cluster.ClusterNode;
import org.apache.ignite.configuration.IgniteConfiguration;
import org.apache.ignite.events.Event;
import org.apache.ignite.internal.managers.communication.GridIoMessage;
+import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi;
import org.apache.ignite.internal.util.typedef.G;
import org.apache.ignite.internal.util.typedef.internal.U;
import org.apache.ignite.lang.IgniteFuture;
@@ -143,6 +145,14 @@ public abstract class IgniteClientReconnectAbstractTest extends GridCommonAbstra
/**
* @param ignite Node.
+ * @return Discovery SPI.
+ */
+ protected static IgniteDiscoverySpi spi0(Ignite ignite) {
+ return ((IgniteDiscoverySpi)ignite.configuration().getDiscoverySpi());
+ }
+
+ /**
+ * @param ignite Node.
* @return Communication SPI.
*/
protected BlockTcpCommunicationSpi commSpi(Ignite ignite) {
@@ -185,16 +195,28 @@ public abstract class IgniteClientReconnectAbstractTest extends GridCommonAbstra
* @return Server node client connected to.
*/
protected Ignite clientRouter(Ignite client) {
- TcpDiscoveryNode node = (TcpDiscoveryNode)client.cluster().localNode();
+ if (tcpDiscovery()) {
+ TcpDiscoveryNode node = (TcpDiscoveryNode)client.cluster().localNode();
+
+ assertTrue(node.isClient());
+ assertNotNull(node.clientRouterNodeId());
- assertTrue(node.isClient());
- assertNotNull(node.clientRouterNodeId());
+ Ignite srv = G.ignite(node.clientRouterNodeId());
- Ignite srv = G.ignite(node.clientRouterNodeId());
+ assertNotNull(srv);
+
+ return srv;
+ }
+ else {
+ for (Ignite node : G.allGrids()) {
+ if (!node.cluster().localNode().isClient())
+ return node;
+ }
- assertNotNull(srv);
+ fail();
- return srv;
+ return null;
+ }
}
/**
@@ -251,15 +273,24 @@ public abstract class IgniteClientReconnectAbstractTest extends GridCommonAbstra
List<Ignite> clients, Ignite srv,
@Nullable Runnable disconnectedC)
throws Exception {
- final TestTcpDiscoverySpi srvSpi = spi(srv);
+ final IgniteDiscoverySpi srvSpi = spi0(srv);
final CountDownLatch disconnectLatch = new CountDownLatch(clients.size());
final CountDownLatch reconnectLatch = new CountDownLatch(clients.size());
log.info("Block reconnect.");
- for (Ignite client : clients)
- spi(client).writeLatch = new CountDownLatch(1);
+ List<DiscoverySpiTestListener> blockLsnrs = new ArrayList<>();
+
+ for (Ignite client : clients) {
+ DiscoverySpiTestListener lsnr = new DiscoverySpiTestListener();
+
+ lsnr.startBlockJoin();
+
+ blockLsnrs.add(lsnr);
+
+ spi0(client).setInternalListener(lsnr);
+ }
IgnitePredicate<Event> p = new IgnitePredicate<Event>() {
@Override public boolean apply(Event evt) {
@@ -291,8 +322,8 @@ public abstract class IgniteClientReconnectAbstractTest extends GridCommonAbstra
log.info("Allow reconnect.");
- for (Ignite client : clients)
- spi(client).writeLatch.countDown();
+ for (DiscoverySpiTestListener blockLsnr : blockLsnrs)
+ blockLsnr.stopBlockJoin();
waitReconnectEvent(log, reconnectLatch);
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectApiExceptionTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectApiExceptionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectApiExceptionTest.java
index 06bde99..43da2d1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectApiExceptionTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectApiExceptionTest.java
@@ -44,6 +44,7 @@ import org.apache.ignite.configuration.CacheConfiguration;
import org.apache.ignite.configuration.CollectionConfiguration;
import org.apache.ignite.configuration.IgniteConfiguration;
import org.apache.ignite.events.Event;
+import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi;
import org.apache.ignite.internal.util.typedef.C1;
import org.apache.ignite.internal.util.typedef.T2;
import org.apache.ignite.internal.util.typedef.internal.U;
@@ -51,6 +52,7 @@ import org.apache.ignite.lang.IgniteBiPredicate;
import org.apache.ignite.lang.IgniteCallable;
import org.apache.ignite.lang.IgnitePredicate;
import org.apache.ignite.resources.IgniteInstanceResource;
+import org.apache.ignite.spi.discovery.DiscoverySpi;
import org.apache.ignite.testframework.GridTestUtils;
import static java.util.concurrent.TimeUnit.SECONDS;
@@ -99,7 +101,7 @@ public class IgniteClientReconnectApiExceptionTest extends IgniteClientReconnect
* @throws Exception If failed.
*/
@SuppressWarnings("unchecked")
- public void dataStructureOperationsTest() throws Exception {
+ private void dataStructureOperationsTest() throws Exception {
clientMode = true;
final Ignite client = startGrid(serverCount());
@@ -219,7 +221,7 @@ public class IgniteClientReconnectApiExceptionTest extends IgniteClientReconnect
* @throws Exception If failed.
*/
@SuppressWarnings("unchecked")
- public void cacheOperationsTest() throws Exception {
+ private void cacheOperationsTest() throws Exception {
clientMode = true;
final Ignite client = startGrid(serverCount());
@@ -537,7 +539,7 @@ public class IgniteClientReconnectApiExceptionTest extends IgniteClientReconnect
* @throws Exception If failed.
*/
@SuppressWarnings("unchecked")
- public void igniteOperationsTest() throws Exception {
+ private void igniteOperationsTest() throws Exception {
clientMode = true;
final Ignite client = startGrid(serverCount());
@@ -775,11 +777,11 @@ public class IgniteClientReconnectApiExceptionTest extends IgniteClientReconnect
throws Exception {
assertNotNull(client.cache(DEFAULT_CACHE_NAME));
- final TestTcpDiscoverySpi clientSpi = spi(client);
+ final IgniteDiscoverySpi clientSpi = spi0(client);
Ignite srv = clientRouter(client);
- TestTcpDiscoverySpi srvSpi = spi(srv);
+ DiscoverySpi srvSpi = spi0(srv);
final CountDownLatch disconnectLatch = new CountDownLatch(1);
@@ -787,7 +789,10 @@ public class IgniteClientReconnectApiExceptionTest extends IgniteClientReconnect
log.info("Block reconnect.");
- clientSpi.writeLatch = new CountDownLatch(1);
+ DiscoverySpiTestListener lsnr = new DiscoverySpiTestListener();
+
+ clientSpi.setInternalListener(lsnr);
+ lsnr.startBlockJoin();
final List<IgniteInternalFuture> futs = new ArrayList<>();
@@ -832,7 +837,7 @@ public class IgniteClientReconnectApiExceptionTest extends IgniteClientReconnect
log.info("Allow reconnect.");
- clientSpi.writeLatch.countDown();
+ lsnr.stopBlockJoin();
waitReconnectEvent(reconnectLatch);
@@ -857,7 +862,7 @@ public class IgniteClientReconnectApiExceptionTest extends IgniteClientReconnect
}
}
finally {
- clientSpi.writeLatch.countDown();
+ lsnr.stopBlockJoin();
for (IgniteInternalFuture fut : futs)
fut.cancel();
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAtomicsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAtomicsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAtomicsTest.java
index 00daf5f..d1e3ade 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAtomicsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAtomicsTest.java
@@ -111,7 +111,7 @@ public class IgniteClientReconnectAtomicsTest extends IgniteClientReconnectAbstr
assertTrue(client.cluster().localNode().isClient());
- Ignite srv = clientRouter(client);
+ Ignite srv = ignite(0);
IgniteAtomicSequence clientAtomicSeq = client.atomicSequence("atomicSeq", 0, true);
@@ -144,7 +144,7 @@ public class IgniteClientReconnectAtomicsTest extends IgniteClientReconnectAbstr
assertTrue(client.cluster().localNode().isClient());
- Ignite srv = clientRouter(client);
+ Ignite srv = ignite(0);
final IgniteAtomicSequence clientAtomicSeq = client.atomicSequence("atomicSeqRmv", 0, true);
@@ -192,7 +192,7 @@ public class IgniteClientReconnectAtomicsTest extends IgniteClientReconnectAbstr
assertTrue(client.cluster().localNode().isClient());
- Ignite srv = clientRouter(client);
+ Ignite srv = ignite(0);
BlockTcpCommunicationSpi commSpi = commSpi(srv);
@@ -253,7 +253,7 @@ public class IgniteClientReconnectAtomicsTest extends IgniteClientReconnectAbstr
assertTrue(client.cluster().localNode().isClient());
- Ignite srv = clientRouter(client);
+ Ignite srv = ignite(0);
IgniteAtomicReference<String> clientAtomicRef = client.atomicReference("atomicRef", "1st value", true);
@@ -294,7 +294,7 @@ public class IgniteClientReconnectAtomicsTest extends IgniteClientReconnectAbstr
assertTrue(client.cluster().localNode().isClient());
- Ignite srv = clientRouter(client);
+ Ignite srv = ignite(0);
final IgniteAtomicReference<String> clientAtomicRef =
client.atomicReference("atomicRefRemoved", "1st value", true);
@@ -347,7 +347,7 @@ public class IgniteClientReconnectAtomicsTest extends IgniteClientReconnectAbstr
assertTrue(client.cluster().localNode().isClient());
- Ignite srv = clientRouter(client);
+ Ignite srv = ignite(0);
final IgniteAtomicReference<String> clientAtomicRef =
client.atomicReference("atomicRefInProg", "1st value", true);
@@ -414,7 +414,7 @@ public class IgniteClientReconnectAtomicsTest extends IgniteClientReconnectAbstr
assertTrue(client.cluster().localNode().isClient());
- Ignite srv = clientRouter(client);
+ Ignite srv = ignite(0);
IgniteAtomicStamped clientAtomicStamped = client.atomicStamped("atomicStamped", 0, 0, true);
@@ -455,7 +455,7 @@ public class IgniteClientReconnectAtomicsTest extends IgniteClientReconnectAbstr
assertTrue(client.cluster().localNode().isClient());
- Ignite srv = clientRouter(client);
+ Ignite srv = ignite(0);
final IgniteAtomicStamped clientAtomicStamped = client.atomicStamped("atomicStampedRemoved", 0, 0, true);
@@ -506,7 +506,7 @@ public class IgniteClientReconnectAtomicsTest extends IgniteClientReconnectAbstr
assertTrue(client.cluster().localNode().isClient());
- Ignite srv = clientRouter(client);
+ Ignite srv = ignite(0);
final IgniteAtomicStamped clientAtomicStamped = client.atomicStamped("atomicStampedInProgress", 0, 0, true);
@@ -574,7 +574,7 @@ public class IgniteClientReconnectAtomicsTest extends IgniteClientReconnectAbstr
assertTrue(client.cluster().localNode().isClient());
- Ignite srv = clientRouter(client);
+ Ignite srv = ignite(0);
IgniteAtomicLong clientAtomicLong = client.atomicLong("atomicLong", 0, true);
@@ -605,7 +605,7 @@ public class IgniteClientReconnectAtomicsTest extends IgniteClientReconnectAbstr
assertTrue(client.cluster().localNode().isClient());
- Ignite srv = clientRouter(client);
+ Ignite srv = ignite(0);
final IgniteAtomicLong clientAtomicLong = client.atomicLong("atomicLongRmv", 0, true);
@@ -646,7 +646,7 @@ public class IgniteClientReconnectAtomicsTest extends IgniteClientReconnectAbstr
assertTrue(client.cluster().localNode().isClient());
- Ignite srv = clientRouter(client);
+ Ignite srv = ignite(0);
BlockTcpCommunicationSpi commSpi = commSpi(srv);
@@ -701,7 +701,7 @@ public class IgniteClientReconnectAtomicsTest extends IgniteClientReconnectAbstr
assertTrue(client.cluster().localNode().isClient());
- Ignite srv = clientRouter(client);
+ Ignite srv = ignite(0);
IgniteCountDownLatch clientLatch = client.countDownLatch("latch1", 3, false, true);
@@ -742,7 +742,7 @@ public class IgniteClientReconnectAtomicsTest extends IgniteClientReconnectAbstr
assertTrue(client.cluster().localNode().isClient());
- Ignite srv = clientRouter(client);
+ Ignite srv = ignite(0);
IgniteSemaphore clientSemaphore = client.semaphore("semaphore1", 3, false, true);
@@ -789,7 +789,7 @@ public class IgniteClientReconnectAtomicsTest extends IgniteClientReconnectAbstr
assertTrue(client.cluster().localNode().isClient());
- Ignite srv = clientRouter(client);
+ Ignite srv = ignite(0);
IgniteLock clientLock = client.reentrantLock("lock1", true, fair, true);
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java
index 518e674..3cb82e0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java
@@ -49,6 +49,7 @@ import org.apache.ignite.events.DiscoveryEvent;
import org.apache.ignite.events.Event;
import org.apache.ignite.internal.managers.communication.GridIoMessage;
import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager;
+import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi;
import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateResponse;
import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsFullMessage;
import org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetResponse;
@@ -67,6 +68,7 @@ import org.apache.ignite.plugin.extensions.communication.Message;
import org.apache.ignite.resources.LoggerResource;
import org.apache.ignite.spi.IgniteSpiException;
import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+import org.apache.ignite.spi.discovery.DiscoverySpi;
import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
import org.apache.ignite.testframework.GridTestUtils;
import org.apache.ignite.transactions.Transaction;
@@ -155,11 +157,11 @@ public class IgniteClientReconnectCacheTest extends IgniteClientReconnectAbstrac
IgniteEx client = startGrid(SRV_CNT);
- final TestTcpDiscoverySpi clientSpi = spi(client);
+ final IgniteDiscoverySpi clientSpi = spi0(client);
- Ignite srv = clientRouter(client);
+ Ignite srv = ignite(0);
- TestTcpDiscoverySpi srvSpi = spi(srv);
+ DiscoverySpi srvSpi = ignite(0).configuration().getDiscoverySpi();
final IgniteCache<Object, Object> cache = client.getOrCreateCache(new CacheConfiguration<>(DEFAULT_CACHE_NAME));
@@ -188,7 +190,11 @@ public class IgniteClientReconnectCacheTest extends IgniteClientReconnectAbstrac
log.info("Block reconnect.");
- clientSpi.writeLatch = new CountDownLatch(1);
+ DiscoverySpiTestListener lsnr = new DiscoverySpiTestListener();
+
+ clientSpi.setInternalListener(lsnr);
+
+ lsnr.startBlockJoin();
final AtomicReference<IgniteInternalFuture> blockPutRef = new AtomicReference<>();
@@ -254,7 +260,7 @@ public class IgniteClientReconnectCacheTest extends IgniteClientReconnectAbstrac
log.info("Allow reconnect.");
- clientSpi.writeLatch.countDown();
+ lsnr.stopBlockJoin();
assertTrue(reconnectLatch.await(5000, MILLISECONDS));
@@ -319,7 +325,7 @@ public class IgniteClientReconnectCacheTest extends IgniteClientReconnectAbstrac
IgniteEx client = startGrid(SRV_CNT);
- Ignite srv = clientRouter(client);
+ Ignite srv = ignite(0);
CacheConfiguration<Object, Object> ccfg = new CacheConfiguration<>(DEFAULT_CACHE_NAME);
@@ -412,17 +418,21 @@ public class IgniteClientReconnectCacheTest extends IgniteClientReconnectAbstrac
final TransactionConcurrency txConcurrency,
final IgniteCache<Object, Object> cache)
throws Exception {
- Ignite srv = clientRouter(client);
+ Ignite srv = ignite(0);
- final TestTcpDiscoverySpi clientSpi = spi(client);
- final TestTcpDiscoverySpi srvSpi = spi(srv);
+ final IgniteDiscoverySpi clientSpi = spi0(client);
+ final DiscoverySpi srvSpi = spi0(srv);
final CountDownLatch disconnectLatch = new CountDownLatch(1);
final CountDownLatch reconnectLatch = new CountDownLatch(1);
log.info("Block reconnect.");
- clientSpi.writeLatch = new CountDownLatch(1);
+ DiscoverySpiTestListener lsnr = new DiscoverySpiTestListener();
+
+ clientSpi.setInternalListener(lsnr);
+
+ lsnr.startBlockJoin();
client.events().localListen(new IgnitePredicate<Event>() {
@Override public boolean apply(Event evt) {
@@ -530,7 +540,7 @@ public class IgniteClientReconnectCacheTest extends IgniteClientReconnectAbstrac
assertTrue(putFailed.await(5000, MILLISECONDS));
- clientSpi.writeLatch.countDown();
+ lsnr.stopBlockJoin();
waitReconnectEvent(reconnectLatch);
@@ -604,9 +614,9 @@ public class IgniteClientReconnectCacheTest extends IgniteClientReconnectAbstrac
IgniteEx client = startGrid(SRV_CNT);
- Ignite srv = clientRouter(client);
+ Ignite srv = ignite(0);
- TestTcpDiscoverySpi srvSpi = spi(srv);
+ DiscoverySpi srvSpi = spi0(srv);
TestCommunicationSpi coordCommSpi = (TestCommunicationSpi)grid(0).configuration().getCommunicationSpi();
@@ -691,7 +701,7 @@ public class IgniteClientReconnectCacheTest extends IgniteClientReconnectAbstrac
IgniteInternalFuture<Boolean> fut = GridTestUtils.runAsync(new Callable<Boolean>() {
@Override public Boolean call() throws Exception {
try {
- Ignition.start(optimize(getConfiguration(getTestIgniteInstanceName(SRV_CNT))));
+ startGrid(optimize(getConfiguration(getTestIgniteInstanceName(SRV_CNT))));
// Commented due to IGNITE-4473, because
// IgniteClientDisconnectedException won't
@@ -722,7 +732,7 @@ public class IgniteClientReconnectCacheTest extends IgniteClientReconnectAbstrac
}
});
- TestTcpDiscoverySpi srvSpi = spi(srv);
+ DiscoverySpi srvSpi = spi0(srv);
try {
if (!joinLatch.await(10_000, MILLISECONDS)) {
@@ -1256,30 +1266,35 @@ public class IgniteClientReconnectCacheTest extends IgniteClientReconnectAbstrac
*
*/
static class TestClass1 implements Serializable {
+ // No-op.
}
/**
*
*/
static class TestClass2 implements Serializable {
+ // No-op.
}
/**
*
*/
static class TestClass3 implements Serializable {
+ // No-op.
}
/**
*
*/
static class TestClass4 implements Serializable {
+ // No-op.
}
/**
*
*/
static class TestClass5 implements Serializable {
+ // No-op.
}
/**
@@ -1294,11 +1309,11 @@ public class IgniteClientReconnectCacheTest extends IgniteClientReconnectAbstrac
Class<?> msgToBlock,
final IgniteInClosure<IgniteCache<Object, Object>> c)
throws Exception {
- Ignite srv = clientRouter(client);
+ Ignite srv = ignite(0);
final UUID id = client.localNode().id();
- TestTcpDiscoverySpi srvSpi = spi(srv);
+ DiscoverySpi srvSpi = spi0(srv);
final IgniteCache<Object, Object> cache = client.getOrCreateCache(ccfg);
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCollectionsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCollectionsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCollectionsTest.java
index 3f0e33d..5be59b0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCollectionsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCollectionsTest.java
@@ -180,7 +180,7 @@ public class IgniteClientReconnectCollectionsTest extends IgniteClientReconnectA
private void serverNodeReconnect(CollectionConfiguration colCfg) throws Exception {
final Ignite client = grid(serverCount());
- final Ignite srv = clientRouter(client);
+ final Ignite srv = ignite(0);
assertNotNull(srv.queue("q", 0, colCfg));
assertNotNull(srv.set("s", colCfg));
@@ -201,7 +201,7 @@ public class IgniteClientReconnectCollectionsTest extends IgniteClientReconnectA
assertTrue(client.cluster().localNode().isClient());
- Ignite srv = clientRouter(client);
+ Ignite srv = ignite(0);
final String setName = "set-" + colCfg.getAtomicityMode();
@@ -235,7 +235,7 @@ public class IgniteClientReconnectCollectionsTest extends IgniteClientReconnectA
assertTrue(client.cluster().localNode().isClient());
- final Ignite srv = clientRouter(client);
+ final Ignite srv = ignite(0);
final String setName = "set-rm-" + colCfg.getAtomicityMode();
@@ -281,7 +281,7 @@ public class IgniteClientReconnectCollectionsTest extends IgniteClientReconnectA
assertTrue(client.cluster().localNode().isClient());
- final Ignite srv = clientRouter(client);
+ final Ignite srv = ignite(0);
final String setName = "set-in-progress-" + colCfg.getAtomicityMode();
@@ -347,7 +347,7 @@ public class IgniteClientReconnectCollectionsTest extends IgniteClientReconnectA
assertTrue(client.cluster().localNode().isClient());
- Ignite srv = clientRouter(client);
+ Ignite srv = ignite(0);
final String setName = "queue-" + colCfg.getAtomicityMode();
@@ -379,7 +379,7 @@ public class IgniteClientReconnectCollectionsTest extends IgniteClientReconnectA
assertTrue(client.cluster().localNode().isClient());
- Ignite srv = clientRouter(client);
+ Ignite srv = ignite(0);
final String setName = "queue-rmv" + colCfg.getAtomicityMode();
@@ -423,7 +423,7 @@ public class IgniteClientReconnectCollectionsTest extends IgniteClientReconnectA
assertTrue(client.cluster().localNode().isClient());
- Ignite srv = clientRouter(client);
+ Ignite srv = ignite(0);
final String setName = "queue-rmv" + colCfg.getAtomicityMode();
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectComputeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectComputeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectComputeTest.java
index cce0c7e..57d3188 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectComputeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectComputeTest.java
@@ -49,7 +49,7 @@ public class IgniteClientReconnectComputeTest extends IgniteClientReconnectAbstr
assertTrue(client.cluster().localNode().isClient());
- Ignite srv = clientRouter(client);
+ Ignite srv = ignite(0);
IgniteCache<Integer, Integer> cache = client.getOrCreateCache("test-cache");
@@ -103,7 +103,7 @@ public class IgniteClientReconnectComputeTest extends IgniteClientReconnectAbstr
assertTrue(client.cluster().localNode().isClient());
- Ignite srv = clientRouter(client);
+ Ignite srv = ignite(0);
BlockTcpCommunicationSpi commSpi = commSpi(srv);
@@ -152,7 +152,7 @@ public class IgniteClientReconnectComputeTest extends IgniteClientReconnectAbstr
assertTrue(client.cluster().localNode().isClient());
- Ignite srv = clientRouter(client);
+ Ignite srv = ignite(0);
BlockTcpCommunicationSpi commSpi = commSpi(srv);
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectContinuousProcessorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectContinuousProcessorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectContinuousProcessorTest.java
index ca0d889..d68fc1c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectContinuousProcessorTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectContinuousProcessorTest.java
@@ -28,6 +28,7 @@ import org.apache.ignite.cache.query.QueryCursor;
import org.apache.ignite.configuration.CacheConfiguration;
import org.apache.ignite.events.Event;
import org.apache.ignite.events.EventType;
+import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi;
import org.apache.ignite.internal.util.typedef.P2;
import org.apache.ignite.lang.IgnitePredicate;
import org.apache.ignite.lang.IgniteRunnable;
@@ -61,9 +62,9 @@ public class IgniteClientReconnectContinuousProcessorTest extends IgniteClientRe
assertTrue(client.cluster().localNode().isClient());
- Ignite srv = clientRouter(client);
+ Ignite srv = ignite(0);
- TestTcpDiscoverySpi srvSpi = spi(srv);
+ IgniteDiscoverySpi srvSpi = spi0(srv);
EventListener lsnr = new EventListener();
@@ -133,9 +134,9 @@ public class IgniteClientReconnectContinuousProcessorTest extends IgniteClientRe
assertTrue(client.cluster().localNode().isClient());
- Ignite srv = clientRouter(client);
+ Ignite srv = ignite(0);
- TestTcpDiscoverySpi srvSpi = spi(srv);
+ IgniteDiscoverySpi srvSpi = spi0(srv);
final String topic = "testTopic";
@@ -309,9 +310,9 @@ public class IgniteClientReconnectContinuousProcessorTest extends IgniteClientRe
CacheEventListener lsnr)
throws Exception
{
- Ignite srv = clientRouter(client);
+ Ignite srv = ignite(0);
- TestTcpDiscoverySpi srvSpi = spi(srv);
+ IgniteDiscoverySpi srvSpi = spi0(srv);
final CountDownLatch reconnectLatch = new CountDownLatch(1);
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectDiscoveryStateTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectDiscoveryStateTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectDiscoveryStateTest.java
index c071ee2..6e77742 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectDiscoveryStateTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectDiscoveryStateTest.java
@@ -27,6 +27,7 @@ import org.apache.ignite.cluster.ClusterNode;
import org.apache.ignite.events.Event;
import org.apache.ignite.lang.IgniteFuture;
import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.spi.discovery.DiscoverySpi;
import static org.apache.ignite.events.EventType.EVT_CLIENT_NODE_DISCONNECTED;
import static org.apache.ignite.events.EventType.EVT_CLIENT_NODE_RECONNECTED;
@@ -64,20 +65,23 @@ public class IgniteClientReconnectDiscoveryStateTest extends IgniteClientReconne
nodeCnt.put(1, 1);
nodeCnt.put(2, 2);
nodeCnt.put(3, 3);
- nodeCnt.put(4, 4);
- for (Map.Entry<Integer, Integer> e : nodeCnt.entrySet()) {
- Collection<ClusterNode> nodes = cluster.topology(e.getKey());
+ if (tcpDiscovery()) {
+ nodeCnt.put(4, 4);
- assertNotNull("No nodes for topology: " + e.getKey(), nodes);
- assertEquals((int)e.getValue(), nodes.size());
+ for (Map.Entry<Integer, Integer> e : nodeCnt.entrySet()) {
+ Collection<ClusterNode> nodes = cluster.topology(e.getKey());
+
+ assertNotNull("No nodes for topology: " + e.getKey(), nodes);
+ assertEquals((int)e.getValue(), nodes.size());
+ }
}
ClusterNode locNode = cluster.localNode();
assertEquals(topVer, locNode.order());
- TestTcpDiscoverySpi srvSpi = spi(clientRouter(client));
+ DiscoverySpi srvSpi = ignite(0).configuration().getDiscoverySpi();
final CountDownLatch reconnectLatch = new CountDownLatch(1);
@@ -112,7 +116,11 @@ public class IgniteClientReconnectDiscoveryStateTest extends IgniteClientReconne
assertEquals(topVer, locNode.order());
assertEquals(topVer, cluster.topologyVersion());
- nodeCnt.put(5, 3);
+ if (tcpDiscovery())
+ nodeCnt.put(5, 3);
+ else
+ nodeCnt.clear();
+
nodeCnt.put(6, 4);
for (Map.Entry<Integer, Integer> e : nodeCnt.entrySet()) {
[03/12] ignite git commit: IGNITE-7222 Added ZooKeeper discovery SPI
Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/test/java/org/apache/ZookeeperNodeStart.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/test/java/org/apache/ZookeeperNodeStart.java b/modules/zookeeper/src/test/java/org/apache/ZookeeperNodeStart.java
new file mode 100644
index 0000000..ef4d5f4
--- /dev/null
+++ b/modules/zookeeper/src/test/java/org/apache/ZookeeperNodeStart.java
@@ -0,0 +1,46 @@
+/*
+ * 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;
+
+import org.apache.ignite.Ignition;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.spi.discovery.zk.ZookeeperDiscoverySpi;
+
+/**
+ *
+ */
+public class ZookeeperNodeStart {
+ public static void main(String[] args) throws Exception {
+ try {
+ IgniteConfiguration cfg = new IgniteConfiguration();
+
+ ZookeeperDiscoverySpi spi = new ZookeeperDiscoverySpi();
+
+ spi.setZkConnectionString("localhost:2181");
+
+ cfg.setDiscoverySpi(spi);
+
+ Ignition.start(cfg);
+ }
+ catch (Throwable e) {
+ e.printStackTrace(System.out);
+
+ System.exit(1);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/IgniteCacheEntryListenerWithZkDiscoAtomicTest.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/IgniteCacheEntryListenerWithZkDiscoAtomicTest.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/IgniteCacheEntryListenerWithZkDiscoAtomicTest.java
new file mode 100644
index 0000000..754a6bf
--- /dev/null
+++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/IgniteCacheEntryListenerWithZkDiscoAtomicTest.java
@@ -0,0 +1,32 @@
+/*
+ * 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.spi.discovery.zk;
+
+import org.apache.ignite.internal.processors.cache.IgniteCacheEntryListenerAtomicTest;
+
+/**
+ * Class is added to mute {@link #testConcurrentRegisterDeregister} test in ZooKeeper suite
+ * (see related ticket).
+ *
+ * When slow down is tracked down and fixed this class can be replaced back with its parent.
+ */
+public class IgniteCacheEntryListenerWithZkDiscoAtomicTest extends IgniteCacheEntryListenerAtomicTest {
+ /** {@inheritDoc} */
+ @Override public void testConcurrentRegisterDeregister() throws Exception {
+ fail("https://issues.apache.org/jira/browse/IGNITE-8109");
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiAbstractTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiAbstractTestSuite.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiAbstractTestSuite.java
new file mode 100644
index 0000000..766635c
--- /dev/null
+++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiAbstractTestSuite.java
@@ -0,0 +1,118 @@
+/*
+ * 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.spi.discovery.zk;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+import junit.framework.TestSuite;
+import org.apache.curator.test.InstanceSpec;
+import org.apache.curator.test.TestingCluster;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.spi.discovery.DiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.testframework.config.GridTestProperties;
+
+/**
+ * Allows to run regular Ignite tests with {@link org.apache.ignite.spi.discovery.zk.ZookeeperDiscoverySpi}.
+ */
+public abstract class ZookeeperDiscoverySpiAbstractTestSuite extends TestSuite {
+ /** */
+ private static TestingCluster testingCluster;
+
+ /**
+ * @throws Exception If failed.
+ */
+ public static void initSuite() throws Exception {
+ System.setProperty("zookeeper.forceSync", "false");
+
+ testingCluster = createTestingCluster(3);
+
+ testingCluster.start();
+
+ System.setProperty(GridTestProperties.IGNITE_CFG_PREPROCESSOR_CLS, ZookeeperDiscoverySpiAbstractTestSuite.class.getName());
+ }
+
+ /**
+ * Called via reflection by {@link org.apache.ignite.testframework.junits.GridAbstractTest}.
+ *
+ * @param cfg Configuration to change.
+ */
+ public synchronized static void preprocessConfiguration(IgniteConfiguration cfg) {
+ if (testingCluster == null)
+ throw new IllegalStateException("Test Zookeeper cluster is not started.");
+
+ ZookeeperDiscoverySpi zkSpi = new ZookeeperDiscoverySpi();
+
+ DiscoverySpi spi = cfg.getDiscoverySpi();
+
+ if (spi instanceof TcpDiscoverySpi)
+ zkSpi.setClientReconnectDisabled(((TcpDiscoverySpi)spi).isClientReconnectDisabled());
+
+ zkSpi.setSessionTimeout(30_000);
+ zkSpi.setZkConnectionString(testingCluster.getConnectString());
+
+ cfg.setDiscoverySpi(zkSpi);
+ }
+
+ /**
+ * @param instances Number of instances in
+ * @return Test cluster.
+ */
+ public static TestingCluster createTestingCluster(int instances) {
+ String tmpDir = System.getProperty("java.io.tmpdir");
+
+ List<InstanceSpec> specs = new ArrayList<>();
+
+ for (int i = 0; i < instances; i++) {
+ File file = new File(tmpDir, "apacheIgniteTestZk-" + i);
+
+ if (file.isDirectory())
+ deleteRecursively0(file);
+ else {
+ if (!file.mkdirs())
+ throw new IgniteException("Failed to create directory for test Zookeeper server: " + file.getAbsolutePath());
+ }
+
+ specs.add(new InstanceSpec(file, -1, -1, -1, true, -1, -1, 500));
+ }
+
+ return new TestingCluster(specs);
+ }
+
+ /**
+ * @param file File or directory to delete.
+ */
+ private static void deleteRecursively0(File file) {
+ File[] files = file.listFiles();
+
+ if (files == null)
+ return;
+
+ for (File f : files) {
+ if (f.isDirectory())
+ deleteRecursively0(f);
+ else {
+ if (!f.delete())
+ throw new IgniteException("Failed to delete file: " + f.getAbsolutePath());
+ }
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiTestSuite1.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiTestSuite1.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiTestSuite1.java
new file mode 100644
index 0000000..860488b
--- /dev/null
+++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiTestSuite1.java
@@ -0,0 +1,44 @@
+/*
+ * 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.spi.discovery.zk;
+
+import junit.framework.TestSuite;
+import org.apache.ignite.spi.discovery.zk.internal.ZookeeperClientTest;
+import org.apache.ignite.spi.discovery.zk.internal.ZookeeperDiscoverySpiSaslSuccessfulAuthTest;
+import org.apache.ignite.spi.discovery.zk.internal.ZookeeperDiscoverySpiTest;
+
+/**
+ *
+ */
+public class ZookeeperDiscoverySpiTestSuite1 extends TestSuite {
+ /**
+ * @return Test suite.
+ * @throws Exception Thrown in case of the failure.
+ */
+ public static TestSuite suite() throws Exception {
+ System.setProperty("zookeeper.forceSync", "false");
+
+ TestSuite suite = new TestSuite("ZookeeperDiscoverySpi Test Suite");
+
+ suite.addTestSuite(ZookeeperClientTest.class);
+ suite.addTestSuite(ZookeeperDiscoverySpiTest.class);
+ suite.addTestSuite(ZookeeperDiscoverySpiSaslSuccessfulAuthTest.class);
+
+ return suite;
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiTestSuite2.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiTestSuite2.java
new file mode 100644
index 0000000..3775aa1
--- /dev/null
+++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiTestSuite2.java
@@ -0,0 +1,94 @@
+/*
+ * 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.spi.discovery.zk;
+
+import junit.framework.TestSuite;
+import org.apache.curator.test.TestingCluster;
+import org.apache.ignite.internal.ClusterNodeMetricsUpdateTest;
+import org.apache.ignite.internal.IgniteClientReconnectCacheTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheEntryListenerAtomicTest;
+import org.apache.ignite.internal.processors.cache.datastructures.IgniteClientDataStructuresTest;
+import org.apache.ignite.internal.processors.cache.datastructures.partitioned.GridCachePartitionedNodeRestartTxSelfTest;
+import org.apache.ignite.internal.processors.cache.datastructures.partitioned.GridCachePartitionedSequenceApiSelfTest;
+import org.apache.ignite.internal.processors.cache.datastructures.replicated.GridCacheReplicatedSequenceApiSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCachePutRetryAtomicSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCachePutRetryTransactionalSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicMultiNodeFullApiSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedMultiNodeFullApiSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedNodeRestartTest;
+import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCacheReplicatedAtomicMultiNodeFullApiSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCacheReplicatedMultiNodeFullApiSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCacheReplicatedNodeRestartSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedQuerySelfTest;
+import org.apache.ignite.internal.processors.cache.multijvm.GridCacheAtomicMultiJvmFullApiSelfTest;
+import org.apache.ignite.internal.processors.cache.multijvm.GridCachePartitionedMultiJvmFullApiSelfTest;
+import org.apache.ignite.internal.processors.continuous.GridEventConsumeSelfTest;
+
+/**
+ * Regular Ignite tests executed with {@link org.apache.ignite.spi.discovery.zk.ZookeeperDiscoverySpi}.
+ */
+public class ZookeeperDiscoverySpiTestSuite2 extends ZookeeperDiscoverySpiAbstractTestSuite {
+ /** */
+ private static TestingCluster testingCluster;
+
+ /**
+ * @return Test suite.
+ * @throws Exception Thrown in case of the failure.
+ */
+ public static TestSuite suite() throws Exception {
+ System.setProperty("H2_JDBC_CONNECTIONS", "500"); // For multi-jvm tests.
+
+ initSuite();
+
+ TestSuite suite = new TestSuite("ZookeeperDiscoverySpi Test Suite");
+
+ suite.addTestSuite(ZookeeperDiscoverySuitePreprocessorTest.class);
+
+ suite.addTestSuite(GridCacheReplicatedNodeRestartSelfTest.class);
+ suite.addTestSuite(GridCachePartitionedNodeRestartTest.class);
+
+ suite.addTestSuite(IgniteCacheEntryListenerWithZkDiscoAtomicTest.class);
+
+ suite.addTestSuite(GridEventConsumeSelfTest.class);
+
+ suite.addTestSuite(IgniteClientReconnectCacheTest.class);
+
+ suite.addTestSuite(IgniteCachePutRetryAtomicSelfTest.class);
+ suite.addTestSuite(IgniteCachePutRetryTransactionalSelfTest.class);
+
+ suite.addTestSuite(ClusterNodeMetricsUpdateTest.class);
+
+ suite.addTestSuite(GridCachePartitionedMultiNodeFullApiSelfTest.class);
+ suite.addTestSuite(GridCacheReplicatedMultiNodeFullApiSelfTest.class);
+
+ suite.addTestSuite(GridCacheAtomicMultiNodeFullApiSelfTest.class);
+ suite.addTestSuite(GridCacheReplicatedAtomicMultiNodeFullApiSelfTest.class);
+
+ suite.addTestSuite(GridCachePartitionedNodeRestartTxSelfTest.class);
+ suite.addTestSuite(IgniteClientDataStructuresTest.class);
+ suite.addTestSuite(GridCacheReplicatedSequenceApiSelfTest.class);
+ suite.addTestSuite(GridCachePartitionedSequenceApiSelfTest.class);
+
+ suite.addTestSuite(IgniteCacheReplicatedQuerySelfTest.class);
+
+ suite.addTestSuite(GridCacheAtomicMultiJvmFullApiSelfTest.class);
+ suite.addTestSuite(GridCachePartitionedMultiJvmFullApiSelfTest.class);
+
+ return suite;
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySuitePreprocessorTest.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySuitePreprocessorTest.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySuitePreprocessorTest.java
new file mode 100644
index 0000000..28cf17f
--- /dev/null
+++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySuitePreprocessorTest.java
@@ -0,0 +1,101 @@
+/*
+ * 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.spi.discovery.zk;
+
+import java.util.List;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.spi.discovery.DiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.config.GridTestProperties;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ * Sanity test verifying that configuration callback specified via
+ * {@link GridTestProperties#IGNITE_CFG_PREPROCESSOR_CLS} really works.
+ * <p>
+ * This test should be run as part of {@link ZookeeperDiscoverySpiTestSuite2}.
+ */
+public class ZookeeperDiscoverySuitePreprocessorTest extends GridCommonAbstractTest {
+ /** */
+ private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+ /** {@inheritDoc} */
+ @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+ IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+ // Test sets TcpDiscoverySpi, but it should be automatically changed to ZookeeperDiscoverySpi.
+ TcpDiscoverySpi spi = new TcpDiscoverySpi();
+
+ spi.setIpFinder(IP_FINDER);
+
+ cfg.setDiscoverySpi(spi);
+
+ return cfg;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected void afterTest() throws Exception {
+ stopAllGrids();
+
+ super.afterTest();
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testSpiConfigurationIsChanged() throws Exception {
+ startGrid(0);
+
+ checkDiscoverySpi(1);
+
+ startGrid(1);
+
+ checkDiscoverySpi(2);
+
+ startGridsMultiThreaded(2, 2);
+
+ checkDiscoverySpi(4);
+
+ startGrid();
+
+ checkDiscoverySpi(5);
+ }
+
+ /**
+ * @param expNodes Expected nodes number.
+ * @throws Exception If failed.
+ */
+ private void checkDiscoverySpi(int expNodes) throws Exception {
+ List<Ignite> nodes = G.allGrids();
+
+ assertEquals(expNodes, nodes.size());
+
+ for (Ignite node : nodes) {
+ DiscoverySpi spi = node.configuration().getDiscoverySpi();
+
+ assertTrue("Node should be started with " + ZookeeperDiscoverySpi.class.getName(),
+ spi instanceof ZookeeperDiscoverySpi);
+ }
+
+ waitForTopology(expNodes);
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClientTest.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClientTest.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClientTest.java
new file mode 100644
index 0000000..e7cb97a
--- /dev/null
+++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClientTest.java
@@ -0,0 +1,495 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.curator.test.TestingCluster;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteRunnable;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.zookeeper.AsyncCallback;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.data.Stat;
+
+/**
+ *
+ */
+public class ZookeeperClientTest extends GridCommonAbstractTest {
+ /** */
+ private static final int SES_TIMEOUT = 60_000;
+
+ /** */
+ private TestingCluster zkCluster;
+
+ /** {@inheritDoc} */
+ @Override protected void afterTest() throws Exception {
+ closeZK();
+
+ super.afterTest();
+ }
+
+ /**
+ * @param sesTimeout Session timeout.
+ * @return Client.
+ * @throws Exception If failed.
+ */
+ private ZookeeperClient createClient(int sesTimeout) throws Exception {
+ return new ZookeeperClient(log, zkCluster.getConnectString(), sesTimeout, null);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testSaveLargeValue() throws Exception {
+ startZK(1);
+
+ final ZookeeperClient client = createClient(SES_TIMEOUT);
+
+ byte[] data = new byte[1024 * 1024];
+
+ String basePath = "/ignite";
+
+ assertTrue(client.needSplitNodeData(basePath, data, 2));
+
+ List<byte[]> parts = client.splitNodeData(basePath, data, 2);
+
+ assertTrue(parts.size() > 1);
+
+ ZooKeeper zk = client.zk();
+
+ for (int i = 0; i < parts.size(); i++) {
+ byte[] part = parts.get(i);
+
+ assertTrue(part.length > 0);
+
+ String path0 = basePath + ":" + i;
+
+ zk.create(path0, part, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+ }
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testClose() throws Exception {
+ startZK(1);
+
+ final ZookeeperClient client = createClient(SES_TIMEOUT);
+
+ client.createIfNeeded("/apacheIgnite1", null, CreateMode.PERSISTENT);
+
+ client.zk().close();
+
+ GridTestUtils.assertThrows(log, new Callable<Void>() {
+ @Override public Void call() throws Exception {
+ client.createIfNeeded("/apacheIgnite2", null, CreateMode.PERSISTENT);
+
+ return null;
+ }
+ }, ZookeeperClientFailedException.class, null);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testCreateAll() throws Exception {
+ startZK(1);
+
+ ZookeeperClient client = createClient(SES_TIMEOUT);
+
+ client.createIfNeeded("/apacheIgnite", null, CreateMode.PERSISTENT);
+
+ List<String> paths = new ArrayList<>();
+
+ paths.add("/apacheIgnite/1");
+ paths.add("/apacheIgnite/2");
+ paths.add("/apacheIgnite/3");
+
+ client.createAll(paths, CreateMode.PERSISTENT);
+
+ assertEquals(3, client.getChildren("/apacheIgnite").size());
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testDeleteAll() throws Exception {
+ startZK(1);
+
+ ZookeeperClient client = createClient(SES_TIMEOUT);
+
+ client.createIfNeeded("/apacheIgnite", null, CreateMode.PERSISTENT);
+ client.createIfNeeded("/apacheIgnite/1", null, CreateMode.PERSISTENT);
+ client.createIfNeeded("/apacheIgnite/2", null, CreateMode.PERSISTENT);
+
+ client.deleteAll("/apacheIgnite", Arrays.asList("1", "2"), -1);
+
+ assertTrue(client.getChildren("/apacheIgnite").isEmpty());
+
+ client.createIfNeeded("/apacheIgnite/1", null, CreateMode.PERSISTENT);
+ client.deleteAll("/apacheIgnite", Collections.singletonList("1"), -1);
+
+ assertTrue(client.getChildren("/apacheIgnite").isEmpty());
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testConnectionLoss1() throws Exception {
+ ZookeeperClient client = new ZookeeperClient(log, "localhost:2200", 3000, null);
+
+ try {
+ client.createIfNeeded("/apacheIgnite", null, CreateMode.PERSISTENT);
+
+ fail();
+ }
+ catch (ZookeeperClientFailedException e) {
+ info("Expected error: " + e);
+ }
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testConnectionLoss2() throws Exception {
+ startZK(1);
+
+ ZookeeperClient client = createClient(3000);
+
+ client.createIfNeeded("/apacheIgnite1", null, CreateMode.PERSISTENT);
+
+ closeZK();
+
+ try {
+ client.createIfNeeded("/apacheIgnite2", null, CreateMode.PERSISTENT);
+
+ fail();
+ }
+ catch (ZookeeperClientFailedException e) {
+ info("Expected error: " + e);
+ }
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testConnectionLoss3() throws Exception {
+ startZK(1);
+
+ CallbackFuture cb = new CallbackFuture();
+
+ ZookeeperClient client = new ZookeeperClient(log, zkCluster.getConnectString(), 3000, cb);
+
+ client.createIfNeeded("/apacheIgnite1", null, CreateMode.PERSISTENT);
+
+ closeZK();
+
+ final AtomicBoolean res = new AtomicBoolean();
+
+ client.getChildrenAsync("/apacheIgnite1", null, new AsyncCallback.Children2Callback() {
+ @Override public void processResult(int rc, String path, Object ctx, List<String> children, Stat stat) {
+ if (rc == 0)
+ res.set(true);
+ }
+ });
+
+ cb.get(60_000);
+
+ assertFalse(res.get());
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testConnectionLoss4() throws Exception {
+ startZK(1);
+
+ CallbackFuture cb = new CallbackFuture();
+
+ final ZookeeperClient client = new ZookeeperClient(log, zkCluster.getConnectString(), 3000, cb);
+
+ client.createIfNeeded("/apacheIgnite1", null, CreateMode.PERSISTENT);
+
+ final CountDownLatch l = new CountDownLatch(1);
+
+ client.getChildrenAsync("/apacheIgnite1", null, new AsyncCallback.Children2Callback() {
+ @Override public void processResult(int rc, String path, Object ctx, List<String> children, Stat stat) {
+ closeZK();
+
+ try {
+ client.createIfNeeded("/apacheIgnite2", null, CreateMode.PERSISTENT);
+ }
+ catch (ZookeeperClientFailedException e) {
+ info("Expected error: " + e);
+
+ l.countDown();
+ }
+ catch (Exception e) {
+ fail("Unexpected error: " + e);
+ }
+ }
+ });
+
+ assertTrue(l.await(10, TimeUnit.SECONDS));
+
+ cb.get();
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testReconnect1() throws Exception {
+ startZK(1);
+
+ ZookeeperClient client = createClient(SES_TIMEOUT);
+
+ client.createIfNeeded("/apacheIgnite1", null, CreateMode.PERSISTENT);
+
+ zkCluster.getServers().get(0).stop();
+
+ IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable<Void>() {
+ @Override public Void call() throws Exception {
+ U.sleep(2000);
+
+ info("Restart zookeeper server");
+
+ zkCluster.getServers().get(0).restart();
+
+ info("Zookeeper server restarted");
+
+ return null;
+ }
+ }, "start-zk");
+
+ client.createIfNeeded("/apacheIgnite2", null, CreateMode.PERSISTENT);
+
+ fut.get();
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testReconnect1_Callback() throws Exception {
+ startZK(1);
+
+ ZookeeperClient client = createClient(SES_TIMEOUT);
+
+ client.createIfNeeded("/apacheIgnite1", null, CreateMode.PERSISTENT);
+
+ zkCluster.getServers().get(0).stop();
+
+ final CountDownLatch l = new CountDownLatch(1);
+
+ client.getChildrenAsync("/apacheIgnite1", null, new AsyncCallback.Children2Callback() {
+ @Override public void processResult(int rc, String path, Object ctx, List<String> children, Stat stat) {
+ info("Callback: " + rc);
+
+ if (rc == 0)
+ l.countDown();
+ }
+ });
+
+ IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable<Void>() {
+ @Override public Void call() throws Exception {
+ U.sleep(2000);
+
+ info("Restart zookeeper server");
+
+ zkCluster.getServers().get(0).restart();
+
+ info("Zookeeper server restarted");
+
+ return null;
+ }
+ }, "start-zk");
+
+ assertTrue(l.await(10, TimeUnit.SECONDS));
+
+ fut.get();
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testReconnect1_InCallback() throws Exception {
+ startZK(1);
+
+ final ZookeeperClient client = createClient(SES_TIMEOUT);
+
+ client.createIfNeeded("/apacheIgnite1", null, CreateMode.PERSISTENT);
+
+ final CountDownLatch l = new CountDownLatch(1);
+
+ client.getChildrenAsync("/apacheIgnite1", null, new AsyncCallback.Children2Callback() {
+ @Override public void processResult(int rc, String path, Object ctx, List<String> children, Stat stat) {
+ try {
+ zkCluster.getServers().get(0).stop();
+
+ IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable<Void>() {
+ @Override public Void call() throws Exception {
+ U.sleep(2000);
+
+ info("Restart zookeeper server");
+
+ zkCluster.getServers().get(0).restart();
+
+ info("Zookeeper server restarted");
+
+ return null;
+ }
+ }, "start-zk");
+
+ client.createIfNeeded("/apacheIgnite2", null, CreateMode.PERSISTENT);
+
+ l.countDown();
+
+ fut.get();
+ }
+ catch (Exception e) {
+ fail("Unexpected error: " + e);
+ }
+ }
+ });
+
+ assertTrue(l.await(10, TimeUnit.SECONDS));
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testReconnect2() throws Exception {
+ startZK(1);
+
+ ZookeeperClient client = createClient(SES_TIMEOUT);
+
+ client.createIfNeeded("/apacheIgnite1", null, CreateMode.PERSISTENT);
+
+ zkCluster.getServers().get(0).restart();
+
+ client.createIfNeeded("/apacheIgnite2", null, CreateMode.PERSISTENT);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testReconnect3() throws Exception {
+ startZK(3);
+
+ ZookeeperClient client = createClient(SES_TIMEOUT);
+
+ ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+ for (int i = 0; i < 30; i++) {
+ info("Iteration: " + i);
+
+ int idx = rnd.nextInt(3);
+
+ zkCluster.getServers().get(idx).restart();
+
+ doSleep(rnd.nextLong(100) + 1);
+
+ client.createIfNeeded("/apacheIgnite" + i, null, CreateMode.PERSISTENT);
+ }
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testReconnect4() throws Exception {
+ startZK(3);
+
+ ZookeeperClient client = new ZookeeperClient(log,
+ zkCluster.getServers().get(2).getInstanceSpec().getConnectString(),
+ 60_000,
+ null);
+
+ client.createIfNeeded("/apacheIgnite1", null, CreateMode.PERSISTENT);
+
+ zkCluster.getServers().get(0).stop();
+ zkCluster.getServers().get(1).stop();
+
+ IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable<Void>() {
+ @Override public Void call() throws Exception {
+ U.sleep(2000);
+
+ info("Restart zookeeper server");
+
+ zkCluster.getServers().get(0).restart();
+
+ info("Zookeeper server restarted");
+
+ return null;
+ }
+ }, "start-zk");
+
+ client.createIfNeeded("/apacheIgnite2", null, CreateMode.PERSISTENT);
+
+ fut.get();
+ }
+
+ /**
+ * @param instances Number of servers in ZK ensemble.
+ * @throws Exception If failed.
+ */
+ private void startZK(int instances) throws Exception {
+ assert zkCluster == null;
+
+ zkCluster = new TestingCluster(instances);
+
+ zkCluster.start();
+ }
+
+ /**
+ *
+ */
+ private void closeZK() {
+ if (zkCluster != null) {
+ try {
+ zkCluster.close();
+ }
+ catch (Exception e) {
+ U.error(log, "Failed to stop Zookeeper client: " + e, e);
+ }
+
+ zkCluster = null;
+ }
+ }
+
+ /**
+ *
+ */
+ private static class CallbackFuture extends GridFutureAdapter<Void> implements IgniteRunnable {
+ /** {@inheritDoc} */
+ @Override public void run() {
+ onDone();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiSaslAuthAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiSaslAuthAbstractTest.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiSaslAuthAbstractTest.java
new file mode 100644
index 0000000..ac94bf2
--- /dev/null
+++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiSaslAuthAbstractTest.java
@@ -0,0 +1,247 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.nio.file.Paths;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.discovery.zk.ZookeeperDiscoverySpi;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.zookeeper.client.ZooKeeperSaslClient;
+import org.apache.zookeeper.server.ServerCnxnFactory;
+import org.apache.zookeeper.server.ZKDatabase;
+import org.apache.zookeeper.server.ZooKeeperServer;
+
+import static org.apache.curator.test.DirectoryUtils.deleteRecursively;
+
+/**
+ * Implements methods to prepare SASL tests infrastructure: jaas.conf files, starting up ZooKeeper server,
+ * clean up procedures when the test has finished etc.
+ */
+public abstract class ZookeeperDiscoverySpiSaslAuthAbstractTest extends GridCommonAbstractTest {
+ /** */
+ private File tmpDir = createTmpDir();
+
+ /** */
+ private static final String JAAS_CONF_FILE = "jaas.conf";
+
+ /** */
+ private static final String AUTH_PROVIDER = "zookeeper.authProvider.1";
+
+ /** */
+ private static final String SASL_CONFIG = "java.security.auth.login.config";
+
+ /** */
+ private long joinTimeout = 2_000;
+
+ /** */
+ private long sesTimeout = 10_000;
+
+ /** */
+ private ServerCnxnFactory serverFactory;
+
+ /** */
+ private String hostPort = "localhost:2181";
+
+ /** */
+ private int maxCnxns;
+
+ /** {@inheritDoc} */
+ @Override protected IgniteConfiguration getConfiguration(String instanceName) throws Exception {
+ IgniteConfiguration cfg = super.getConfiguration(instanceName);
+
+ ZookeeperDiscoverySpi zkSpi = new ZookeeperDiscoverySpi();
+
+ if (joinTimeout != 0)
+ zkSpi.setJoinTimeout(joinTimeout);
+
+ zkSpi.setSessionTimeout(sesTimeout > 0 ? sesTimeout : 10_000);
+
+ zkSpi.setZkConnectionString(hostPort);
+
+ cfg.setDiscoverySpi(zkSpi);
+
+ return cfg;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected void beforeTest() throws Exception {
+ prepareJaasConfigFile();
+
+ prepareSaslSystemProperties();
+
+ startZooKeeperServer();
+ }
+
+ /** {@inheritDoc} */
+ @Override protected void afterTest() throws Exception {
+ stopZooKeeperServer();
+
+ stopAllGrids();
+
+ clearSaslSystemProperties();
+
+ clearTmpDir();
+ }
+
+ /** */
+ private void clearTmpDir() throws Exception {
+ deleteRecursively(tmpDir);
+ }
+
+ /** */
+ protected void clearSaslSystemProperties() {
+ System.clearProperty(AUTH_PROVIDER);
+
+ System.clearProperty(SASL_CONFIG);
+
+ System.clearProperty(ZooKeeperSaslClient.LOGIN_CONTEXT_NAME_KEY);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ private void prepareJaasConfigFile() throws Exception {
+ U.ensureDirectory(tmpDir, "Temp directory for JAAS configuration file wasn't created", null);
+
+ File saslConfFile = new File(tmpDir, JAAS_CONF_FILE);
+
+ FileWriter fwriter = new FileWriter(saslConfFile);
+
+ writeServerConfigSection(fwriter, "validPassword");
+
+ writeClientConfigSection(fwriter, "ValidZookeeperClient", "validPassword");
+
+ writeClientConfigSection(fwriter, "InvalidZookeeperClient", "invalidPassword");
+
+ fwriter.close();
+ }
+
+ /** */
+ private void prepareSaslSystemProperties() {
+ System.setProperty(SASL_CONFIG, Paths.get(tmpDir.getPath().toString(), JAAS_CONF_FILE).toString());
+
+ System.setProperty(AUTH_PROVIDER, "org.apache.zookeeper.server.auth.SASLAuthenticationProvider");
+ }
+
+ /** */
+ private void writeClientConfigSection(FileWriter fwriter, String clientName, String pass) throws IOException {
+ fwriter.write(clientName + "{\n" +
+ " org.apache.zookeeper.server.auth.DigestLoginModule required\n" +
+ " username=\"zkUser\"\n" +
+ " password=\"" + pass + "\";\n" +
+ "};" + "\n");
+ }
+
+ /** */
+ private void writeServerConfigSection(FileWriter fwriter, String pass) throws IOException {
+ fwriter.write("Server {\n" +
+ " org.apache.zookeeper.server.auth.DigestLoginModule required\n" +
+ " user_zkUser=\"" + pass + "\";\n" +
+ "};\n");
+ }
+
+ /** */
+ private File createTmpDir() {
+ File jaasConfDir = Paths.get(System.getProperty("java.io.tmpdir"), "zk_disco_spi_test").toFile();
+
+ try {
+ U.ensureDirectory(jaasConfDir, "", null);
+ }
+ catch (IgniteCheckedException e) {
+ // ignored
+ }
+
+ return jaasConfDir;
+ }
+
+ /** */
+ private void stopZooKeeperServer() throws Exception {
+ shutdownServerInstance(serverFactory);
+ serverFactory = null;
+ }
+
+ /** */
+ private void shutdownServerInstance(ServerCnxnFactory factory)
+ {
+ if (factory != null) {
+ ZKDatabase zkDb = null;
+ {
+ ZooKeeperServer zs = getServer(factory);
+ if (zs != null)
+ zkDb = zs.getZKDatabase();
+ }
+ factory.shutdown();
+ try {
+ if (zkDb != null)
+ zkDb.close();
+ } catch (IOException ie) {
+ // ignore
+ }
+ }
+ }
+
+ /** */
+ private ZooKeeperServer getServer(ServerCnxnFactory fac) {
+ ZooKeeperServer zs = U.field(fac, "zkServer");
+
+ return zs;
+ }
+
+ /** */
+ private void startZooKeeperServer() throws Exception {
+ serverFactory = createNewServerInstance(serverFactory, hostPort,
+ maxCnxns);
+ startServerInstance(tmpDir, serverFactory);
+ }
+
+ /** */
+ private ServerCnxnFactory createNewServerInstance(
+ ServerCnxnFactory factory, String hostPort, int maxCnxns)
+ throws IOException {
+ final int port = getPort(hostPort);
+
+ if (factory == null)
+ factory = ServerCnxnFactory.createFactory(port, maxCnxns);
+
+ return factory;
+ }
+
+ /** */
+ private void startServerInstance(File dataDir,
+ ServerCnxnFactory factory) throws IOException,
+ InterruptedException {
+ ZooKeeperServer zks = new ZooKeeperServer(dataDir, dataDir, 3000);
+ factory.startup(zks);
+ }
+
+ /** */
+ private int getPort(String hostPort) {
+ String[] split = hostPort.split(":");
+ String portstr = split[split.length-1];
+ String[] pc = portstr.split("/");
+
+ if (pc.length > 1)
+ portstr = pc[0];
+
+ return Integer.parseInt(portstr);
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiSaslFailedAuthTest.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiSaslFailedAuthTest.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiSaslFailedAuthTest.java
new file mode 100644
index 0000000..864ac96
--- /dev/null
+++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiSaslFailedAuthTest.java
@@ -0,0 +1,44 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import org.apache.zookeeper.client.ZooKeeperSaslClient;
+import org.junit.Assert;
+
+/**
+ *
+ */
+public class ZookeeperDiscoverySpiSaslFailedAuthTest extends ZookeeperDiscoverySpiSaslAuthAbstractTest {
+ /**
+ * @throws Exception If failed.
+ */
+ public void testIgniteNodeWithInvalidPasswordFailsToJoin() throws Exception {
+ System.setProperty(ZooKeeperSaslClient.LOGIN_CONTEXT_NAME_KEY,
+ "InvalidZookeeperClient");
+
+ System.setProperty("IGNITE_ZOOKEEPER_DISCOVERY_MAX_RETRY_COUNT", Integer.toString(1));
+
+ try {
+ startGrid(0);
+
+ Assert.fail("Ignite node with invalid password should fail on join.");
+ }
+ catch (Exception e) {
+ //ignored
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiSaslSuccessfulAuthTest.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiSaslSuccessfulAuthTest.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiSaslSuccessfulAuthTest.java
new file mode 100644
index 0000000..5ee0a43
--- /dev/null
+++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiSaslSuccessfulAuthTest.java
@@ -0,0 +1,48 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import org.apache.zookeeper.client.ZooKeeperSaslClient;
+
+/**
+ *
+ */
+public class ZookeeperDiscoverySpiSaslSuccessfulAuthTest extends ZookeeperDiscoverySpiSaslAuthAbstractTest {
+ /**
+ * @throws Exception If failed.
+ */
+ public void testIgniteNodesWithValidPasswordSuccessfullyJoins() throws Exception {
+ System.setProperty(ZooKeeperSaslClient.LOGIN_CONTEXT_NAME_KEY,
+ "ValidZookeeperClient");
+
+ startGrids(3);
+
+ waitForTopology(3);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testIgniteNodeWithoutSaslConfigurationSuccessfullyJoins() throws Exception {
+ //clearing SASL-related system properties that were set in beforeTest
+ clearSaslSystemProperties();
+
+ startGrid(0);
+
+ waitForTopology(1);
+ }
+}
[05/12] ignite git commit: IGNITE-7222 Added ZooKeeper discovery SPI
Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClient.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClient.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClient.java
new file mode 100644
index 0000000..21703c6
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClient.java
@@ -0,0 +1,1219 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteRunnable;
+import org.apache.zookeeper.AsyncCallback;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Stat;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Zookeeper Client.
+ */
+public class ZookeeperClient implements Watcher {
+ /** */
+ private static final long RETRY_TIMEOUT =
+ IgniteSystemProperties.getLong("IGNITE_ZOOKEEPER_DISCOVERY_RETRY_TIMEOUT", 2000);
+
+ /** */
+ private static final int MAX_RETRY_COUNT =
+ IgniteSystemProperties.getInteger("IGNITE_ZOOKEEPER_DISCOVERY_MAX_RETRY_COUNT", 10);
+
+ /** */
+ private final AtomicInteger retryCount = new AtomicInteger();
+
+ /** */
+ private static final int MAX_REQ_SIZE = 1048528;
+
+ /** */
+ private static final List<ACL> ZK_ACL = ZooDefs.Ids.OPEN_ACL_UNSAFE;
+
+ /** */
+ private static final byte[] EMPTY_BYTES = {};
+
+ /** */
+ private final ZooKeeper zk;
+
+ /** */
+ private final IgniteLogger log;
+
+ /** */
+ private ConnectionState state = ConnectionState.Disconnected;
+
+ /** */
+ private long connLossTimeout;
+
+ /** */
+ private volatile long connStartTime;
+
+ /** */
+ private final Object stateMux = new Object();
+
+ /** */
+ private final IgniteRunnable connLostC;
+
+ /** */
+ private final Timer connTimer;
+
+ /** */
+ private final ArrayDeque<ZkAsyncOperation> retryQ = new ArrayDeque<>();
+
+ /** */
+ private volatile boolean closing;
+
+ /**
+ * @param log Logger.
+ * @param connectString ZK connection string.
+ * @param sesTimeout ZK session timeout.
+ * @param connLostC Lost connection callback.
+ * @throws Exception If failed.
+ */
+ ZookeeperClient(IgniteLogger log, String connectString, int sesTimeout, IgniteRunnable connLostC) throws Exception {
+ this(null, log, connectString, sesTimeout, connLostC);
+ }
+
+ /**
+ * @param igniteInstanceName Ignite instance name.
+ * @param log Logger.
+ * @param connectString ZK connection string.
+ * @param sesTimeout ZK session timeout.
+ * @param connLostC Lost connection callback.
+ * @throws Exception If failed.
+ */
+ ZookeeperClient(String igniteInstanceName,
+ IgniteLogger log,
+ String connectString,
+ int sesTimeout,
+ IgniteRunnable connLostC)
+ throws Exception
+ {
+ this.log = log.getLogger(getClass());
+ this.connLostC = connLostC;
+
+ connLossTimeout = sesTimeout;
+
+ long connStartTime = this.connStartTime = System.currentTimeMillis();
+
+ connTimer = new Timer("zk-client-timer-" + igniteInstanceName);
+
+ String threadName = Thread.currentThread().getName();
+
+ // ZK generates internal threads' names using current thread name.
+ Thread.currentThread().setName("zk-" + igniteInstanceName);
+
+ try {
+ zk = new ZooKeeper(connectString, sesTimeout, this);
+ }
+ finally {
+ Thread.currentThread().setName(threadName);
+ }
+
+ synchronized (stateMux) {
+ if (connStartTime == this.connStartTime && state == ConnectionState.Disconnected)
+ scheduleConnectionCheck();
+ }
+ }
+
+ /**
+ * @return Zookeeper client.
+ */
+ ZooKeeper zk() {
+ return zk;
+ }
+
+ /**
+ * @return {@code True} if connected to ZooKeeper.
+ */
+ boolean connected() {
+ synchronized (stateMux) {
+ return state == ConnectionState.Connected;
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override public void process(WatchedEvent evt) {
+ if (closing)
+ return;
+
+ if (evt.getType() == Event.EventType.None) {
+ ConnectionState newState;
+
+ synchronized (stateMux) {
+ if (state == ConnectionState.Lost) {
+ U.warn(log, "Received event after connection was lost [evtState=" + evt.getState() + "]");
+
+ return;
+ }
+
+ if (!zk.getState().isAlive())
+ return;
+
+ Event.KeeperState zkState = evt.getState();
+
+ switch (zkState) {
+ case SaslAuthenticated:
+ return; // No-op.
+
+ case AuthFailed:
+ newState = state;
+
+ break;
+
+ case Disconnected:
+ newState = ConnectionState.Disconnected;
+
+ break;
+
+ case SyncConnected:
+ newState = ConnectionState.Connected;
+
+ break;
+
+ case Expired:
+ U.warn(log, "Session expired, changing state to Lost");
+
+ newState = ConnectionState.Lost;
+
+ break;
+
+ default:
+ U.error(log, "Unexpected state for ZooKeeper client, close connection: " + zkState);
+
+ newState = ConnectionState.Lost;
+ }
+
+ if (newState != state) {
+ if (log.isInfoEnabled())
+ log.info("ZooKeeper client state changed [prevState=" + state + ", newState=" + newState + ']');
+
+ state = newState;
+
+ if (newState == ConnectionState.Disconnected) {
+ connStartTime = System.currentTimeMillis();
+
+ scheduleConnectionCheck();
+ }
+ else if (newState == ConnectionState.Connected) {
+ retryCount.set(0);
+
+ stateMux.notifyAll();
+ }
+ else
+ assert state == ConnectionState.Lost : state;
+ }
+ else
+ return;
+ }
+
+ if (newState == ConnectionState.Lost) {
+ closeClient();
+
+ notifyConnectionLost();
+ }
+ else if (newState == ConnectionState.Connected) {
+ for (ZkAsyncOperation op : retryQ)
+ op.execute();
+ }
+ }
+ }
+
+ /**
+ *
+ */
+ private void notifyConnectionLost() {
+ if (!closing && state == ConnectionState.Lost && connLostC != null)
+ connLostC.run();
+
+ connTimer.cancel();
+ }
+
+ /**
+ * @param path Path.
+ * @return {@code True} if node exists.
+ * @throws ZookeeperClientFailedException If connection to zk was lost.
+ * @throws InterruptedException If interrupted.
+ */
+ boolean exists(String path) throws ZookeeperClientFailedException, InterruptedException {
+ for (;;) {
+ long connStartTime = this.connStartTime;
+
+ try {
+ return zk.exists(path, false) != null;
+ }
+ catch (Exception e) {
+ onZookeeperError(connStartTime, e);
+ }
+ }
+ }
+
+ /**
+ *
+ * @param paths Paths to create.
+ * @param createMode Create mode.
+ * @throws KeeperException.NodeExistsException If at least one of target node already exists.
+ * @throws ZookeeperClientFailedException If connection to zk was lost.
+ * @throws InterruptedException If interrupted.
+ */
+ void createAll(List<String> paths, CreateMode createMode)
+ throws ZookeeperClientFailedException, InterruptedException, KeeperException.NodeExistsException
+ {
+ // TODO ZK: https://issues.apache.org/jira/browse/IGNITE-8188
+ List<Op> ops = new ArrayList<>(paths.size());
+
+ for (String path : paths)
+ ops.add(Op.create(path, EMPTY_BYTES, ZK_ACL, createMode));
+
+ for (;;) {
+ long connStartTime = this.connStartTime;
+
+ try {
+ zk.multi(ops);
+
+ return;
+ }
+ catch (KeeperException.NodeExistsException e) {
+ throw e;
+ }
+ catch (Exception e) {
+ onZookeeperError(connStartTime, e);
+ }
+ }
+ }
+
+ /**
+ * @param path Path.
+ * @param data Data.
+ * @param overhead Extra overhead.
+ * @return {@code True} If data size exceeds max request size and should be splitted into multiple parts.
+ */
+ boolean needSplitNodeData(String path, byte[] data, int overhead) {
+ return requestOverhead(path) + data.length + overhead > MAX_REQ_SIZE;
+ }
+
+ /**
+ * @param path Path.
+ * @param data Data.
+ * @param overhead Extra overhead.
+ * @return Splitted data.
+ */
+ List<byte[]> splitNodeData(String path, byte[] data, int overhead) {
+ int partSize = MAX_REQ_SIZE - requestOverhead(path) - overhead;
+
+ int partCnt = data.length / partSize;
+
+ if (data.length % partSize != 0)
+ partCnt++;
+
+ assert partCnt > 1 : "Do not need split";
+
+ List<byte[]> parts = new ArrayList<>(partCnt);
+
+ int remaining = data.length;
+
+ for (int i = 0; i < partCnt; i++) {
+ int partSize0 = Math.min(remaining, partSize);
+
+ byte[] part = new byte[partSize0];
+
+ System.arraycopy(data, i * partSize, part, 0, part.length);
+
+ remaining -= partSize0;
+
+ parts.add(part);
+ }
+
+ assert remaining == 0 : remaining;
+
+ return parts;
+ }
+
+ /**
+ * TODO ZK: https://issues.apache.org/jira/browse/IGNITE-8187
+ * @param path Request path.
+ * @return Marshalled request overhead.
+ */
+ private int requestOverhead(String path) {
+ return path.length();
+ }
+
+ /**
+ * @param path Path.
+ * @param data Data.
+ * @param createMode Create mode.
+ * @return Created path.
+ * @throws ZookeeperClientFailedException If connection to zk was lost.
+ * @throws InterruptedException If interrupted.
+ */
+ String createIfNeeded(String path, byte[] data, CreateMode createMode)
+ throws ZookeeperClientFailedException, InterruptedException
+ {
+ assert !createMode.isSequential() : createMode;
+
+ if (data == null)
+ data = EMPTY_BYTES;
+
+ for (;;) {
+ long connStartTime = this.connStartTime;
+
+ try {
+ return zk.create(path, data, ZK_ACL, createMode);
+ }
+ catch (KeeperException.NodeExistsException e) {
+ if (log.isDebugEnabled())
+ log.debug("Node already exists: " + path);
+
+ return path;
+ }
+ catch (Exception e) {
+ onZookeeperError(connStartTime, e);
+ }
+ }
+ }
+
+ /**
+ * @param checkPrefix Unique prefix to check in case of retry.
+ * @param parentPath Parent node path.
+ * @param path Node to create.
+ * @param data Node data.
+ * @param createMode Create mode.
+ * @return Create path.
+ * @throws ZookeeperClientFailedException If connection to zk was lost.
+ * @throws InterruptedException If interrupted.
+ */
+ String createSequential(String checkPrefix, String parentPath, String path, byte[] data, CreateMode createMode)
+ throws ZookeeperClientFailedException, InterruptedException
+ {
+ assert createMode.isSequential() : createMode;
+
+ if (data == null)
+ data = EMPTY_BYTES;
+
+ boolean first = true;
+
+ for (;;) {
+ long connStartTime = this.connStartTime;
+
+ try {
+ if (!first) {
+ List<String> children = zk.getChildren(parentPath, false);
+
+ for (int i = 0; i < children.size(); i++) {
+ String child = children.get(i);
+
+ if (children.get(i).startsWith(checkPrefix)) {
+ String resPath = parentPath + "/" + child;
+
+ if (log.isDebugEnabled())
+ log.debug("Check before retry, node already created: " + resPath);
+
+ return resPath;
+ }
+ }
+ }
+
+ return zk.create(path, data, ZK_ACL, createMode);
+ }
+ catch (KeeperException.NodeExistsException e) {
+ assert !createMode.isSequential() : createMode;
+
+ if (log.isDebugEnabled())
+ log.debug("Node already exists: " + path);
+
+ return path;
+ }
+ catch (Exception e) {
+ onZookeeperError(connStartTime, e);
+ }
+
+ first = false;
+ }
+ }
+
+ /**
+ * @param path Path.
+ * @return Children nodes.
+ * @throws ZookeeperClientFailedException If connection to zk was lost.
+ * @throws InterruptedException If interrupted.
+ */
+ List<String> getChildren(String path)
+ throws ZookeeperClientFailedException, InterruptedException
+ {
+ for (;;) {
+ long connStartTime = this.connStartTime;
+
+ try {
+ return zk.getChildren(path, false);
+ }
+ catch (Exception e) {
+ onZookeeperError(connStartTime, e);
+ }
+ }
+ }
+
+ /**
+ * @param path Path.
+ * @throws InterruptedException If interrupted.
+ * @throws KeeperException In case of error.
+ * @return {@code True} if given path exists.
+ */
+ boolean existsNoRetry(String path) throws InterruptedException, KeeperException {
+ return zk.exists(path, false) != null;
+ }
+
+ /**
+ * @param path Path.
+ * @param ver Expected version.
+ * @throws InterruptedException If interrupted.
+ * @throws KeeperException In case of error.
+ */
+ void deleteIfExistsNoRetry(String path, int ver) throws InterruptedException, KeeperException {
+ try {
+ zk.delete(path, ver);
+ }
+ catch (KeeperException.NoNodeException e) {
+ // No-op if znode does not exist.
+ }
+ }
+
+ /**
+ * @param path Path.
+ * @param ver Version.
+ * @throws ZookeeperClientFailedException If connection to zk was lost.
+ * @throws InterruptedException If interrupted.
+ */
+ void deleteIfExists(String path, int ver)
+ throws ZookeeperClientFailedException, InterruptedException
+ {
+ try {
+ delete(path, ver);
+ }
+ catch (KeeperException.NoNodeException e) {
+ // No-op if znode does not exist.
+ }
+ }
+
+ /**
+ * @param parent Parent path.
+ * @param paths Children paths.
+ * @param ver Version.
+ * @throws KeeperException.NoNodeException If at least one of nodes does not exist.
+ * @throws ZookeeperClientFailedException If connection to zk was lost.
+ * @throws InterruptedException If interrupted.
+ */
+ void deleteAll(@Nullable String parent, List<String> paths, int ver)
+ throws KeeperException.NoNodeException, ZookeeperClientFailedException, InterruptedException
+ {
+ if (paths.isEmpty())
+ return;
+
+ // TODO ZK: https://issues.apache.org/jira/browse/IGNITE-8188
+ List<Op> ops = new ArrayList<>(paths.size());
+
+ for (String path : paths) {
+ String path0 = parent != null ? parent + "/" + path : path;
+
+ ops.add(Op.delete(path0, ver));
+ }
+
+ for (;;) {
+ long connStartTime = this.connStartTime;
+
+ try {
+ zk.multi(ops);
+
+ return;
+ }
+ catch (KeeperException.NoNodeException e) {
+ throw e;
+ }
+ catch (Exception e) {
+ onZookeeperError(connStartTime, e);
+ }
+ }
+ }
+
+ /**
+ * @param path Path.
+ * @param ver Version.
+ * @throws KeeperException.NoNodeException If target node does not exist.
+ * @throws ZookeeperClientFailedException If connection to zk was lost.
+ * @throws InterruptedException If interrupted.
+ */
+ private void delete(String path, int ver)
+ throws KeeperException.NoNodeException, ZookeeperClientFailedException, InterruptedException
+ {
+ for (;;) {
+ long connStartTime = this.connStartTime;
+
+ try {
+ zk.delete(path, ver);
+
+ return;
+ }
+ catch (KeeperException.NoNodeException e) {
+ throw e;
+ }
+ catch (Exception e) {
+ onZookeeperError(connStartTime, e);
+ }
+ }
+ }
+
+ /**
+ * @param path Path.
+ * @param data Data.
+ * @param ver Version.
+ * @throws ZookeeperClientFailedException If connection to zk was lost.
+ * @throws InterruptedException If interrupted.
+ * @throws KeeperException.NoNodeException If node does not exist.
+ * @throws KeeperException.BadVersionException If version does not match.
+ */
+ void setData(String path, byte[] data, int ver)
+ throws ZookeeperClientFailedException, InterruptedException, KeeperException.NoNodeException,
+ KeeperException.BadVersionException
+ {
+ if (data == null)
+ data = EMPTY_BYTES;
+
+ for (;;) {
+ long connStartTime = this.connStartTime;
+
+ try {
+ zk.setData(path, data, ver);
+
+ return;
+ }
+ catch (KeeperException.BadVersionException | KeeperException.NoNodeException e) {
+ throw e;
+ }
+ catch (Exception e) {
+ onZookeeperError(connStartTime, e);
+ }
+ }
+ }
+
+ /**
+ * @param path Path.
+ * @param stat Optional {@link Stat} instance to return znode state.
+ * @return Data.
+ * @throws KeeperException.NoNodeException If target node does not exist.
+ * @throws ZookeeperClientFailedException If connection to zk was lost.
+ * @throws InterruptedException If interrupted.
+ */
+ byte[] getData(String path, @Nullable Stat stat)
+ throws KeeperException.NoNodeException, ZookeeperClientFailedException, InterruptedException {
+ for (;;) {
+ long connStartTime = this.connStartTime;
+
+ try {
+ return zk.getData(path, false, stat);
+ }
+ catch (KeeperException.NoNodeException e) {
+ throw e;
+ }
+ catch (Exception e) {
+ onZookeeperError(connStartTime, e);
+ }
+ }
+ }
+
+ /**
+ * @param path Path.
+ * @return Data.
+ * @throws KeeperException.NoNodeException If target node does not exist.
+ * @throws ZookeeperClientFailedException If connection to zk was lost.
+ * @throws InterruptedException If interrupted.
+ */
+ byte[] getData(String path)
+ throws KeeperException.NoNodeException, ZookeeperClientFailedException, InterruptedException
+ {
+ return getData(path, null);
+ }
+
+ /**
+ * @param path Path.
+ */
+ void deleteIfExistsAsync(String path) {
+ new DeleteIfExistsOperation(path).execute();
+ }
+
+ /**
+ * @param path Path.
+ * @param watcher Watcher.
+ * @param cb Callback.
+ */
+ void existsAsync(String path, Watcher watcher, AsyncCallback.StatCallback cb) {
+ ExistsOperation op = new ExistsOperation(path, watcher, cb);
+
+ zk.exists(path, watcher, new StatCallbackWrapper(op), null);
+ }
+
+ /**
+ * @param path Path.
+ * @param watcher Watcher.
+ * @param cb Callback.
+ */
+ void getChildrenAsync(String path, Watcher watcher, AsyncCallback.Children2Callback cb) {
+ GetChildrenOperation op = new GetChildrenOperation(path, watcher, cb);
+
+ zk.getChildren(path, watcher, new ChildrenCallbackWrapper(op), null);
+ }
+
+ /**
+ * @param path Path.
+ * @param watcher Watcher.
+ * @param cb Callback.
+ */
+ void getDataAsync(String path, Watcher watcher, AsyncCallback.DataCallback cb) {
+ GetDataOperation op = new GetDataOperation(path, watcher, cb);
+
+ zk.getData(path, watcher, new DataCallbackWrapper(op), null);
+ }
+
+ /**
+ * @param path Path.
+ * @param data Data.
+ * @param createMode Create mode.
+ * @param cb Callback.
+ */
+ private void createAsync(String path, byte[] data, CreateMode createMode, AsyncCallback.StringCallback cb) {
+ if (data == null)
+ data = EMPTY_BYTES;
+
+ CreateOperation op = new CreateOperation(path, data, createMode, cb);
+
+ zk.create(path, data, ZK_ACL, createMode, new CreateCallbackWrapper(op), null);
+ }
+
+ /**
+ *
+ */
+ void onCloseStart() {
+ closing = true;
+
+ synchronized (stateMux) {
+ stateMux.notifyAll();
+ }
+ }
+
+ /**
+ *
+ */
+ public void close() {
+ closeClient();
+ }
+
+ /**
+ * @param prevConnStartTime Time when connection was established.
+ * @param e Error.
+ * @throws ZookeeperClientFailedException If connection to zk was lost.
+ * @throws InterruptedException If interrupted.
+ */
+ private void onZookeeperError(long prevConnStartTime, Exception e)
+ throws ZookeeperClientFailedException, InterruptedException
+ {
+ ZookeeperClientFailedException err = null;
+
+ synchronized (stateMux) {
+ if (closing)
+ throw new ZookeeperClientFailedException("ZooKeeper client is closed.");
+
+ U.warn(log, "Failed to execute ZooKeeper operation [err=" + e + ", state=" + state + ']');
+
+ if (state == ConnectionState.Lost) {
+ U.error(log, "Operation failed with unexpected error, connection lost: " + e, e);
+
+ throw new ZookeeperClientFailedException(e);
+ }
+
+ boolean retry = (e instanceof KeeperException) && needRetry(((KeeperException)e).code().intValue());
+
+ if (retry) {
+ long remainingTime;
+
+ if (state == ConnectionState.Connected && connStartTime == prevConnStartTime) {
+ state = ConnectionState.Disconnected;
+
+ connStartTime = System.currentTimeMillis();
+
+ remainingTime = connLossTimeout;
+ }
+ else {
+ assert connStartTime != 0;
+
+ assert state == ConnectionState.Disconnected : state;
+
+ remainingTime = connLossTimeout - (System.currentTimeMillis() - connStartTime);
+
+ if (remainingTime <= 0) {
+ state = ConnectionState.Lost;
+
+ U.warn(log, "Failed to establish ZooKeeper connection, close client " +
+ "[timeout=" + connLossTimeout + ']');
+
+ err = new ZookeeperClientFailedException(e);
+ }
+ }
+
+ if (err == null) {
+ U.warn(log, "ZooKeeper operation failed, will retry [err=" + e +
+ ", retryTimeout=" + RETRY_TIMEOUT +
+ ", connLossTimeout=" + connLossTimeout +
+ ", path=" + ((KeeperException)e).getPath() +
+ ", remainingWaitTime=" + remainingTime + ']');
+
+ stateMux.wait(RETRY_TIMEOUT);
+
+ if (closing)
+ throw new ZookeeperClientFailedException("ZooKeeper client is closed.");
+ }
+ }
+ else {
+ U.error(log, "Operation failed with unexpected error, close ZooKeeper client: " + e, e);
+
+ state = ConnectionState.Lost;
+
+ err = new ZookeeperClientFailedException(e);
+ }
+ }
+
+ if (err != null) {
+ closeClient();
+
+ notifyConnectionLost();
+
+ throw err;
+ }
+ }
+
+ /**
+ * @param code Zookeeper error code.
+ * @return {@code True} if can retry operation.
+ */
+ private boolean needRetry(int code) {
+ boolean retryByErrorCode = code == KeeperException.Code.CONNECTIONLOSS.intValue() ||
+ code == KeeperException.Code.SESSIONMOVED.intValue() ||
+ code == KeeperException.Code.OPERATIONTIMEOUT.intValue();
+
+ if (retryByErrorCode) {
+ if (MAX_RETRY_COUNT <= 0 || retryCount.incrementAndGet() < MAX_RETRY_COUNT)
+ return true;
+ else
+ return false;
+ }
+ else
+ return false;
+ }
+
+ /**
+ *
+ */
+ private void closeClient() {
+ try {
+ zk.close();
+ }
+ catch (Exception closeErr) {
+ U.warn(log, "Failed to close ZooKeeper client: " + closeErr, closeErr);
+ }
+
+ connTimer.cancel();
+ }
+
+ /**
+ *
+ */
+ private void scheduleConnectionCheck() {
+ assert state == ConnectionState.Disconnected : state;
+
+ connTimer.schedule(new ConnectionTimeoutTask(connStartTime), connLossTimeout);
+ }
+
+ /**
+ *
+ */
+ interface ZkAsyncOperation {
+ /**
+ *
+ */
+ void execute();
+ }
+
+ /**
+ *
+ */
+ class GetChildrenOperation implements ZkAsyncOperation {
+ /** */
+ private final String path;
+
+ /** */
+ private final Watcher watcher;
+
+ /** */
+ private final AsyncCallback.Children2Callback cb;
+
+ /**
+ * @param path Path.
+ * @param watcher Watcher.
+ * @param cb Callback.
+ */
+ GetChildrenOperation(String path, Watcher watcher, AsyncCallback.Children2Callback cb) {
+ this.path = path;
+ this.watcher = watcher;
+ this.cb = cb;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void execute() {
+ getChildrenAsync(path, watcher, cb);
+ }
+ }
+
+ /**
+ *
+ */
+ class GetDataOperation implements ZkAsyncOperation {
+ /** */
+ private final String path;
+
+ /** */
+ private final Watcher watcher;
+
+ /** */
+ private final AsyncCallback.DataCallback cb;
+
+ /**
+ * @param path Path.
+ * @param watcher Watcher.
+ * @param cb Callback.
+ */
+ GetDataOperation(String path, Watcher watcher, AsyncCallback.DataCallback cb) {
+ this.path = path;
+ this.watcher = watcher;
+ this.cb = cb;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void execute() {
+ getDataAsync(path, watcher, cb);
+ }
+ }
+
+ /**
+ *
+ */
+ class ExistsOperation implements ZkAsyncOperation {
+ /** */
+ private final String path;
+
+ /** */
+ private final Watcher watcher;
+
+ /** */
+ private final AsyncCallback.StatCallback cb;
+
+ /**
+ * @param path Path.
+ * @param watcher Watcher.
+ * @param cb Callback.
+ */
+ ExistsOperation(String path, Watcher watcher, AsyncCallback.StatCallback cb) {
+ this.path = path;
+ this.watcher = watcher;
+ this.cb = cb;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void execute() {
+ existsAsync(path, watcher, cb);
+ }
+ }
+
+ /**
+ *
+ */
+ class CreateOperation implements ZkAsyncOperation {
+ /** */
+ private final String path;
+
+ /** */
+ private final byte[] data;
+
+ /** */
+ private final CreateMode createMode;
+
+ /** */
+ private final AsyncCallback.StringCallback cb;
+
+ /**
+ * @param path path.
+ * @param data Data.
+ * @param createMode Create mode.
+ * @param cb Callback.
+ */
+ CreateOperation(String path, byte[] data, CreateMode createMode, AsyncCallback.StringCallback cb) {
+ this.path = path;
+ this.data = data;
+ this.createMode = createMode;
+ this.cb = cb;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void execute() {
+ createAsync(path, data, createMode, cb);
+ }
+ }
+
+ /**
+ *
+ */
+ class DeleteIfExistsOperation implements AsyncCallback.VoidCallback, ZkAsyncOperation {
+ /** */
+ private final String path;
+
+ /**
+ * @param path Path.
+ */
+ DeleteIfExistsOperation(String path) {
+ this.path = path;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void execute() {
+ zk.delete(path, -1, this, null);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void processResult(int rc, String path, Object ctx) {
+ if (closing)
+ return;
+
+ if (rc == KeeperException.Code.NONODE.intValue())
+ return;
+
+ if (needRetry(rc)) {
+ U.warn(log, "Failed to execute async operation, connection lost. Will retry after connection restore [" +
+ "path=" + path + ']');
+
+ retryQ.add(this);
+ }
+ else if (rc == KeeperException.Code.SESSIONEXPIRED.intValue())
+ U.warn(log, "Failed to execute async operation, connection lost [path=" + path + ']');
+ else
+ assert rc == 0 : KeeperException.Code.get(rc);
+ }
+ }
+
+ /**
+ *
+ */
+ class CreateCallbackWrapper implements AsyncCallback.StringCallback {
+ /** */
+ final CreateOperation op;
+
+ /**
+ * @param op Operation.
+ */
+ CreateCallbackWrapper(CreateOperation op) {
+ this.op = op;
+ }
+
+ @Override public void processResult(int rc, String path, Object ctx, String name) {
+ if (closing)
+ return;
+
+ if (rc == KeeperException.Code.NODEEXISTS.intValue())
+ return;
+
+ if (needRetry(rc)) {
+ U.warn(log, "Failed to execute async operation, connection lost. Will retry after connection restore [path=" + path + ']');
+
+ retryQ.add(op);
+ }
+ else if (rc == KeeperException.Code.SESSIONEXPIRED.intValue())
+ U.warn(log, "Failed to execute async operation, connection lost [path=" + path + ']');
+ else {
+ if (op.cb != null)
+ op.cb.processResult(rc, path, ctx, name);
+ }
+ }
+ }
+
+ /**
+ *
+ */
+ class ChildrenCallbackWrapper implements AsyncCallback.Children2Callback {
+ /** */
+ private final GetChildrenOperation op;
+
+ /**
+ * @param op Operation.
+ */
+ private ChildrenCallbackWrapper(GetChildrenOperation op) {
+ this.op = op;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void processResult(int rc, String path, Object ctx, List<String> children, Stat stat) {
+ if (closing)
+ return;
+
+ if (needRetry(rc)) {
+ U.warn(log, "Failed to execute async operation, connection lost. Will retry after connection restore [path=" + path + ']');
+
+ retryQ.add(op);
+ }
+ else if (rc == KeeperException.Code.SESSIONEXPIRED.intValue())
+ U.warn(log, "Failed to execute async operation, connection lost [path=" + path + ']');
+ else
+ op.cb.processResult(rc, path, ctx, children, stat);
+ }
+ }
+
+ /**
+ *
+ */
+ class DataCallbackWrapper implements AsyncCallback.DataCallback {
+ /** */
+ private final GetDataOperation op;
+
+ /**
+ * @param op Operation.
+ */
+ private DataCallbackWrapper(GetDataOperation op) {
+ this.op = op;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat) {
+ if (closing)
+ return;
+
+ if (needRetry(rc)) {
+ U.warn(log, "Failed to execute async operation, connection lost. Will retry after connection restore [path=" + path + ']');
+
+ retryQ.add(op);
+ }
+ else if (rc == KeeperException.Code.SESSIONEXPIRED.intValue())
+ U.warn(log, "Failed to execute async operation, connection lost [path=" + path + ']');
+ else
+ op.cb.processResult(rc, path, ctx, data, stat);
+ }
+ }
+
+ /**
+ *
+ */
+ class StatCallbackWrapper implements AsyncCallback.StatCallback {
+ /** */
+ private final ExistsOperation op;
+
+ /**
+ * @param op Operation.
+ */
+ private StatCallbackWrapper(ExistsOperation op) {
+ this.op = op;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void processResult(int rc, String path, Object ctx, Stat stat) {
+ if (closing)
+ return;
+
+ if (needRetry(rc)) {
+ U.warn(log, "Failed to execute async operation, connection lost. Will retry after connection restore [path=" + path + ']');
+
+ retryQ.add(op);
+ }
+ else if (rc == KeeperException.Code.SESSIONEXPIRED.intValue())
+ U.warn(log, "Failed to execute async operation, connection lost [path=" + path + ']');
+ else
+ op.cb.processResult(rc, path, ctx, stat);
+ }
+ }
+
+ /**
+ *
+ */
+ private class ConnectionTimeoutTask extends TimerTask {
+ /** */
+ private final long connectStartTime;
+
+ /**
+ * @param connectStartTime Time was connection started.
+ */
+ ConnectionTimeoutTask(long connectStartTime) {
+ this.connectStartTime = connectStartTime;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void run() {
+ boolean connLoss = false;
+
+ synchronized (stateMux) {
+ if (closing)
+ return;
+
+ if (state == ConnectionState.Disconnected &&
+ ZookeeperClient.this.connStartTime == connectStartTime) {
+
+ state = ConnectionState.Lost;
+
+ U.warn(log, "Failed to establish ZooKeeper connection, close client " +
+ "[timeout=" + connLossTimeout + ']');
+
+ connLoss = true;
+ }
+ }
+
+ if (connLoss) {
+ closeClient();
+
+ notifyConnectionLost();
+ }
+ }
+ }
+
+ /**
+ *
+ */
+ private enum ConnectionState {
+ /** */
+ Connected,
+ /** */
+ Disconnected,
+ /** */
+ Lost
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClientFailedException.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClientFailedException.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClientFailedException.java
new file mode 100644
index 0000000..01d011b
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClientFailedException.java
@@ -0,0 +1,40 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+/**
+ *
+ */
+class ZookeeperClientFailedException extends Exception {
+ /** */
+ private static final long serialVersionUID = 0L;
+
+ /**
+ * @param msg Message.
+ */
+ ZookeeperClientFailedException(String msg) {
+ super(msg);
+ }
+
+ /**
+ * @param cause Cause.
+ */
+ ZookeeperClientFailedException(Throwable cause) {
+ super(cause);
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClusterNode.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClusterNode.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClusterNode.java
new file mode 100644
index 0000000..3cb5fad
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClusterNode.java
@@ -0,0 +1,362 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.cache.CacheMetrics;
+import org.apache.ignite.cluster.ClusterMetrics;
+import org.apache.ignite.internal.IgniteNodeAttributes;
+import org.apache.ignite.internal.managers.discovery.IgniteClusterNode;
+import org.apache.ignite.internal.util.tostring.GridToStringExclude;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.lang.IgniteProductVersion;
+import org.apache.ignite.spi.discovery.DiscoveryMetricsProvider;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_DAEMON;
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_NODE_CONSISTENT_ID;
+
+/**
+ * Zookeeper Cluster Node.
+ */
+public class ZookeeperClusterNode implements IgniteClusterNode, Serializable, Comparable<ZookeeperClusterNode> {
+ /** */
+ private static final long serialVersionUID = 0L;
+
+ /** */
+ private static final byte CLIENT_NODE_MASK = 0x01;
+
+ /** */
+ private UUID id;
+
+ /** */
+ private Serializable consistentId;
+
+ /** */
+ private long internalId;
+
+ /** */
+ private long order;
+
+ /** */
+ private IgniteProductVersion ver;
+
+ /** Node attributes. */
+ private Map<String, Object> attrs;
+
+ /** Internal discovery addresses as strings. */
+ private Collection<String> addrs;
+
+ /** Internal discovery host names as strings. */
+ private Collection<String> hostNames;
+
+ /** */
+ private long sesTimeout;
+
+ /** Metrics provider. */
+ private transient DiscoveryMetricsProvider metricsProvider;
+
+ /** */
+ private transient boolean loc;
+
+ /** */
+ private transient volatile ClusterMetrics metrics;
+
+ /** Node cache metrics. */
+ @GridToStringExclude
+ private transient volatile Map<Integer, CacheMetrics> cacheMetrics;
+
+ /** */
+ private byte flags;
+
+ /** Daemon node flag. */
+ @GridToStringExclude
+ private transient boolean daemon;
+
+ /** Daemon node initialization flag. */
+ @GridToStringExclude
+ private transient volatile boolean daemonInit;
+
+ /**
+ * @param id Node ID.
+ * @param addrs Node addresses.
+ * @param hostNames Node host names.
+ * @param ver Node version.
+ * @param attrs Node attributes.
+ * @param consistentId Consistent ID.
+ * @param sesTimeout Zookeeper session timeout.
+ * @param client Client node flag.
+ * @param metricsProvider Metrics provider.
+ */
+ public ZookeeperClusterNode(
+ UUID id,
+ Collection<String> addrs,
+ Collection<String> hostNames,
+ IgniteProductVersion ver,
+ Map<String, Object> attrs,
+ Serializable consistentId,
+ long sesTimeout,
+ boolean client,
+ DiscoveryMetricsProvider metricsProvider
+ ) {
+ assert id != null;
+ assert consistentId != null;
+
+ this.id = id;
+ this.ver = ver;
+ this.attrs = Collections.unmodifiableMap(attrs);
+ this.addrs = addrs;
+ this.hostNames = hostNames;
+ this.consistentId = consistentId;
+ this.sesTimeout = sesTimeout;
+ this.metricsProvider = metricsProvider;
+
+ if (client)
+ flags |= CLIENT_NODE_MASK;
+ }
+
+ /** {@inheritDoc} */
+ @Override public UUID id() {
+ return id;
+ }
+
+ /** {@inheritDoc} */
+ @Override public Object consistentId() {
+ return consistentId;
+ }
+
+ /** {@inheritDoc} */
+ public void setConsistentId(Serializable consistentId) {
+ this.consistentId = consistentId;
+
+ final Map<String, Object> map = new HashMap<>(attrs);
+
+ map.put(ATTR_NODE_CONSISTENT_ID, consistentId);
+
+ attrs = Collections.unmodifiableMap(map);
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean isCacheClient() {
+ return isClient();
+ }
+
+ /** {@inheritDoc} */
+ @Nullable @Override public <T> T attribute(String name) {
+ // Even though discovery SPI removes this attribute after authentication, keep this check for safety.
+ if (IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS.equals(name))
+ return null;
+
+ return (T)attrs.get(name);
+ }
+
+ /**
+ * Sets node attributes.
+ *
+ * @param attrs Node attributes.
+ */
+ void setAttributes(Map<String, Object> attrs) {
+ this.attrs = U.sealMap(attrs);
+ }
+
+ /**
+ * Gets node attributes without filtering.
+ *
+ * @return Node attributes without filtering.
+ */
+ Map<String, Object> getAttributes() {
+ return attrs;
+ }
+
+ /** {@inheritDoc} */
+ @Override public ClusterMetrics metrics() {
+ if (metricsProvider != null) {
+ ClusterMetrics metrics0 = metricsProvider.metrics();
+
+ assert metrics0 != null;
+
+ metrics = metrics0;
+
+ return metrics0;
+ }
+
+ return metrics;
+ }
+
+ /** {@inheritDoc} */
+ public void setMetrics(ClusterMetrics metrics) {
+ assert metrics != null;
+
+ this.metrics = metrics;
+ }
+
+ /** {@inheritDoc} */
+ @Override public Map<Integer, CacheMetrics> cacheMetrics() {
+ if (metricsProvider != null) {
+ Map<Integer, CacheMetrics> cacheMetrics0 = metricsProvider.cacheMetrics();
+
+ cacheMetrics = cacheMetrics0;
+
+ return cacheMetrics0;
+ }
+
+ return cacheMetrics;
+ }
+
+ /** {@inheritDoc} */
+ public void setCacheMetrics(Map<Integer, CacheMetrics> cacheMetrics) {
+ this.cacheMetrics = cacheMetrics != null ? cacheMetrics : Collections.<Integer, CacheMetrics>emptyMap();
+ }
+
+ /** {@inheritDoc} */
+ @Override public Map<String, Object> attributes() {
+ // Even though discovery SPI removes this attribute after authentication, keep this check for safety.
+ return F.view(attrs, new IgnitePredicate<String>() {
+ @Override public boolean apply(String s) {
+ return !IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS.equals(s);
+ }
+ });
+ }
+
+ /** {@inheritDoc} */
+ @Override public Collection<String> addresses() {
+ return addrs;
+ }
+
+ /** {@inheritDoc} */
+ @Override public Collection<String> hostNames() {
+ return hostNames;
+ }
+
+ /** {@inheritDoc} */
+ @Override public long order() {
+ return order;
+ }
+
+ /**
+ * @return Internal ID corresponds to Zookeeper sequential node.
+ */
+ long internalId() {
+ return internalId;
+ }
+
+ /**
+ * @param internalId Internal ID corresponds to Zookeeper sequential node.
+ */
+ void internalId(long internalId) {
+ this.internalId = internalId;
+ }
+
+ /**
+ * @param order Node order.
+ */
+ void order(long order) {
+ assert order > 0 : order;
+
+ this.order = order;
+ }
+
+ /**
+ * @param newId New node ID.
+ */
+ public void onClientDisconnected(UUID newId) {
+ id = newId;
+ }
+
+ /**
+ * @return Session timeout.
+ */
+ long sessionTimeout() {
+ return sesTimeout;
+ }
+
+ /** {@inheritDoc} */
+ @Override public IgniteProductVersion version() {
+ return ver;
+ }
+
+ /**
+ * @param loc Local node flag.
+ */
+ public void local(boolean loc) {
+ this.loc = loc;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean isLocal() {
+ return loc;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean isDaemon() {
+ if (!daemonInit) {
+ daemon = "true".equalsIgnoreCase((String)attribute(ATTR_DAEMON));
+
+ daemonInit = true;
+ }
+
+ return daemon;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean isClient() {
+ return (CLIENT_NODE_MASK & flags) != 0;
+ }
+
+ /** {@inheritDoc} */
+ @Override public int compareTo(@Nullable ZookeeperClusterNode node) {
+ if (node == null)
+ return 1;
+
+ int res = Long.compare(order, node.order);
+
+ if (res == 0) {
+ assert id().equals(node.id()) : "Duplicate order [this=" + this + ", other=" + node + ']';
+
+ res = id().compareTo(node.id());
+ }
+
+ return res;
+ }
+
+ /** {@inheritDoc} */
+ @Override public int hashCode() {
+ return id.hashCode();
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean equals(Object obj) {
+ return F.eqNodes(this, obj);
+ }
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return "ZookeeperClusterNode [id=" + id +
+ ", addrs=" + addrs +
+ ", order=" + order +
+ ", loc=" + loc +
+ ", client=" + isClient() + ']';
+ }
+}
[08/12] ignite git commit: IGNITE-7222 Added ZooKeeper discovery SPI
Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectFailoverAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectFailoverAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectFailoverAbstractTest.java
index 3e98051..37292ff 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectFailoverAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectFailoverAbstractTest.java
@@ -29,6 +29,7 @@ import org.apache.ignite.Ignite;
import org.apache.ignite.IgniteClientDisconnectedException;
import org.apache.ignite.configuration.IgniteConfiguration;
import org.apache.ignite.events.Event;
+import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi;
import org.apache.ignite.internal.util.typedef.internal.U;
import org.apache.ignite.lang.IgnitePredicate;
import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
@@ -87,9 +88,9 @@ public abstract class IgniteClientReconnectFailoverAbstractTest extends IgniteCl
assertTrue(client.cluster().localNode().isClient());
- Ignite srv = clientRouter(client);
+ Ignite srv = ignite(0);
- TestTcpDiscoverySpi srvSpi = spi(srv);
+ IgniteDiscoverySpi srvSpi = spi0(srv);
final AtomicBoolean stop = new AtomicBoolean(false);
@@ -209,14 +210,17 @@ public abstract class IgniteClientReconnectFailoverAbstractTest extends IgniteCl
}
if (err != null) {
- log.error(err);
+ log.error("Test error: " + err);
U.dumpThreads(log);
CyclicBarrier barrier0 = barrier;
- if (barrier0 != null)
+ if (barrier0 != null) {
+ barrier = null;
+
barrier0.reset();
+ }
stop.set(true);
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectServicesTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectServicesTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectServicesTest.java
index 3e961e5..1e6dd64 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectServicesTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectServicesTest.java
@@ -65,7 +65,7 @@ public class IgniteClientReconnectServicesTest extends IgniteClientReconnectAbst
assertEquals((Object)topVer, srvc.test());
- Ignite srv = clientRouter(client);
+ Ignite srv = ignite(0);
reconnectClientNode(client, srv, null);
@@ -88,7 +88,7 @@ public class IgniteClientReconnectServicesTest extends IgniteClientReconnectAbst
assertTrue(client.cluster().localNode().isClient());
- Ignite srv = clientRouter(client);
+ Ignite srv = ignite(0);
IgniteServices clnServices = client.services();
@@ -132,7 +132,7 @@ public class IgniteClientReconnectServicesTest extends IgniteClientReconnectAbst
final IgniteServices services = client.services();
- Ignite srv = clientRouter(client);
+ Ignite srv = ignite(0);
BlockTcpCommunicationSpi commSpi = commSpi(srv);
@@ -179,7 +179,7 @@ public class IgniteClientReconnectServicesTest extends IgniteClientReconnectAbst
final IgniteServices services = client.services();
- final Ignite srv = clientRouter(client);
+ final Ignite srv = ignite(0);
services.deployClusterSingleton("testReconnectInProgress", new TestServiceImpl());
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectStopTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectStopTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectStopTest.java
index e863cdf..b5c3ee8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectStopTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectStopTest.java
@@ -23,8 +23,10 @@ import org.apache.ignite.IgniteClientDisconnectedException;
import org.apache.ignite.IgniteException;
import org.apache.ignite.configuration.CacheConfiguration;
import org.apache.ignite.events.Event;
+import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi;
import org.apache.ignite.lang.IgniteFuture;
import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.spi.discovery.DiscoverySpi;
import static org.apache.ignite.events.EventType.EVT_CLIENT_NODE_DISCONNECTED;
import static org.apache.ignite.events.EventType.EVT_CLIENT_NODE_RECONNECTED;
@@ -50,15 +52,19 @@ public class IgniteClientReconnectStopTest extends IgniteClientReconnectAbstract
Ignite srv = clientRouter(client);
- TestTcpDiscoverySpi srvSpi = spi(srv);
+ DiscoverySpi srvSpi = spi0(srv);
final CountDownLatch disconnectLatch = new CountDownLatch(1);
final CountDownLatch reconnectLatch = new CountDownLatch(1);
- final TestTcpDiscoverySpi clientSpi = spi(client);
+ final IgniteDiscoverySpi clientSpi = spi0(client);
+
+ DiscoverySpiTestListener lsnr = new DiscoverySpiTestListener();
+
+ clientSpi.setInternalListener(lsnr);
log.info("Block reconnect.");
- clientSpi.writeLatch = new CountDownLatch(1);
+ lsnr.startBlockJoin();
client.events().localListen(new IgnitePredicate<Event>() {
@Override public boolean apply(Event evt) {
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectStreamerTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectStreamerTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectStreamerTest.java
index 3959feb..36b9890 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectStreamerTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectStreamerTest.java
@@ -71,7 +71,7 @@ public class IgniteClientReconnectStreamerTest extends IgniteClientReconnectAbst
assertTrue(client.cluster().localNode().isClient());
- Ignite srv = clientRouter(client);
+ Ignite srv = ignite(0);
final IgniteCache<Object, Object> srvCache = srv.cache(CACHE_NAME);
@@ -135,7 +135,7 @@ public class IgniteClientReconnectStreamerTest extends IgniteClientReconnectAbst
assertTrue(client.cluster().localNode().isClient());
- Ignite srv = clientRouter(client);
+ Ignite srv = ignite(0);
final IgniteCache<Object, Object> srvCache = srv.cache(CACHE_NAME);
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientRejoinTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientRejoinTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientRejoinTest.java
index a5d42e9..8edbb52 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientRejoinTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientRejoinTest.java
@@ -256,6 +256,9 @@ public class IgniteClientRejoinTest extends GridCommonAbstractTest {
Ignite srv1 = startGrid("server1");
+ if (!tcpDiscovery())
+ return;
+
crd = ((IgniteKernal)srv1).localNode();
Ignite srv2 = startGrid("server2");
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
index a8afa8b..8fad640 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
@@ -178,16 +178,20 @@ public class GridDiscoveryManagerAliveCacheSelfTest extends GridCommonAbstractTe
* Waits while topology on all nodes became equals to the expected size.
*
* @param nodesCnt Expected nodes count.
- * @throws InterruptedException If interrupted.
+ * @throws Exception If interrupted.
*/
@SuppressWarnings("BusyWait")
- private void awaitDiscovery(long nodesCnt) throws InterruptedException {
- for (Ignite g : alive) {
- ((TcpDiscoverySpi)g.configuration().getDiscoverySpi()).waitForClientMessagePrecessed();
+ private void awaitDiscovery(int nodesCnt) throws Exception {
+ if (tcpDiscovery()) {
+ for (Ignite g : alive) {
+ ((TcpDiscoverySpi)g.configuration().getDiscoverySpi()).waitForClientMessagePrecessed();
- while (g.cluster().nodes().size() != nodesCnt)
- Thread.sleep(10);
+ while (g.cluster().nodes().size() != nodesCnt)
+ Thread.sleep(10);
+ }
}
+ else
+ waitForTopology(nodesCnt);
}
/**
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessorAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessorAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessorAbstractSelfTest.java
index 1d70246..aa2abae 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessorAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessorAbstractSelfTest.java
@@ -91,12 +91,12 @@ public abstract class GridAffinityProcessorAbstractSelfTest extends GridCommonAb
@Override protected void beforeTestsStarted() throws Exception {
assert NODES_CNT >= 1;
- withCache = false;
+ withCache = true;
for (int i = 0; i < NODES_CNT; i++)
startGrid(i);
- withCache = true;
+ withCache = false;
for (int i = NODES_CNT; i < 2 * NODES_CNT; i++)
startGrid(i);
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMetricsForClusterGroupSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMetricsForClusterGroupSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMetricsForClusterGroupSelfTest.java
index aefbc23..b8f9d70 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMetricsForClusterGroupSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMetricsForClusterGroupSelfTest.java
@@ -27,9 +27,9 @@ import org.apache.ignite.configuration.CacheConfiguration;
import org.apache.ignite.configuration.IgniteConfiguration;
import org.apache.ignite.events.Event;
import org.apache.ignite.events.EventType;
+import org.apache.ignite.internal.managers.discovery.IgniteClusterNode;
import org.apache.ignite.lang.IgniteClosure;
import org.apache.ignite.lang.IgnitePredicate;
-import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode;
import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
import static org.apache.ignite.events.EventType.EVT_NODE_METRICS_UPDATED;
@@ -103,7 +103,7 @@ public class CacheMetricsForClusterGroupSelfTest extends GridCommonAbstractTest
Collection<ClusterNode> nodes = grid(0).cluster().forRemotes().nodes();
for (ClusterNode node : nodes) {
- Map<Integer, CacheMetrics> metrics = ((TcpDiscoveryNode)node).cacheMetrics();
+ Map<Integer, CacheMetrics> metrics = ((IgniteClusterNode)node).cacheMetrics();
assertNotNull(metrics);
assertFalse(metrics.isEmpty());
}
@@ -118,6 +118,8 @@ public class CacheMetricsForClusterGroupSelfTest extends GridCommonAbstractTest
/**
* Test cluster group metrics in case of statistics disabled.
+ *
+ * @throws Exception If failed.
*/
public void testMetricsStatisticsDisabled() throws Exception {
createCaches(false);
@@ -134,7 +136,7 @@ public class CacheMetricsForClusterGroupSelfTest extends GridCommonAbstractTest
Collection<ClusterNode> nodes = grid(0).cluster().forRemotes().nodes();
for (ClusterNode node : nodes) {
- Map<Integer, CacheMetrics> metrics = ((TcpDiscoveryNode) node).cacheMetrics();
+ Map<Integer, CacheMetrics> metrics = ((IgniteClusterNode)node).cacheMetrics();
assertNotNull(metrics);
assertTrue(metrics.isEmpty());
}
@@ -172,7 +174,9 @@ public class CacheMetricsForClusterGroupSelfTest extends GridCommonAbstractTest
}
/**
- * Wait for {@link EventType#EVT_NODE_METRICS_UPDATED} event will be receieved.
+ * Wait for {@link EventType#EVT_NODE_METRICS_UPDATED} event will be received.
+ *
+ * @throws InterruptedException If interrupted.
*/
private void awaitMetricsUpdate() throws InterruptedException {
final CountDownLatch latch = new CountDownLatch((GRID_CNT + 1) * 2);
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
index df93ae4..299dbf4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
@@ -115,6 +115,8 @@ public abstract class GridCacheAbstractSelfTest extends GridCommonAbstractTest {
protected void initStoreStrategy() throws IgniteCheckedException {
if (storeStgy == null)
storeStgy = isMultiJvm() ? new H2CacheStoreStrategy() : new MapCacheStoreStrategy();
+ else if (isMultiJvm() && !(storeStgy instanceof H2CacheStoreStrategy))
+ storeStgy = new H2CacheStoreStrategy();
}
/** {@inheritDoc} */
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNearLockValueSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNearLockValueSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNearLockValueSelfTest.java
index 0069110..c135f2d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNearLockValueSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNearLockValueSelfTest.java
@@ -45,7 +45,9 @@ public class IgniteCacheNearLockValueSelfTest extends GridCommonAbstractTest {
/** {@inheritDoc} */
@Override protected void beforeTestsStarted() throws Exception {
- startGridsMultiThreaded(2);
+ startGrid(1);
+
+ startGrid(0);
}
/** {@inheritDoc} */
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
index f321453..55ff31a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
@@ -67,6 +67,17 @@ public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTes
}
/** {@inheritDoc} */
+ @Override protected void startGrids() throws Exception {
+ int cnt = gridCount();
+
+ assert cnt >= 1 : "At least one grid must be started";
+
+ startGridsMultiThreaded(1, cnt - 1);
+
+ startGrid(0);
+ }
+
+ /** {@inheritDoc} */
@Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java
index 2337329..838e56d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java
@@ -38,6 +38,7 @@ import org.apache.ignite.internal.IgniteClientReconnectAbstractTest;
import org.apache.ignite.internal.IgniteEx;
import org.apache.ignite.internal.IgniteInternalFuture;
import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi;
import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsFullMessage;
import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsSingleMessage;
@@ -1109,6 +1110,70 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
}
/**
+ * @throws Exception If failed.
+ */
+ public void testActivateFailover3() throws Exception {
+ stateChangeFailover3(true);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testDeactivateFailover3() throws Exception {
+ stateChangeFailover3(false);
+ }
+
+ /**
+ * @param activate If {@code true} tests activation, otherwise deactivation.
+ * @throws Exception If failed.
+ */
+ private void stateChangeFailover3(boolean activate) throws Exception {
+ testReconnectSpi = true;
+
+ startNodesAndBlockStatusChange(4, 0, 0, !activate);
+
+ client = false;
+
+ IgniteInternalFuture startFut1 = GridTestUtils.runAsync(new Callable() {
+ @Override public Object call() throws Exception {
+ startGrid(4);
+
+ return null;
+ }
+ }, "start-node1");
+
+ IgniteInternalFuture startFut2 = GridTestUtils.runAsync(new Callable() {
+ @Override public Object call() throws Exception {
+ startGrid(5);
+
+ return null;
+ }
+ }, "start-node2");
+
+ U.sleep(1000);
+
+ // Stop all nodes participating in state change and not allow last node to finish exchange.
+ for (int i = 0; i < 4; i++)
+ ((IgniteDiscoverySpi)ignite(i).configuration().getDiscoverySpi()).simulateNodeFailure();
+
+ for (int i = 0; i < 4; i++)
+ stopGrid(getTestIgniteInstanceName(i), true, false);
+
+ startFut1.get();
+ startFut2.get();
+
+ assertFalse(ignite(4).active());
+ assertFalse(ignite(5).active());
+
+ ignite(4).active(true);
+
+ for (int i = 0; i < 4; i++)
+ startGrid(i);
+
+ checkCaches1(6);
+ }
+
+ /**
* @param exp If {@code true} there should be recorded messages.
*/
private void checkRecordedMessages(boolean exp) {
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDaemonNodeMarshallerCacheTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDaemonNodeMarshallerCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDaemonNodeMarshallerCacheTest.java
index 566860d..2f9bd53 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDaemonNodeMarshallerCacheTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDaemonNodeMarshallerCacheTest.java
@@ -79,7 +79,7 @@ public class IgniteDaemonNodeMarshallerCacheTest extends GridCommonAbstractTest
* @param startFirst If {@code true} daemon node is started first.
* @throws Exception If failed.
*/
- public void marshalOnDaemonNode(boolean startFirst) throws Exception {
+ private void marshalOnDaemonNode(boolean startFirst) throws Exception {
int nodeIdx = 0;
if (!startFirst) {
@@ -92,6 +92,7 @@ public class IgniteDaemonNodeMarshallerCacheTest extends GridCommonAbstractTest
Ignite daemonNode = startGrid(nodeIdx++);
+ assertTrue(daemonNode.cluster().localNode().isDaemon());
assertEquals("true", daemonNode.cluster().localNode().attribute(ATTR_DAEMON));
daemon = false;
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataUpdatesFlowTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataUpdatesFlowTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataUpdatesFlowTest.java
index 3ee51c8..7e8c086 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataUpdatesFlowTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataUpdatesFlowTest.java
@@ -184,6 +184,7 @@ public class BinaryMetadataUpdatesFlowTest extends GridCommonAbstractTest {
* Starts new ignite node and submits computation job to it.
* @param idx Index.
* @param stopFlag Stop flag.
+ * @throws Exception If failed.
*/
private void startComputation(int idx, AtomicBoolean stopFlag) throws Exception {
clientMode = false;
@@ -199,6 +200,7 @@ public class BinaryMetadataUpdatesFlowTest extends GridCommonAbstractTest {
* @param idx Index.
* @param deafClient Deaf client.
* @param observedIds Observed ids.
+ * @throws Exception If failed.
*/
private void startListening(int idx, boolean deafClient, Set<Integer> observedIds) throws Exception {
clientMode = true;
@@ -269,7 +271,7 @@ public class BinaryMetadataUpdatesFlowTest extends GridCommonAbstractTest {
}
/**
- *
+ * @throws Exception If failed.
*/
public void testFlowNoConflicts() throws Exception {
startComputation(0, stopFlag0);
@@ -311,11 +313,14 @@ public class BinaryMetadataUpdatesFlowTest extends GridCommonAbstractTest {
}
/**
- *
+ * @throws Exception If failed.
*/
public void testFlowNoConflictsWithClients() throws Exception {
startComputation(0, stopFlag0);
+ if (!tcpDiscovery())
+ return;
+
startComputation(1, stopFlag1);
startComputation(2, stopFlag2);
@@ -617,6 +622,9 @@ public class BinaryMetadataUpdatesFlowTest extends GridCommonAbstractTest {
while (!updatesQueue.isEmpty()) {
BinaryUpdateDescription desc = updatesQueue.poll();
+ if (desc == null)
+ break;
+
BinaryObjectBuilder builder = ignite.binary().builder(BINARY_TYPE_NAME);
BinaryObject bo = newBinaryObject(builder, desc);
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheClientNodeBinaryObjectMetadataMultinodeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheClientNodeBinaryObjectMetadataMultinodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheClientNodeBinaryObjectMetadataMultinodeTest.java
index 313aaf9..81614cb 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheClientNodeBinaryObjectMetadataMultinodeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheClientNodeBinaryObjectMetadataMultinodeTest.java
@@ -242,7 +242,7 @@ public class GridCacheClientNodeBinaryObjectMetadataMultinodeTest extends GridCo
@Override public boolean apply() {
Collection<BinaryType> metaCol = p0.types();
- return metaCol.size() == 1000;
+ return metaCol.size() >= 1000;
}
}, getTestTimeout());
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueClientDisconnectTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueClientDisconnectTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueClientDisconnectTest.java
index ed54377..dac3ff6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueClientDisconnectTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueClientDisconnectTest.java
@@ -32,6 +32,9 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
import org.apache.ignite.testframework.GridTestUtils;
import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+/**
+ *
+ */
public class GridCacheQueueClientDisconnectTest extends GridCommonAbstractTest {
/** */
private static final String IGNITE_QUEUE_NAME = "ignite-queue-client-reconnect-test";
@@ -66,6 +69,10 @@ public class GridCacheQueueClientDisconnectTest extends GridCommonAbstractTest {
return cfg;
}
+ /**
+ * @param cacheAtomicityMode Atomicity mode.
+ * @return Configuration.
+ */
private static CollectionConfiguration collectionConfiguration(CacheAtomicityMode cacheAtomicityMode) {
CollectionConfiguration colCfg = new CollectionConfiguration();
@@ -74,6 +81,9 @@ public class GridCacheQueueClientDisconnectTest extends GridCommonAbstractTest {
return colCfg;
}
+ /**
+ * @throws Exception If failed.
+ */
public void testClientDisconnect() throws Exception {
try {
Ignite server = startGrid(0);
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java
index 51764b5..d85201a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java
@@ -494,7 +494,8 @@ public abstract class IgniteClientDataStructuresAbstractTest extends GridCommonA
assertTrue(ignite.configuration().isClientMode());
- assertEquals(clientDiscovery(), ignite.configuration().getDiscoverySpi().isClientMode());
+ if (tcpDiscovery())
+ assertEquals(clientDiscovery(), ignite.configuration().getDiscoverySpi().isClientMode());
return ignite;
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentTest.java
index 0704dbd..e456047 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentTest.java
@@ -52,6 +52,7 @@ import org.apache.ignite.configuration.DataRegionConfiguration;
import org.apache.ignite.configuration.DataStorageConfiguration;
import org.apache.ignite.configuration.IgniteConfiguration;
import org.apache.ignite.events.DiscoveryEvent;
+import org.apache.ignite.internal.DiscoverySpiTestListener;
import org.apache.ignite.internal.GridKernalContext;
import org.apache.ignite.internal.cluster.NodeOrderComparator;
import org.apache.ignite.internal.cluster.NodeOrderLegacyComparator;
@@ -60,7 +61,7 @@ import org.apache.ignite.internal.IgniteKernal;
import org.apache.ignite.internal.IgniteNodeAttributes;
import org.apache.ignite.internal.TestRecordingCommunicationSpi;
import org.apache.ignite.internal.cluster.ClusterTopologyServerNotFoundException;
-import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
+import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi;
import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
import org.apache.ignite.internal.processors.affinity.GridAffinityFunctionContextImpl;
import org.apache.ignite.internal.processors.cache.CacheAffinityChangeMessage;
@@ -88,7 +89,6 @@ import org.apache.ignite.plugin.extensions.communication.Message;
import org.apache.ignite.resources.IgniteInstanceResource;
import org.apache.ignite.services.Service;
import org.apache.ignite.services.ServiceContext;
-import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
@@ -158,7 +158,7 @@ public class CacheLateAffinityAssignmentTest extends GridCommonAbstractTest {
cfg.setCommunicationSpi(commSpi);
- TestTcpDiscoverySpi discoSpi = new TestTcpDiscoverySpi();
+ TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
discoSpi.setForceServerMode(forceSrvMode);
discoSpi.setIpFinder(ipFinder);
@@ -674,9 +674,11 @@ public class CacheLateAffinityAssignmentTest extends GridCommonAbstractTest {
checkAffinity(4, topVer(4, 0), true);
- TestTcpDiscoverySpi discoSpi = (TestTcpDiscoverySpi)ignite0.configuration().getDiscoverySpi();
+ DiscoverySpiTestListener lsnr = new DiscoverySpiTestListener();
- discoSpi.blockCustomEvent();
+ ((IgniteDiscoverySpi)ignite0.configuration().getDiscoverySpi()).setInternalListener(lsnr);
+
+ lsnr.blockCustomEvent(CacheAffinityChangeMessage.class);
stopGrid(1);
@@ -687,7 +689,7 @@ public class CacheLateAffinityAssignmentTest extends GridCommonAbstractTest {
for (IgniteInternalFuture<?> fut : futs)
assertFalse(fut.isDone());
- discoSpi.stopBlock();
+ lsnr.stopBlockCustomEvents();
checkAffinity(3, topVer(5, 0), false);
@@ -1409,8 +1411,10 @@ public class CacheLateAffinityAssignmentTest extends GridCommonAbstractTest {
public void testDelayAssignmentAffinityChanged() throws Exception {
Ignite ignite0 = startServer(0, 1);
- TestTcpDiscoverySpi discoSpi0 =
- (TestTcpDiscoverySpi)ignite0.configuration().getDiscoverySpi();
+ DiscoverySpiTestListener lsnr = new DiscoverySpiTestListener();
+
+ ((IgniteDiscoverySpi)ignite0.configuration().getDiscoverySpi()).setInternalListener(lsnr);
+
TestRecordingCommunicationSpi commSpi0 =
(TestRecordingCommunicationSpi)ignite0.configuration().getCommunicationSpi();
@@ -1418,19 +1422,19 @@ public class CacheLateAffinityAssignmentTest extends GridCommonAbstractTest {
checkAffinity(2, topVer(2, 0), true);
- discoSpi0.blockCustomEvent();
+ lsnr.blockCustomEvent(CacheAffinityChangeMessage.class);
startServer(2, 3);
checkAffinity(3, topVer(3, 0), false);
- discoSpi0.waitCustomEvent();
+ lsnr.waitCustomEvent();
blockSupplySend(commSpi0, CACHE_NAME1);
startServer(3, 4);
- discoSpi0.stopBlock();
+ lsnr.stopBlockCustomEvents();
checkAffinity(4, topVer(4, 0), false);
@@ -1452,8 +1456,10 @@ public class CacheLateAffinityAssignmentTest extends GridCommonAbstractTest {
try {
Ignite ignite0 = startServer(0, 1);
- TestTcpDiscoverySpi discoSpi0 =
- (TestTcpDiscoverySpi)ignite0.configuration().getDiscoverySpi();
+ DiscoverySpiTestListener lsnr = new DiscoverySpiTestListener();
+
+ ((IgniteDiscoverySpi)ignite0.configuration().getDiscoverySpi()).setInternalListener(lsnr);
+
TestRecordingCommunicationSpi commSpi0 =
(TestRecordingCommunicationSpi)ignite0.configuration().getCommunicationSpi();
@@ -1465,11 +1471,11 @@ public class CacheLateAffinityAssignmentTest extends GridCommonAbstractTest {
checkAffinity(3, topVer(3, 1), false);
- discoSpi0.blockCustomEvent();
+ lsnr.blockCustomEvent(CacheAffinityChangeMessage.class);
stopNode(2, 4);
- discoSpi0.waitCustomEvent();
+ lsnr.waitCustomEvent();
blockSupplySend(commSpi0, CACHE_NAME1);
@@ -1483,7 +1489,7 @@ public class CacheLateAffinityAssignmentTest extends GridCommonAbstractTest {
Thread.sleep(2_000);
- discoSpi0.stopBlock();
+ lsnr.stopBlockCustomEvents();
boolean started = GridTestUtils.waitForCondition(new GridAbsPredicate() {
@Override public boolean apply() {
@@ -1534,14 +1540,16 @@ public class CacheLateAffinityAssignmentTest extends GridCommonAbstractTest {
ignite0.createCache(ccfg);
- TestTcpDiscoverySpi discoSpi0 =
- (TestTcpDiscoverySpi)ignite0.configuration().getDiscoverySpi();
+ DiscoverySpiTestListener lsnr = new DiscoverySpiTestListener();
+
+ ((IgniteDiscoverySpi)ignite0.configuration().getDiscoverySpi()).setInternalListener(lsnr);
+
TestRecordingCommunicationSpi spi =
(TestRecordingCommunicationSpi)ignite0.configuration().getCommunicationSpi();
blockSupplySend(spi, CACHE_NAME2);
- discoSpi0.blockCustomEvent();
+ lsnr.blockCustomEvent(CacheAffinityChangeMessage.class);
startServer(1, 2);
@@ -1551,7 +1559,7 @@ public class CacheLateAffinityAssignmentTest extends GridCommonAbstractTest {
spi.stopBlock();
- discoSpi0.waitCustomEvent();
+ lsnr.waitCustomEvent();
ignite0.destroyCache(CACHE_NAME2);
@@ -1561,7 +1569,7 @@ public class CacheLateAffinityAssignmentTest extends GridCommonAbstractTest {
ignite0.createCache(ccfg);
- discoSpi0.stopBlock();
+ lsnr.stopBlockCustomEvents();
checkAffinity(3, topVer(3, 1), false);
checkAffinity(3, topVer(3, 2), false);
@@ -2967,83 +2975,6 @@ public class CacheLateAffinityAssignmentTest extends GridCommonAbstractTest {
/**
*
*/
- static class TestTcpDiscoverySpi extends TcpDiscoverySpi {
- /** */
- private boolean blockCustomEvt;
-
- /** */
- private final Object mux = new Object();
-
- /** */
- private List<DiscoverySpiCustomMessage> blockedMsgs = new ArrayList<>();
-
- /** {@inheritDoc} */
- @Override public void sendCustomEvent(DiscoverySpiCustomMessage msg) throws IgniteException {
- synchronized (mux) {
- if (blockCustomEvt) {
- DiscoveryCustomMessage msg0 = GridTestUtils.getFieldValue(msg, "delegate");
-
- if (msg0 instanceof CacheAffinityChangeMessage) {
- log.info("Block custom message: " + msg0);
-
- blockedMsgs.add(msg);
-
- mux.notifyAll();
-
- return;
- }
- }
- }
-
- super.sendCustomEvent(msg);
- }
-
- /**
- *
- */
- public void blockCustomEvent() {
- synchronized (mux) {
- assert blockedMsgs.isEmpty() : blockedMsgs;
-
- blockCustomEvt = true;
- }
- }
-
- /**
- * @throws InterruptedException If interrupted.
- */
- public void waitCustomEvent() throws InterruptedException {
- synchronized (mux) {
- while (blockedMsgs.isEmpty())
- mux.wait();
- }
- }
-
- /**
- *
- */
- public void stopBlock() {
- List<DiscoverySpiCustomMessage> msgs;
-
- synchronized (this) {
- msgs = new ArrayList<>(blockedMsgs);
-
- blockCustomEvt = false;
-
- blockedMsgs.clear();
- }
-
- for (DiscoverySpiCustomMessage msg : msgs) {
- log.info("Resend blocked message: " + msg);
-
- super.sendCustomEvent(msg);
- }
- }
- }
-
- /**
- *
- */
static class TestEntryProcessor implements EntryProcessor<Object, Object, Object> {
/** */
private Object val;
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheNodeFailureAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheNodeFailureAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheNodeFailureAbstractTest.java
index 3834df9..5dea5d9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheNodeFailureAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheNodeFailureAbstractTest.java
@@ -45,6 +45,7 @@ import org.apache.ignite.transactions.TransactionIsolation;
import static org.apache.ignite.IgniteState.STOPPED;
import static org.apache.ignite.IgniteSystemProperties.IGNITE_TX_SALVAGE_TIMEOUT;
import static org.apache.ignite.IgniteSystemProperties.getInteger;
+import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC;
import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
@@ -188,7 +189,7 @@ public abstract class GridCacheNodeFailureAbstractTest extends GridCommonAbstrac
return true;
}
- }, EVT_NODE_LEFT);
+ }, EVT_NODE_LEFT, EVT_NODE_FAILED);
stopGrid(idx);
@@ -268,7 +269,7 @@ public abstract class GridCacheNodeFailureAbstractTest extends GridCommonAbstrac
return true;
}
- }, EVT_NODE_LEFT);
+ }, EVT_NODE_LEFT, EVT_NODE_FAILED);
stopGrid(idx);
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCache150ClientsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCache150ClientsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCache150ClientsTest.java
index e71d3ee..b7ae844 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCache150ClientsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCache150ClientsTest.java
@@ -168,6 +168,8 @@ public class IgniteCache150ClientsTest extends GridCommonAbstractTest {
log.info("Started all clients.");
+ waitForTopology(CLIENTS + 1);
+
checkNodes(CLIENTS + 1);
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheManyClientsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheManyClientsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheManyClientsTest.java
index a0be40e..7785a3c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheManyClientsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheManyClientsTest.java
@@ -178,7 +178,7 @@ public class IgniteCacheManyClientsTest extends GridCommonAbstractTest {
log.info("All clients started.");
try {
- checkNodes(SRVS + CLIENTS);
+ checkNodes0(SRVS + CLIENTS);
}
finally {
for (Ignite client : clients)
@@ -188,6 +188,30 @@ public class IgniteCacheManyClientsTest extends GridCommonAbstractTest {
/**
* @param expCnt Expected number of nodes.
+ * @throws Exception If failed.
+ */
+ private void checkNodes0(final int expCnt) throws Exception {
+ boolean wait = GridTestUtils.waitForCondition(new GridAbsPredicate() {
+ @Override public boolean apply() {
+ try {
+ checkNodes(expCnt);
+
+ return true;
+ }
+ catch (AssertionFailedError e) {
+ log.info("Check failed, will retry: " + e);
+ }
+
+ return false;
+ }
+ }, 10_000);
+
+ if (!wait)
+ checkNodes(expCnt);
+ }
+
+ /**
+ * @param expCnt Expected number of nodes.
*/
private void checkNodes(int expCnt) {
assertEquals(expCnt, G.allGrids().size());
@@ -297,23 +321,7 @@ public class IgniteCacheManyClientsTest extends GridCommonAbstractTest {
if (err0 != null)
throw err0;
- boolean wait = GridTestUtils.waitForCondition(new GridAbsPredicate() {
- @Override public boolean apply() {
- try {
- checkNodes(SRVS + THREADS);
-
- return true;
- }
- catch (AssertionFailedError e) {
- log.info("Check failed, will retry: " + e);
- }
-
- return false;
- }
- }, 10_000);
-
- if (!wait)
- checkNodes(SRVS + THREADS);
+ checkNodes0(SRVS + THREADS);
log.info("Stop clients.");
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteOptimisticTxSuspendResumeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteOptimisticTxSuspendResumeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteOptimisticTxSuspendResumeTest.java
index a55f21d..486fd60 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteOptimisticTxSuspendResumeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteOptimisticTxSuspendResumeTest.java
@@ -713,6 +713,8 @@ public class IgniteOptimisticTxSuspendResumeTest extends GridCommonAbstractTest
", backups=" + ccfg.getBackups() +
", near=" + (ccfg.getNearConfiguration() != null) + "]");
+ awaitPartitionMapExchange();
+
int srvNum = serversNumber();
if (serversNumber() > 1) {
ignite(serversNumber() + 1).createNearCache(ccfg.getName(), new NearCacheConfiguration<>());
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMultiThreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMultiThreadedSelfTest.java
index e3fa116..60f2f0a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMultiThreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMultiThreadedSelfTest.java
@@ -116,6 +116,8 @@ public class GridCacheDhtPreloadMultiThreadedSelfTest extends GridCommonAbstract
@Nullable @Override public Object call() throws Exception {
IgniteConfiguration cfg = loadConfiguration("modules/core/src/test/config/spring-multicache.xml");
+ cfg.setGridLogger(getTestResources().getLogger());
+
startGrid(Thread.currentThread().getName(), cfg);
return null;
@@ -161,6 +163,8 @@ public class GridCacheDhtPreloadMultiThreadedSelfTest extends GridCommonAbstract
@Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
IgniteConfiguration cfg = loadConfiguration("modules/core/src/test/config/spring-multicache.xml");
+ cfg.setGridLogger(getTestResources().getLogger());
+
cfg.setIgniteInstanceName(igniteInstanceName);
for (CacheConfiguration cCfg : cfg.getCacheConfiguration()) {
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
index 05a9759..83eff89 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
@@ -308,6 +308,8 @@ public class GridCacheDhtPreloadSelfTest extends GridCommonAbstractTest {
// Check all left nodes.
checkActiveState(ignites);
+
+ awaitPartitionMapExchange(); // Need wait, otherwise test logic is broken if EVT_NODE_FAILED exchanges are merged.
}
info("Waiting for preload futures: " + F.view(futs, new IgnitePredicate<IgniteFuture<?>>() {
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/TxRecoveryStoreEnabledTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/TxRecoveryStoreEnabledTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/TxRecoveryStoreEnabledTest.java
index 7b350c8..060af21 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/TxRecoveryStoreEnabledTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/TxRecoveryStoreEnabledTest.java
@@ -34,11 +34,11 @@ import org.apache.ignite.configuration.CacheConfiguration;
import org.apache.ignite.configuration.IgniteConfiguration;
import org.apache.ignite.internal.IgniteInternalFuture;
import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi;
import org.apache.ignite.internal.util.typedef.internal.U;
import org.apache.ignite.plugin.extensions.communication.Message;
import org.apache.ignite.spi.IgniteSpiException;
import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
-import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
import org.apache.ignite.transactions.Transaction;
import org.apache.ignite.transactions.TransactionConcurrency;
@@ -65,7 +65,6 @@ public class TxRecoveryStoreEnabledTest extends GridCommonAbstractTest {
IgniteConfiguration cfg = super.getConfiguration(gridName);
cfg.setCommunicationSpi(new TestCommunicationSpi());
- cfg.setDiscoverySpi(new TestDiscoverySpi());
CacheConfiguration ccfg = defaultCacheConfiguration();
@@ -126,7 +125,7 @@ public class TxRecoveryStoreEnabledTest extends GridCommonAbstractTest {
IgniteConfiguration cfg = node0.configuration();
((TestCommunicationSpi)cfg.getCommunicationSpi()).block();
- ((TestDiscoverySpi)cfg.getDiscoverySpi()).simulateNodeFailure();
+ ((IgniteDiscoverySpi)cfg.getDiscoverySpi()).simulateNodeFailure();
}
catch (InterruptedException e) {
Thread.currentThread().interrupt();
@@ -201,16 +200,6 @@ public class TxRecoveryStoreEnabledTest extends GridCommonAbstractTest {
/**
*
*/
- private static class TestDiscoverySpi extends TcpDiscoverySpi {
- /** {@inheritDoc} */
- @Override protected void simulateNodeFailure() {
- super.simulateNodeFailure();
- }
- }
-
- /**
- *
- */
private static class TestCommunicationSpi extends TcpCommunicationSpi {
/** Block. */
private volatile boolean block;
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java
index 3c57957..96fb8f6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java
@@ -37,6 +37,7 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
/**
@@ -125,7 +126,7 @@ public class GridCachePartitionedExplicitLockNodeFailureSelfTest extends GridCom
return true;
}
- }, EVT_NODE_LEFT));
+ }, EVT_NODE_LEFT, EVT_NODE_FAILED));
}
stopGrid(idx);
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ClientReconnectContinuousQueryTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ClientReconnectContinuousQueryTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ClientReconnectContinuousQueryTest.java
index c8b3bb6..9b531c6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ClientReconnectContinuousQueryTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ClientReconnectContinuousQueryTest.java
@@ -17,6 +17,7 @@
package org.apache.ignite.internal.processors.cache.query.continuous;
+import java.util.concurrent.Callable;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import javax.cache.event.CacheEntryListenerException;
@@ -28,6 +29,7 @@ import org.apache.ignite.configuration.IgniteConfiguration;
import org.apache.ignite.events.Event;
import org.apache.ignite.events.EventType;
import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
import org.apache.ignite.internal.managers.communication.GridIoManager;
import org.apache.ignite.internal.util.nio.GridNioServer;
import org.apache.ignite.internal.util.typedef.internal.U;
@@ -90,7 +92,7 @@ public class ClientReconnectContinuousQueryTest extends GridCommonAbstractTest {
try {
startGrids(2);
- IgniteEx client = grid(CLIENT_IDX);
+ final IgniteEx client = grid(CLIENT_IDX);
client.events().localListen(new DisconnectListener(), EventType.EVT_CLIENT_NODE_DISCONNECTED);
@@ -112,11 +114,19 @@ public class ClientReconnectContinuousQueryTest extends GridCommonAbstractTest {
skipRead(client, true);
- putSomeKeys(1_000);
+ IgniteInternalFuture<?> fut = GridTestUtils.runAsync(new Callable<Void>() {
+ @Override public Void call() throws Exception {
+ assertTrue(disconLatch.await(10_000, TimeUnit.MILLISECONDS));
+
+ skipRead(client, false);
- assertTrue(disconLatch.await(10_000, TimeUnit.MILLISECONDS));
+ return null;
+ }
+ });
- skipRead(client, false);
+ putSomeKeys(1_000);
+
+ fut.get();
assertTrue(reconLatch.await(10_000, TimeUnit.MILLISECONDS));
@@ -129,7 +139,6 @@ public class ClientReconnectContinuousQueryTest extends GridCommonAbstractTest {
finally {
stopAllGrids();
}
-
}
/**
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ContinuousQueryRemoteFilterMissingInClassPathSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ContinuousQueryRemoteFilterMissingInClassPathSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ContinuousQueryRemoteFilterMissingInClassPathSelfTest.java
index 92c1760..226302f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ContinuousQueryRemoteFilterMissingInClassPathSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ContinuousQueryRemoteFilterMissingInClassPathSelfTest.java
@@ -104,7 +104,7 @@ public class ContinuousQueryRemoteFilterMissingInClassPathSelfTest extends GridC
setExternalLoader = true;
final Ignite ignite0 = startGrid(1);
- executeContiniouseQuery(ignite0.cache("simple"));
+ executeContinuousQuery(ignite0.cache("simple"));
log = new GridStringLogger();
clientMode = true;
@@ -112,8 +112,10 @@ public class ContinuousQueryRemoteFilterMissingInClassPathSelfTest extends GridC
startGrid(2);
- assertTrue(log.toString().contains("Failed to unmarshal continuous query remote filter on client node. " +
- "Can be ignored."));
+ String logStr = log.toString();
+
+ assertTrue(logStr.contains("Failed to unmarshal continuous query remote filter on client node. " +
+ "Can be ignored.") || logStr.contains("Failed to unmarshal continuous routine handler"));
}
/**
@@ -127,7 +129,7 @@ public class ContinuousQueryRemoteFilterMissingInClassPathSelfTest extends GridC
clientMode = false;
final Ignite ignite0 = startGrid(1);
- executeContiniouseQuery(ignite0.cache("simple"));
+ executeContinuousQuery(ignite0.cache("simple"));
log = new GridStringLogger();
clientMode = true;
@@ -149,15 +151,18 @@ public class ContinuousQueryRemoteFilterMissingInClassPathSelfTest extends GridC
setExternalLoader = true;
final Ignite ignite0 = startGrid(1);
- executeContiniouseQuery(ignite0.cache("simple"));
+ executeContinuousQuery(ignite0.cache("simple"));
log = new GridStringLogger();
setExternalLoader = false;
startGrid(2);
- assertTrue(log.toString().contains("class org.apache.ignite.IgniteCheckedException: " +
- "Failed to find class with given class loader for unmarshalling"));
+ String logStr = log.toString();
+
+ assertTrue(logStr.contains("class org.apache.ignite.IgniteCheckedException: " +
+ "Failed to find class with given class loader for unmarshalling")
+ || logStr.contains("Failed to unmarshal continuous routine handler"));
}
/**
@@ -171,7 +176,7 @@ public class ContinuousQueryRemoteFilterMissingInClassPathSelfTest extends GridC
setExternalLoader = true;
final Ignite ignite0 = startGrid(1);
- executeContiniouseQuery(ignite0.cache("simple"));
+ executeContinuousQuery(ignite0.cache("simple"));
log = new GridStringLogger();
@@ -185,7 +190,7 @@ public class ContinuousQueryRemoteFilterMissingInClassPathSelfTest extends GridC
* @param cache Ignite cache.
* @throws Exception If fail.
*/
- private void executeContiniouseQuery(IgniteCache cache) throws Exception {
+ private void executeContinuousQuery(IgniteCache cache) throws Exception {
ContinuousQuery<Integer, String> qry = new ContinuousQuery<>();
qry.setLocalListener(
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryClientReconnectTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryClientReconnectTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryClientReconnectTest.java
index 9ad6d4e..906cc7d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryClientReconnectTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryClientReconnectTest.java
@@ -119,6 +119,9 @@ public class IgniteCacheContinuousQueryClientReconnectTest extends IgniteClientR
* @throws Exception If failed.
*/
public void testReconnectClientAndLeftRouter() throws Exception {
+ if (!tcpDiscovery())
+ return;
+
Ignite client = grid(serverCount());
final Ignite srv = clientRouter(client);
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryAbstractTest.java
index 61ceef7..16ea848 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryAbstractTest.java
@@ -18,11 +18,12 @@
package org.apache.ignite.internal.processors.cache.version;
import java.util.HashSet;
+import java.util.Map;
import java.util.Set;
-import java.util.concurrent.atomic.AtomicInteger;
import javax.cache.Cache;
import javax.cache.processor.EntryProcessor;
import javax.cache.processor.EntryProcessorException;
+import javax.cache.processor.EntryProcessorResult;
import javax.cache.processor.MutableEntry;
import org.apache.ignite.IgniteCache;
import org.apache.ignite.cache.CacheEntry;
@@ -56,23 +57,15 @@ public abstract class CacheVersionedEntryAbstractTest extends GridCacheAbstractS
public void testInvoke() throws Exception {
Cache<Integer, String> cache = grid(0).cache(DEFAULT_CACHE_NAME);
- final AtomicInteger invoked = new AtomicInteger();
-
- cache.invoke(100, new EntryProcessor<Integer, String, Object>() {
- @Override public Object process(MutableEntry<Integer, String> entry, Object... arguments)
- throws EntryProcessorException {
-
- invoked.incrementAndGet();
-
+ assertNotNull(cache.invoke(100, new EntryProcessor<Integer, String, Object>() {
+ @Override public Object process(MutableEntry<Integer, String> entry, Object... args) {
CacheEntry<Integer, String> verEntry = entry.unwrap(CacheEntry.class);
checkVersionedEntry(verEntry);
- return entry;
+ return verEntry.version();
}
- });
-
- assert invoked.get() > 0;
+ }));
}
/**
@@ -86,23 +79,17 @@ public abstract class CacheVersionedEntryAbstractTest extends GridCacheAbstractS
for (int i = 0; i < ENTRIES_NUM; i++)
keys.add(i);
- final AtomicInteger invoked = new AtomicInteger();
-
- cache.invokeAll(keys, new EntryProcessor<Integer, String, Object>() {
- @Override public Object process(MutableEntry<Integer, String> entry, Object... arguments)
- throws EntryProcessorException {
-
- invoked.incrementAndGet();
-
+ Map<Integer, EntryProcessorResult<Object>> res = cache.invokeAll(keys, new EntryProcessor<Integer, String, Object>() {
+ @Override public Object process(MutableEntry<Integer, String> entry, Object... args) {
CacheEntry<Integer, String> verEntry = entry.unwrap(CacheEntry.class);
checkVersionedEntry(verEntry);
- return null;
+ return verEntry.version();
}
});
- assert invoked.get() > 0;
+ assertEquals(ENTRIES_NUM, res.size());
}
/**
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java
index f07b1a3..1a7abd4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java
@@ -447,7 +447,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
grid(0).compute().broadcast(F.noop());
- assert latch.await(2, SECONDS);
+ assert latch.await(10, SECONDS) : latch;
assertEquals(GRID_CNT, nodeIds.size());
assertEquals(GRID_CNT, cnt.get());
@@ -488,7 +488,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
grid(0).compute().broadcast(F.noop());
- assert latch.await(2, SECONDS);
+ assert latch.await(10, SECONDS) : latch;
assertEquals(GRID_CNT, nodeIds.size());
assertEquals(GRID_CNT, cnt.get());
@@ -532,7 +532,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
grid(0).compute().broadcast(F.noop());
- assert latch.await(2, SECONDS);
+ assert latch.await(10, SECONDS) : latch;
assertEquals(GRID_CNT, nodeIds.size());
assertEquals(GRID_CNT, cnt.get());
@@ -578,7 +578,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
grid(0).compute().broadcast(F.noop());
grid(0).compute().withName("exclude").run(F.noop());
- assert latch.await(2, SECONDS);
+ assert latch.await(10, SECONDS) : latch;
assertEquals(GRID_CNT, nodeIds.size());
assertEquals(GRID_CNT, cnt.get());
@@ -619,7 +619,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
grid(0).compute().broadcast(F.noop());
- assert latch.await(2, SECONDS);
+ assert latch.await(10, SECONDS) : latch;
assertEquals(GRID_CNT - 1, nodeIds.size());
assertEquals(GRID_CNT - 1, cnt.get());
@@ -660,7 +660,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
grid(0).compute().broadcast(F.noop());
- assert latch.await(2, SECONDS);
+ assert latch.await(10, SECONDS) : latch;
assertEquals(GRID_CNT - 1, nodeIds.size());
assertEquals(GRID_CNT - 1, cnt.get());
@@ -701,7 +701,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
grid(0).compute().broadcast(F.noop());
- assert latch.await(2, SECONDS);
+ assert latch.await(10, SECONDS) : latch;
assertEquals(1, nodeIds.size());
assertEquals(1, cnt.get());
@@ -744,7 +744,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
grid(0).compute().broadcast(F.noop());
- assert latch.await(2, SECONDS);
+ assert latch.await(10, SECONDS) : latch;
assertEquals(1, nodeIds.size());
assertEquals(1, cnt.get());
@@ -785,7 +785,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
grid(0).compute().run(F.noop());
- assert latch.await(2, SECONDS);
+ assert latch.await(10, SECONDS) : latch;
assertEquals(1, nodeIds.size());
assertEquals(1, cnt.get());
@@ -828,7 +828,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
compute(grid(0).cluster().forLocal()).run(F.noop());
- assert latch.await(2, SECONDS);
+ assert latch.await(10, SECONDS) : latch;
assertEquals(1, cnt.get());
@@ -878,7 +878,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
grid(0).compute().broadcast(F.noop());
- assert latch.await(2, SECONDS);
+ assert latch.await(10, SECONDS) : latch;
assertEquals(GRID_CNT + 1, nodeIds.size());
assertEquals(GRID_CNT + 1, cnt.get());
@@ -929,7 +929,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
grid(0).compute().broadcast(F.noop());
- assert latch.await(2, SECONDS);
+ assert latch.await(10, SECONDS) : latch;
assertEquals(GRID_CNT, nodeIds.size());
assertEquals(GRID_CNT, cnt.get());
@@ -980,7 +980,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
grid(0).compute().broadcast(F.noop());
- assert latch.await(2, SECONDS);
+ assert latch.await(10, SECONDS) : latch;
assertEquals(GRID_CNT + 1, nodeIds.size());
assertEquals(GRID_CNT + 1, cnt.get());
@@ -1036,7 +1036,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
grid(0).compute().broadcast(F.noop());
- assert latch.await(2, SECONDS);
+ assert latch.await(10, SECONDS) : latch;
assertEquals(GRID_CNT, nodeIds.size());
assertEquals(GRID_CNT, cnt.get());
@@ -1145,7 +1145,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
/**
* @throws Exception If failed.
*/
- public void testMultithreadedWithNodeRestart() throws Exception {
+ public void _testMultithreadedWithNodeRestart() throws Exception {
final AtomicBoolean stop = new AtomicBoolean();
final BlockingQueue<IgniteBiTuple<Integer, UUID>> queue = new LinkedBlockingQueue<>();
final Collection<UUID> started = new GridConcurrentHashSet<>();
@@ -1153,9 +1153,11 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
final Random rnd = new Random();
+ final int consumeCnt = tcpDiscovery() ? CONSUME_CNT : CONSUME_CNT / 2;
+
IgniteInternalFuture<?> starterFut = multithreadedAsync(new Callable<Object>() {
@Override public Object call() throws Exception {
- for (int i = 0; i < CONSUME_CNT; i++) {
+ for (int i = 0; i < consumeCnt; i++) {
int idx = rnd.nextInt(GRID_CNT);
try {
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ClosureServiceClientsNodesTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ClosureServiceClientsNodesTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ClosureServiceClientsNodesTest.java
index c1af323..8f03c4c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ClosureServiceClientsNodesTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ClosureServiceClientsNodesTest.java
@@ -49,6 +49,9 @@ public class ClosureServiceClientsNodesTest extends GridCommonAbstractTest {
/** Number of grids started for tests. */
private static final int NODES_CNT = 4;
+ /** */
+ private static final int CLIENT_IDX = 1;
+
/** Test singleton service name. */
private static final String SINGLETON_NAME = "testSingleton";
@@ -61,11 +64,11 @@ public class ClosureServiceClientsNodesTest extends GridCommonAbstractTest {
cfg.setMarshaller(new BinaryMarshaller());
- cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(ipFinder).setForceServerMode(true));
+ ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
cfg.setCacheConfiguration();
- if (igniteInstanceName.equals(getTestIgniteInstanceName(0)))
+ if (igniteInstanceName.equals(getTestIgniteInstanceName(CLIENT_IDX)))
cfg.setClientMode(true);
return cfg;
@@ -88,8 +91,10 @@ public class ClosureServiceClientsNodesTest extends GridCommonAbstractTest {
public void testDefaultClosure() throws Exception {
Set<String> srvNames = new HashSet<>(NODES_CNT - 1);
- for (int i = 1; i < NODES_CNT; ++i)
- srvNames.add(getTestIgniteInstanceName(i));
+ for (int i = 0; i < NODES_CNT; ++i) {
+ if (i != CLIENT_IDX)
+ srvNames.add(getTestIgniteInstanceName(i));
+ }
for (int i = 0 ; i < NODES_CNT; i++) {
log.info("Iteration: " + i);
@@ -137,7 +142,7 @@ public class ClosureServiceClientsNodesTest extends GridCommonAbstractTest {
assertEquals(1, res.size());
- assertEquals(getTestIgniteInstanceName(0), F.first(res));
+ assertEquals(getTestIgniteInstanceName(CLIENT_IDX), F.first(res));
}
}
@@ -168,7 +173,7 @@ public class ClosureServiceClientsNodesTest extends GridCommonAbstractTest {
* @throws Exception If failed.
*/
public void testDefaultService() throws Exception {
- UUID clientNodeId = grid(0).cluster().localNode().id();
+ UUID clientNodeId = grid(CLIENT_IDX).cluster().localNode().id();
for (int i = 0 ; i < NODES_CNT; i++) {
log.info("Iteration: " + i);
@@ -209,7 +214,7 @@ public class ClosureServiceClientsNodesTest extends GridCommonAbstractTest {
* @throws Exception If failed.
*/
public void testClientService() throws Exception {
- UUID clientNodeId = grid(0).cluster().localNode().id();
+ UUID clientNodeId = grid(CLIENT_IDX).cluster().localNode().id();
for (int i = 0 ; i < NODES_CNT; i++) {
log.info("Iteration: " + i);
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/internal/util/GridTestClockTimer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/GridTestClockTimer.java b/modules/core/src/test/java/org/apache/ignite/internal/util/GridTestClockTimer.java
index 5da9042..2d26b72 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/GridTestClockTimer.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/GridTestClockTimer.java
@@ -32,6 +32,15 @@ public class GridTestClockTimer implements Runnable {
}
}
+ /**
+ * @return {@code True} if need start test time.
+ */
+ public static boolean startTestTimer() {
+ synchronized (IgniteUtils.mux) {
+ return IgniteUtils.gridCnt == 0;
+ }
+ }
+
/** {@inheritDoc} */
@Override public void run() {
while (true) {
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/marshaller/GridMarshallerMappingConsistencyTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/marshaller/GridMarshallerMappingConsistencyTest.java b/modules/core/src/test/java/org/apache/ignite/marshaller/GridMarshallerMappingConsistencyTest.java
index cd8e757..78f3c03 100644
--- a/modules/core/src/test/java/org/apache/ignite/marshaller/GridMarshallerMappingConsistencyTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/marshaller/GridMarshallerMappingConsistencyTest.java
@@ -30,6 +30,7 @@ import org.apache.ignite.internal.util.typedef.internal.U;
import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
/**
@@ -157,6 +158,9 @@ public class GridMarshallerMappingConsistencyTest extends GridCommonAbstractTest
Ignite g2 = startGrid(2);
startGrid(1);
+ assertTrue("Failed to wait for automatic grid activation",
+ GridTestUtils.waitForCondition(() -> g2.cluster().active(), getTestTimeout()));
+
IgniteCache<Integer, DummyObject> c2 = g2.cache(CACHE_NAME);
assertEquals(k, c2.get(k).val);
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/messaging/GridMessagingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/messaging/GridMessagingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/messaging/GridMessagingSelfTest.java
index 7541cec..a7c4521 100644
--- a/modules/core/src/test/java/org/apache/ignite/messaging/GridMessagingSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/messaging/GridMessagingSelfTest.java
@@ -24,7 +24,6 @@ import java.io.ObjectOutput;
import java.io.Serializable;
import java.net.URL;
import java.net.URLClassLoader;
-import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
@@ -37,22 +36,20 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteException;
import org.apache.ignite.IgniteMessaging;
import org.apache.ignite.cluster.ClusterGroup;
import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
+import org.apache.ignite.internal.DiscoverySpiTestListener;
+import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi;
import org.apache.ignite.internal.processors.continuous.StartRoutineDiscoveryMessage;
+import org.apache.ignite.internal.processors.continuous.StartRoutineDiscoveryMessageV2;
import org.apache.ignite.internal.processors.continuous.StopRoutineDiscoveryMessage;
-import org.apache.ignite.internal.processors.marshaller.MappingAcceptedMessage;
-import org.apache.ignite.internal.processors.marshaller.MappingProposedMessage;
import org.apache.ignite.internal.util.GridConcurrentHashSet;
import org.apache.ignite.internal.util.typedef.P2;
import org.apache.ignite.internal.util.typedef.PA;
import org.apache.ignite.internal.util.typedef.internal.U;
import org.apache.ignite.lang.IgniteFuture;
import org.apache.ignite.resources.IgniteInstanceResource;
-import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
@@ -206,11 +203,7 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest implements Ser
@Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
- TestTcpDiscoverySpi discoSpi = new TestTcpDiscoverySpi();
-
- discoSpi.setIpFinder(ipFinder);
-
- cfg.setDiscoverySpi(discoSpi);
+ ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
return cfg;
}
@@ -1036,7 +1029,11 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest implements Ser
public void testAsyncOld() throws Exception {
final AtomicInteger msgCnt = new AtomicInteger();
- TestTcpDiscoverySpi discoSpi = (TestTcpDiscoverySpi)ignite2.configuration().getDiscoverySpi();
+ IgniteDiscoverySpi discoSpi = (IgniteDiscoverySpi)ignite2.configuration().getDiscoverySpi();
+
+ DiscoverySpiTestListener lsnr = new DiscoverySpiTestListener();
+
+ discoSpi.setInternalListener(lsnr);
assertFalse(ignite2.message().isAsync());
@@ -1054,7 +1051,7 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest implements Ser
}
}, IllegalStateException.class, null);
- discoSpi.blockCustomEvent();
+ lsnr.blockCustomEvent(StartRoutineDiscoveryMessage.class, StartRoutineDiscoveryMessageV2.class);
final String topic = "topic";
@@ -1079,7 +1076,7 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest implements Ser
Assert.assertFalse(starFut.isDone());
- discoSpi.stopBlock();
+ lsnr.stopBlockCustomEvents();
GridTestUtils.assertThrows(log, new Callable<Void>() {
@Override public Void call() throws Exception {
@@ -1095,7 +1092,7 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest implements Ser
Assert.assertTrue(starFut.isDone());
- discoSpi.blockCustomEvent();
+ lsnr.blockCustomEvent(StopRoutineDiscoveryMessage.class);
message(ignite1.cluster().forRemotes()).send(topic, "msg1");
@@ -1125,7 +1122,7 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest implements Ser
Assert.assertFalse(stopFut.isDone());
- discoSpi.stopBlock();
+ lsnr.stopBlockCustomEvents();
stopFut.get();
@@ -1144,9 +1141,13 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest implements Ser
public void testAsync() throws Exception {
final AtomicInteger msgCnt = new AtomicInteger();
- TestTcpDiscoverySpi discoSpi = (TestTcpDiscoverySpi)ignite2.configuration().getDiscoverySpi();
+ IgniteDiscoverySpi discoSpi = (IgniteDiscoverySpi)ignite2.configuration().getDiscoverySpi();
- discoSpi.blockCustomEvent();
+ DiscoverySpiTestListener lsnr = new DiscoverySpiTestListener();
+
+ discoSpi.setInternalListener(lsnr);
+
+ lsnr.blockCustomEvent(StartRoutineDiscoveryMessage.class, StartRoutineDiscoveryMessageV2.class);
final String topic = "topic";
@@ -1167,7 +1168,7 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest implements Ser
Assert.assertFalse(starFut.isDone());
- discoSpi.stopBlock();
+ lsnr.stopBlockCustomEvents();
UUID id = starFut.get();
@@ -1175,7 +1176,7 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest implements Ser
Assert.assertTrue(starFut.isDone());
- discoSpi.blockCustomEvent();
+ lsnr.blockCustomEvent(StopRoutineDiscoveryMessage.class);
message(ignite1.cluster().forRemotes()).send(topic, "msg1");
@@ -1195,7 +1196,7 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest implements Ser
Assert.assertFalse(stopFut.isDone());
- discoSpi.stopBlock();
+ lsnr.stopBlockCustomEvents();
stopFut.get();
@@ -1209,89 +1210,6 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest implements Ser
}
/**
- *
- */
- static class TestTcpDiscoverySpi extends TcpDiscoverySpi {
- /** */
- private boolean blockCustomEvt;
-
- /** */
- private final Object mux = new Object();
-
- /** */
- private List<DiscoverySpiCustomMessage> blockedMsgs = new ArrayList<>();
-
- /** {@inheritDoc} */
- @Override public void sendCustomEvent(DiscoverySpiCustomMessage msg) throws IgniteException {
- synchronized (mux) {
- if (blockCustomEvt) {
- DiscoveryCustomMessage msg0 = GridTestUtils.getFieldValue(msg, "delegate");
-
- if (msg0 instanceof MappingProposedMessage || msg0 instanceof MappingAcceptedMessage){
- super.sendCustomEvent(msg);
-
- return;
- }
-
- if (msg0 instanceof StopRoutineDiscoveryMessage || msg0 instanceof StartRoutineDiscoveryMessage) {
- log.info("Block custom message: " + msg0);
-
- blockedMsgs.add(msg);
-
- mux.notifyAll();
-
- return;
- }
- }
- }
-
- super.sendCustomEvent(msg);
- }
-
- /**
- *
- */
- public void blockCustomEvent() {
- synchronized (mux) {
- assert blockedMsgs.isEmpty() : blockedMsgs;
-
- blockCustomEvt = true;
- }
- }
-
- /**
- * @throws InterruptedException If interrupted.
- */
- public void waitCustomEvent() throws InterruptedException {
- synchronized (mux) {
- while (blockedMsgs.isEmpty())
- mux.wait();
- }
- }
-
- /**
- *
- */
- public void stopBlock() {
- List<DiscoverySpiCustomMessage> msgs;
-
- synchronized (this) {
- msgs = new ArrayList<>(blockedMsgs);
-
- blockCustomEvt = false;
-
- blockedMsgs.clear();
- }
-
- for (DiscoverySpiCustomMessage msg : msgs) {
- log.info("Resend blocked message: " + msg);
-
- super.sendCustomEvent(msg);
- }
- }
- }
-
- /**
* Tests that message listener registers only for one oldest node.
*
* @throws Exception If an error occurred.
[02/12] ignite git commit: IGNITE-7222 Added ZooKeeper discovery SPI
Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiTest.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiTest.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiTest.java
new file mode 100644
index 0000000..fb12c3a
--- /dev/null
+++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiTest.java
@@ -0,0 +1,4847 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.BitSet;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.curator.test.TestingCluster;
+import org.apache.curator.test.TestingZooKeeperServer;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.CommunicationFailureContext;
+import org.apache.ignite.configuration.CommunicationFailureResolver;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.WALMode;
+import org.apache.ignite.events.DiscoveryEvent;
+import org.apache.ignite.events.Event;
+import org.apache.ignite.events.EventType;
+import org.apache.ignite.internal.DiscoverySpiTestListener;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.IgnitionEx;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.managers.discovery.CustomEventListener;
+import org.apache.ignite.internal.managers.discovery.DiscoCache;
+import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
+import org.apache.ignite.internal.managers.discovery.DiscoveryLocalJoinData;
+import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager;
+import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi;
+import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpiInternalListener;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cache.GridCacheAbstractFullApiSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.TestCacheNodeExcludingFilter;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessage;
+import org.apache.ignite.internal.processors.security.SecurityContext;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.IgniteFinishedFutureImpl;
+import org.apache.ignite.internal.util.lang.GridAbsPredicate;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.T3;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteBiPredicate;
+import org.apache.ignite.lang.IgniteCallable;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.lang.IgniteOutClosure;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.marshaller.jdk.JdkMarshaller;
+import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.plugin.security.SecurityCredentials;
+import org.apache.ignite.plugin.security.SecurityPermission;
+import org.apache.ignite.plugin.security.SecuritySubject;
+import org.apache.ignite.resources.IgniteInstanceResource;
+import org.apache.ignite.resources.LoggerResource;
+import org.apache.ignite.spi.IgniteSpiException;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+import org.apache.ignite.spi.discovery.DiscoverySpi;
+import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
+import org.apache.ignite.spi.discovery.DiscoverySpiNodeAuthenticator;
+import org.apache.ignite.spi.discovery.zk.ZookeeperDiscoverySpi;
+import org.apache.ignite.spi.discovery.zk.ZookeeperDiscoverySpiTestSuite2;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZKUtil;
+import org.apache.zookeeper.ZkTestClientCnxnSocketNIO;
+import org.apache.zookeeper.ZooKeeper;
+import org.jetbrains.annotations.Nullable;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+import static org.apache.ignite.events.EventType.EVT_CLIENT_NODE_DISCONNECTED;
+import static org.apache.ignite.events.EventType.EVT_CLIENT_NODE_RECONNECTED;
+import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
+import static org.apache.ignite.events.EventType.EVT_NODE_JOINED;
+import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_IGNITE_INSTANCE_NAME;
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS;
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_SECURITY_SUBJECT_V2;
+import static org.apache.ignite.spi.discovery.zk.internal.ZookeeperDiscoveryImpl.IGNITE_ZOOKEEPER_DISCOVERY_SPI_ACK_THRESHOLD;
+import static org.apache.zookeeper.ZooKeeper.ZOOKEEPER_CLIENT_CNXN_SOCKET;
+
+/**
+ *
+ */
+@SuppressWarnings("deprecation")
+public class ZookeeperDiscoverySpiTest extends GridCommonAbstractTest {
+ /** */
+ private static final String IGNITE_ZK_ROOT = ZookeeperDiscoverySpi.DFLT_ROOT_PATH;
+
+ /** */
+ private static final int ZK_SRVS = 3;
+
+ /** */
+ private static TestingCluster zkCluster;
+
+ /** To run test with real local ZK. */
+ private static final boolean USE_TEST_CLUSTER = true;
+
+ /** */
+ private boolean client;
+
+ /** */
+ private static ThreadLocal<Boolean> clientThreadLoc = new ThreadLocal<>();
+
+ /** */
+ private static ConcurrentHashMap<UUID, Map<Long, DiscoveryEvent>> evts = new ConcurrentHashMap<>();
+
+ /** */
+ private static volatile boolean err;
+
+ /** */
+ private boolean testSockNio;
+
+ /** */
+ private boolean testCommSpi;
+
+ /** */
+ private long sesTimeout;
+
+ /** */
+ private long joinTimeout;
+
+ /** */
+ private boolean clientReconnectDisabled;
+
+ /** */
+ private ConcurrentHashMap<String, ZookeeperDiscoverySpi> spis = new ConcurrentHashMap<>();
+
+ /** */
+ private Map<String, Object> userAttrs;
+
+ /** */
+ private boolean dfltConsistenId;
+
+ /** */
+ private UUID nodeId;
+
+ /** */
+ private boolean persistence;
+
+ /** */
+ private IgniteOutClosure<CommunicationFailureResolver> commFailureRslvr;
+
+ /** */
+ private IgniteOutClosure<DiscoverySpiNodeAuthenticator> auth;
+
+ /** */
+ private String zkRootPath;
+
+ /** {@inheritDoc} */
+ @Override protected IgniteConfiguration getConfiguration(final String igniteInstanceName) throws Exception {
+ if (testSockNio)
+ System.setProperty(ZOOKEEPER_CLIENT_CNXN_SOCKET, ZkTestClientCnxnSocketNIO.class.getName());
+
+ IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+ if (nodeId != null)
+ cfg.setNodeId(nodeId);
+
+ if (!dfltConsistenId)
+ cfg.setConsistentId(igniteInstanceName);
+
+ ZookeeperDiscoverySpi zkSpi = new ZookeeperDiscoverySpi();
+
+ if (joinTimeout != 0)
+ zkSpi.setJoinTimeout(joinTimeout);
+
+ zkSpi.setSessionTimeout(sesTimeout > 0 ? sesTimeout : 10_000);
+
+ zkSpi.setClientReconnectDisabled(clientReconnectDisabled);
+
+ // Set authenticator for basic sanity tests.
+ if (auth != null) {
+ zkSpi.setAuthenticator(auth.apply());
+
+ zkSpi.setInternalListener(new IgniteDiscoverySpiInternalListener() {
+ @Override public void beforeJoin(ClusterNode locNode, IgniteLogger log) {
+ ZookeeperClusterNode locNode0 = (ZookeeperClusterNode)locNode;
+
+ Map<String, Object> attrs = new HashMap<>(locNode0.getAttributes());
+
+ attrs.put(ATTR_SECURITY_CREDENTIALS, new SecurityCredentials(null, null, igniteInstanceName));
+
+ locNode0.setAttributes(attrs);
+ }
+
+ @Override public boolean beforeSendCustomEvent(DiscoverySpi spi, IgniteLogger log, DiscoverySpiCustomMessage msg) {
+ return false;
+ }
+ });
+ }
+
+ spis.put(igniteInstanceName, zkSpi);
+
+ if (USE_TEST_CLUSTER) {
+ assert zkCluster != null;
+
+ zkSpi.setZkConnectionString(zkCluster.getConnectString());
+
+ if (zkRootPath != null)
+ zkSpi.setZkRootPath(zkRootPath);
+ }
+ else
+ zkSpi.setZkConnectionString("localhost:2181");
+
+ cfg.setDiscoverySpi(zkSpi);
+
+ CacheConfiguration ccfg = new CacheConfiguration(DEFAULT_CACHE_NAME);
+
+ ccfg.setWriteSynchronizationMode(FULL_SYNC);
+
+ cfg.setCacheConfiguration(ccfg);
+
+ Boolean clientMode = clientThreadLoc.get();
+
+ if (clientMode != null)
+ cfg.setClientMode(clientMode);
+ else
+ cfg.setClientMode(client);
+
+ if (userAttrs != null)
+ cfg.setUserAttributes(userAttrs);
+
+ Map<IgnitePredicate<? extends Event>, int[]> lsnrs = new HashMap<>();
+
+ lsnrs.put(new IgnitePredicate<Event>() {
+ /** */
+ @IgniteInstanceResource
+ private Ignite ignite;
+
+ @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+ @Override public boolean apply(Event evt) {
+ try {
+ DiscoveryEvent discoveryEvt = (DiscoveryEvent)evt;
+
+ UUID locId = ((IgniteKernal)ignite).context().localNodeId();
+
+ Map<Long, DiscoveryEvent> nodeEvts = evts.get(locId);
+
+ if (nodeEvts == null) {
+ Object old = evts.put(locId, nodeEvts = new TreeMap<>());
+
+ assertNull(old);
+
+ synchronized (nodeEvts) {
+ DiscoveryLocalJoinData locJoin = ((IgniteKernal)ignite).context().discovery().localJoin();
+
+ nodeEvts.put(locJoin.event().topologyVersion(), locJoin.event());
+ }
+ }
+
+ synchronized (nodeEvts) {
+ DiscoveryEvent old = nodeEvts.put(discoveryEvt.topologyVersion(), discoveryEvt);
+
+ assertNull(old);
+ }
+ }
+ catch (Throwable e) {
+ error("Unexpected error [evt=" + evt + ", err=" + e + ']', e);
+
+ err = true;
+ }
+
+ return true;
+ }
+ }, new int[]{EVT_NODE_JOINED, EVT_NODE_FAILED, EVT_NODE_LEFT});
+
+ cfg.setLocalEventListeners(lsnrs);
+
+ if (persistence) {
+ DataStorageConfiguration memCfg = new DataStorageConfiguration()
+ .setDefaultDataRegionConfiguration(new DataRegionConfiguration().setMaxSize(100 * 1024 * 1024).
+ setPersistenceEnabled(true))
+ .setPageSize(1024)
+ .setWalMode(WALMode.LOG_ONLY);
+
+ cfg.setDataStorageConfiguration(memCfg);
+ }
+
+ if (testCommSpi)
+ cfg.setCommunicationSpi(new ZkTestCommunicationSpi());
+
+ if (commFailureRslvr != null)
+ cfg.setCommunicationFailureResolver(commFailureRslvr.apply());
+
+ return cfg;
+ }
+
+ /**
+ * @param clientMode Client mode flag for started nodes.
+ */
+ private void clientMode(boolean clientMode) {
+ client = clientMode;
+ }
+
+ /**
+ * @param clientMode Client mode flag for nodes started from current thread.
+ */
+ private void clientModeThreadLocal(boolean clientMode) {
+ clientThreadLoc.set(clientMode);
+ }
+
+ /** {@inheritDoc} */
+ @Override protected void beforeTestsStarted() throws Exception {
+ super.beforeTestsStarted();
+
+ System.setProperty(ZookeeperDiscoveryImpl.IGNITE_ZOOKEEPER_DISCOVERY_SPI_ACK_TIMEOUT, "1000");
+ }
+
+ /** {@inheritDoc} */
+ @Override protected void afterTestsStopped() throws Exception {
+ stopZkCluster();
+
+ System.clearProperty(ZookeeperDiscoveryImpl.IGNITE_ZOOKEEPER_DISCOVERY_SPI_ACK_TIMEOUT);
+
+ super.afterTestsStopped();
+ }
+
+ /**
+ *
+ */
+ private void stopZkCluster() {
+ if (zkCluster != null) {
+ try {
+ zkCluster.close();
+ }
+ catch (Exception e) {
+ U.error(log, "Failed to stop Zookeeper client: " + e, e);
+ }
+
+ zkCluster = null;
+ }
+ }
+
+ /**
+ *
+ */
+ private static void ackEveryEventSystemProperty() {
+ System.setProperty(IGNITE_ZOOKEEPER_DISCOVERY_SPI_ACK_THRESHOLD, "1");
+ }
+
+ /**
+ *
+ */
+ private void clearAckEveryEventSystemProperty() {
+ System.setProperty(IGNITE_ZOOKEEPER_DISCOVERY_SPI_ACK_THRESHOLD, "1");
+ }
+
+ /** {@inheritDoc} */
+ @Override protected void beforeTest() throws Exception {
+ super.beforeTest();
+
+ if (USE_TEST_CLUSTER && zkCluster == null) {
+ zkCluster = ZookeeperDiscoverySpiTestSuite2.createTestingCluster(ZK_SRVS);
+
+ zkCluster.start();
+ }
+
+ reset();
+ }
+
+ /** {@inheritDoc} */
+ @Override protected void afterTest() throws Exception {
+ super.afterTest();
+
+ clearAckEveryEventSystemProperty();
+
+ try {
+ assertFalse("Unexpected error, see log for details", err);
+
+ checkEventsConsistency();
+
+ checkInternalStructuresCleanup();
+
+ //TODO uncomment when https://issues.apache.org/jira/browse/IGNITE-8193 is fixed
+// checkZkNodesCleanup();
+ }
+ finally {
+ reset();
+
+ stopAllGrids();
+ }
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ private void checkInternalStructuresCleanup() throws Exception {
+ for (Ignite node : G.allGrids()) {
+ final AtomicReference<?> res = GridTestUtils.getFieldValue(spi(node), "impl", "commErrProcFut");
+
+ GridTestUtils.waitForCondition(new GridAbsPredicate() {
+ @Override public boolean apply() {
+ return res.get() == null;
+ }
+ }, 30_000);
+
+ assertNull(res.get());
+ }
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testZkRootNotExists() throws Exception {
+ zkRootPath = "/a/b/c";
+
+ for (int i = 0; i < 3; i++) {
+ reset();
+
+ startGridsMultiThreaded(5);
+
+ waitForTopology(5);
+
+ stopAllGrids();
+
+ checkEventsConsistency();
+ }
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testMetadataUpdate() throws Exception {
+ startGrid(0);
+
+ GridTestUtils.runMultiThreaded(new Callable<Void>() {
+ @Override public Void call() throws Exception {
+ ignite(0).configuration().getMarshaller().marshal(new C1());
+ ignite(0).configuration().getMarshaller().marshal(new C2());
+
+ return null;
+ }
+ }, 64, "marshal");
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testNodeAddresses() throws Exception {
+ startGridsMultiThreaded(3);
+
+ clientMode(true);
+
+ startGridsMultiThreaded(3, 3);
+
+ waitForTopology(6);
+
+ for (Ignite node : G.allGrids()) {
+ ClusterNode locNode0 = node.cluster().localNode();
+
+ assertTrue(locNode0.addresses().size() > 0);
+ assertTrue(locNode0.hostNames().size() > 0);
+
+ for (ClusterNode node0 : node.cluster().nodes()) {
+ assertTrue(node0.addresses().size() > 0);
+ assertTrue(node0.hostNames().size() > 0);
+ }
+ }
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testSetConsistentId() throws Exception {
+ startGridsMultiThreaded(3);
+
+ clientMode(true);
+
+ startGridsMultiThreaded(3, 3);
+
+ waitForTopology(6);
+
+ for (Ignite node : G.allGrids()) {
+ ClusterNode locNode0 = node.cluster().localNode();
+
+ assertEquals(locNode0.attribute(ATTR_IGNITE_INSTANCE_NAME),
+ locNode0.consistentId());
+
+ for (ClusterNode node0 : node.cluster().nodes()) {
+ assertEquals(node0.attribute(ATTR_IGNITE_INSTANCE_NAME),
+ node0.consistentId());
+ }
+ }
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testDefaultConsistentId() throws Exception {
+ dfltConsistenId = true;
+
+ startGridsMultiThreaded(3);
+
+ clientMode(true);
+
+ startGridsMultiThreaded(3, 3);
+
+ waitForTopology(6);
+
+ for (Ignite node : G.allGrids()) {
+ ClusterNode locNode0 = node.cluster().localNode();
+
+ assertNotNull(locNode0.consistentId());
+
+ for (ClusterNode node0 : node.cluster().nodes())
+ assertNotNull(node0.consistentId());
+ }
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testClientNodesStatus() throws Exception {
+ startGrid(0);
+
+ for (Ignite node : G.allGrids()) {
+ assertEquals(0, node.cluster().forClients().nodes().size());
+ assertEquals(1, node.cluster().forServers().nodes().size());
+ }
+
+ clientMode(true);
+
+ startGrid(1);
+
+ for (Ignite node : G.allGrids()) {
+ assertEquals(1, node.cluster().forClients().nodes().size());
+ assertEquals(1, node.cluster().forServers().nodes().size());
+ }
+
+ clientMode(false);
+
+ startGrid(2);
+
+ clientMode(true);
+
+ startGrid(3);
+
+ for (Ignite node : G.allGrids()) {
+ assertEquals(2, node.cluster().forClients().nodes().size());
+ assertEquals(2, node.cluster().forServers().nodes().size());
+ }
+
+ stopGrid(1);
+
+ waitForTopology(3);
+
+ for (Ignite node : G.allGrids()) {
+ assertEquals(1, node.cluster().forClients().nodes().size());
+ assertEquals(2, node.cluster().forServers().nodes().size());
+ }
+
+ stopGrid(2);
+
+ waitForTopology(2);
+
+ for (Ignite node : G.allGrids()) {
+ assertEquals(1, node.cluster().forClients().nodes().size());
+ assertEquals(1, node.cluster().forServers().nodes().size());
+ }
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void _testLocalAuthenticationFails() throws Exception {
+ auth = ZkTestNodeAuthenticator.factory(getTestIgniteInstanceName(0));
+
+ Throwable err = GridTestUtils.assertThrows(log, new Callable<Void>() {
+ @Override public Void call() throws Exception {
+ startGrid(0);
+
+ return null;
+ }
+ }, IgniteCheckedException.class, null);
+
+ IgniteSpiException spiErr = X.cause(err, IgniteSpiException.class);
+
+ assertNotNull(spiErr);
+ assertTrue(spiErr.getMessage().contains("Authentication failed for local node"));
+
+ startGrid(1);
+ startGrid(2);
+
+ checkTestSecuritySubject(2);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testAuthentication() throws Exception {
+ auth = ZkTestNodeAuthenticator.factory(getTestIgniteInstanceName(1),
+ getTestIgniteInstanceName(5));
+
+ startGrid(0);
+
+ checkTestSecuritySubject(1);
+
+ {
+ clientMode(false);
+ checkStartFail(1);
+
+ clientMode(true);
+ checkStartFail(1);
+
+ clientMode(false);
+ }
+
+ startGrid(2);
+
+ checkTestSecuritySubject(2);
+
+ stopGrid(2);
+
+ checkTestSecuritySubject(1);
+
+ startGrid(2);
+
+ checkTestSecuritySubject(2);
+
+ stopGrid(0);
+
+ checkTestSecuritySubject(1);
+
+ checkStartFail(1);
+
+ clientMode(false);
+
+ startGrid(3);
+
+ clientMode(true);
+
+ startGrid(4);
+
+ clientMode(false);
+
+ startGrid(0);
+
+ checkTestSecuritySubject(4);
+
+ checkStartFail(1);
+ checkStartFail(5);
+
+ clientMode(true);
+
+ checkStartFail(1);
+ checkStartFail(5);
+ }
+
+ /**
+ * @param nodeIdx Node index.
+ */
+ private void checkStartFail(final int nodeIdx) {
+ Throwable err = GridTestUtils.assertThrows(log, new Callable<Void>() {
+ @Override public Void call() throws Exception {
+ startGrid(nodeIdx);
+
+ return null;
+ }
+ }, IgniteCheckedException.class, null);
+
+ IgniteSpiException spiErr = X.cause(err, IgniteSpiException.class);
+
+ assertNotNull(spiErr);
+ assertTrue(spiErr.getMessage().contains("Authentication failed"));
+ }
+
+ /**
+ * @param expNodes Expected nodes number.
+ * @throws Exception If failed.
+ */
+ private void checkTestSecuritySubject(int expNodes) throws Exception {
+ waitForTopology(expNodes);
+
+ List<Ignite> nodes = G.allGrids();
+
+ JdkMarshaller marsh = new JdkMarshaller();
+
+ for (Ignite ignite : nodes) {
+ Collection<ClusterNode> nodes0 = ignite.cluster().nodes();
+
+ assertEquals(nodes.size(), nodes0.size());
+
+ for (ClusterNode node : nodes0) {
+ byte[] secSubj = node.attribute(ATTR_SECURITY_SUBJECT_V2);
+
+ assertNotNull(secSubj);
+
+ ZkTestNodeAuthenticator.TestSecurityContext secCtx = marsh.unmarshal(secSubj, null);
+
+ assertEquals(node.attribute(ATTR_IGNITE_INSTANCE_NAME), secCtx.nodeName);
+ }
+ }
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testStopNode_1() throws Exception {
+ startGrids(5);
+
+ waitForTopology(5);
+
+ stopGrid(3);
+
+ waitForTopology(4);
+
+ startGrid(3);
+
+ waitForTopology(5);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testCustomEventsSimple1_SingleNode() throws Exception {
+ ackEveryEventSystemProperty();
+
+ Ignite srv0 = startGrid(0);
+
+ srv0.createCache(new CacheConfiguration<>("c1"));
+
+ waitForEventsAcks(srv0);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testCustomEventsSimple1_5_Nodes() throws Exception {
+ ackEveryEventSystemProperty();
+
+ Ignite srv0 = startGrids(5);
+
+ srv0.createCache(new CacheConfiguration<>("c1"));
+
+ awaitPartitionMapExchange();
+
+ waitForEventsAcks(srv0);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testCustomEvents_FastStopProcess_1() throws Exception {
+ customEvents_FastStopProcess(1, 0);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testCustomEvents_FastStopProcess_2() throws Exception {
+ customEvents_FastStopProcess(5, 5);
+ }
+
+ /**
+ * @param srvs Servers number.
+ * @param clients Clients number.
+ * @throws Exception If failed.
+ */
+ private void customEvents_FastStopProcess(int srvs, int clients) throws Exception {
+ ackEveryEventSystemProperty();
+
+ Map<UUID, List<T3<AffinityTopologyVersion, UUID, DiscoveryCustomMessage>>> rcvdMsgs =
+ new ConcurrentHashMap<>();
+
+ Ignite crd = startGrid(0);
+
+ UUID crdId = crd.cluster().localNode().id();
+
+ if (srvs > 1)
+ startGridsMultiThreaded(1, srvs - 1);
+
+ if (clients > 0) {
+ client = true;
+
+ startGridsMultiThreaded(srvs, clients);
+ }
+
+ awaitPartitionMapExchange();
+
+ List<Ignite> nodes = G.allGrids();
+
+ assertEquals(srvs + clients, nodes.size());
+
+ for (Ignite node : nodes)
+ registerTestEventListeners(node, rcvdMsgs);
+
+ int payload = 0;
+
+ AffinityTopologyVersion topVer = ((IgniteKernal)crd).context().discovery().topologyVersionEx();
+
+ for (Ignite node : nodes) {
+ UUID sndId = node.cluster().localNode().id();
+
+ info("Send from node: " + sndId);
+
+ GridDiscoveryManager discoveryMgr = ((IgniteKernal)node).context().discovery();
+
+ {
+ List<T3<AffinityTopologyVersion, UUID, DiscoveryCustomMessage>> expCrdMsgs = new ArrayList<>();
+ List<T3<AffinityTopologyVersion, UUID, DiscoveryCustomMessage>> expNodesMsgs = Collections.emptyList();
+
+ TestFastStopProcessCustomMessage msg = new TestFastStopProcessCustomMessage(false, payload++);
+
+ expCrdMsgs.add(new T3<AffinityTopologyVersion, UUID, DiscoveryCustomMessage>(topVer, sndId, msg));
+
+ discoveryMgr.sendCustomEvent(msg);
+
+ doSleep(200); // Wait some time to check extra messages are not received.
+
+ checkEvents(crd, rcvdMsgs, expCrdMsgs);
+
+ for (Ignite node0 : nodes) {
+ if (node0 != crd)
+ checkEvents(node0, rcvdMsgs, expNodesMsgs);
+ }
+
+ rcvdMsgs.clear();
+ }
+ {
+ List<T3<AffinityTopologyVersion, UUID, DiscoveryCustomMessage>> expCrdMsgs = new ArrayList<>();
+ List<T3<AffinityTopologyVersion, UUID, DiscoveryCustomMessage>> expNodesMsgs = new ArrayList<>();
+
+ TestFastStopProcessCustomMessage msg = new TestFastStopProcessCustomMessage(true, payload++);
+
+ expCrdMsgs.add(new T3<AffinityTopologyVersion, UUID, DiscoveryCustomMessage>(topVer, sndId, msg));
+
+ discoveryMgr.sendCustomEvent(msg);
+
+ TestFastStopProcessCustomMessageAck ackMsg = new TestFastStopProcessCustomMessageAck(msg.payload);
+
+ expCrdMsgs.add(new T3<AffinityTopologyVersion, UUID, DiscoveryCustomMessage>(topVer, crdId, ackMsg));
+ expNodesMsgs.add(new T3<AffinityTopologyVersion, UUID, DiscoveryCustomMessage>(topVer, crdId, ackMsg));
+
+ doSleep(200); // Wait some time to check extra messages are not received.
+
+ checkEvents(crd, rcvdMsgs, expCrdMsgs);
+
+ for (Ignite node0 : nodes) {
+ if (node0 != crd)
+ checkEvents(node0, rcvdMsgs, expNodesMsgs);
+ }
+
+ rcvdMsgs.clear();
+ }
+
+ waitForEventsAcks(crd);
+ }
+ }
+
+ /**
+ * @param node Node to check.
+ * @param rcvdMsgs Received messages.
+ * @param expMsgs Expected messages.
+ * @throws Exception If failed.
+ */
+ private void checkEvents(
+ Ignite node,
+ final Map<UUID, List<T3<AffinityTopologyVersion, UUID, DiscoveryCustomMessage>>> rcvdMsgs,
+ final List<T3<AffinityTopologyVersion, UUID, DiscoveryCustomMessage>> expMsgs) throws Exception {
+ final UUID nodeId = node.cluster().localNode().id();
+
+ assertTrue(GridTestUtils.waitForCondition(new GridAbsPredicate() {
+ @Override public boolean apply() {
+ List<T3<AffinityTopologyVersion, UUID, DiscoveryCustomMessage>> msgs = rcvdMsgs.get(nodeId);
+
+ int size = msgs == null ? 0 : msgs.size();
+
+ return size >= expMsgs.size();
+ }
+ }, 5000));
+
+ List<T3<AffinityTopologyVersion, UUID, DiscoveryCustomMessage>> msgs = rcvdMsgs.get(nodeId);
+
+ if (msgs == null)
+ msgs = Collections.emptyList();
+
+ assertEqualsCollections(expMsgs, msgs);
+ }
+
+ /**
+ * @param node Node.
+ * @param rcvdMsgs Map to store received events.
+ */
+ private void registerTestEventListeners(Ignite node,
+ final Map<UUID, List<T3<AffinityTopologyVersion, UUID, DiscoveryCustomMessage>>> rcvdMsgs) {
+ GridDiscoveryManager discoveryMgr = ((IgniteKernal)node).context().discovery();
+
+ final UUID nodeId = node.cluster().localNode().id();
+
+ discoveryMgr.setCustomEventListener(TestFastStopProcessCustomMessage.class,
+ new CustomEventListener<TestFastStopProcessCustomMessage>() {
+ @Override public void onCustomEvent(AffinityTopologyVersion topVer, ClusterNode snd, TestFastStopProcessCustomMessage msg) {
+ List<T3<AffinityTopologyVersion, UUID, DiscoveryCustomMessage>> list = rcvdMsgs.get(nodeId);
+
+ if (list == null)
+ rcvdMsgs.put(nodeId, list = new ArrayList<>());
+
+ list.add(new T3<>(topVer, snd.id(), (DiscoveryCustomMessage)msg));
+ }
+ }
+ );
+ discoveryMgr.setCustomEventListener(TestFastStopProcessCustomMessageAck.class,
+ new CustomEventListener<TestFastStopProcessCustomMessageAck>() {
+ @Override public void onCustomEvent(AffinityTopologyVersion topVer, ClusterNode snd, TestFastStopProcessCustomMessageAck msg) {
+ List<T3<AffinityTopologyVersion, UUID, DiscoveryCustomMessage>> list = rcvdMsgs.get(nodeId);
+
+ if (list == null)
+ rcvdMsgs.put(nodeId, list = new ArrayList<>());
+
+ list.add(new T3<>(topVer, snd.id(), (DiscoveryCustomMessage)msg));
+ }
+ }
+ );
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testSegmentation1() throws Exception {
+ sesTimeout = 2000;
+ testSockNio = true;
+
+ Ignite node0 = startGrid(0);
+
+ final CountDownLatch l = new CountDownLatch(1);
+
+ node0.events().localListen(new IgnitePredicate<Event>() {
+ @Override public boolean apply(Event evt) {
+ l.countDown();
+
+ return false;
+ }
+ }, EventType.EVT_NODE_SEGMENTED);
+
+ ZkTestClientCnxnSocketNIO c0 = ZkTestClientCnxnSocketNIO.forNode(node0);
+
+ c0.closeSocket(true);
+
+ for (int i = 0; i < 10; i++) {
+ Thread.sleep(1_000);
+
+ if (l.getCount() == 0)
+ break;
+ }
+
+ info("Allow connect");
+
+ c0.allowConnect();
+
+ assertTrue(l.await(10, TimeUnit.SECONDS));
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testSegmentation2() throws Exception {
+ sesTimeout = 2000;
+
+ Ignite node0 = startGrid(0);
+
+ final CountDownLatch l = new CountDownLatch(1);
+
+ node0.events().localListen(new IgnitePredicate<Event>() {
+ @Override public boolean apply(Event evt) {
+ l.countDown();
+
+ return false;
+ }
+ }, EventType.EVT_NODE_SEGMENTED);
+
+ try {
+ zkCluster.close();
+
+ assertTrue(l.await(10, TimeUnit.SECONDS));
+ }
+ finally {
+ zkCluster = ZookeeperDiscoverySpiTestSuite2.createTestingCluster(ZK_SRVS);
+
+ zkCluster.start();
+ }
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testSegmentation3() throws Exception {
+ fail("https://issues.apache.org/jira/browse/IGNITE-8183");
+
+ sesTimeout = 5000;
+
+ Ignite node0 = startGrid(0);
+
+ final CountDownLatch l = new CountDownLatch(1);
+
+ node0.events().localListen(new IgnitePredicate<Event>() {
+ @Override public boolean apply(Event evt) {
+ l.countDown();
+
+ return false;
+ }
+ }, EventType.EVT_NODE_SEGMENTED);
+
+ List<TestingZooKeeperServer> srvs = zkCluster.getServers();
+
+ assertEquals(3, srvs.size());
+
+ try {
+ srvs.get(0).stop();
+ srvs.get(1).stop();
+
+ assertTrue(l.await(20, TimeUnit.SECONDS));
+ }
+ finally {
+ zkCluster.close();
+
+ zkCluster = ZookeeperDiscoverySpiTestSuite2.createTestingCluster(ZK_SRVS);
+
+ zkCluster.start();
+ }
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testQuorumRestore() throws Exception {
+ fail("https://issues.apache.org/jira/browse/IGNITE-8180");
+
+ sesTimeout = 15_000;
+
+ startGrids(3);
+
+ waitForTopology(3);
+
+ List<TestingZooKeeperServer> srvs = zkCluster.getServers();
+
+ assertEquals(3, srvs.size());
+
+ try {
+ srvs.get(0).stop();
+ srvs.get(1).stop();
+
+ U.sleep(2000);
+
+ srvs.get(1).restart();
+
+ U.sleep(4000);
+
+ startGrid(4);
+
+ waitForTopology(4);
+ }
+ finally {
+ zkCluster.close();
+
+ zkCluster = ZookeeperDiscoverySpiTestSuite2.createTestingCluster(ZK_SRVS);
+
+ zkCluster.start();
+ }
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testConnectionRestore1() throws Exception {
+ testSockNio = true;
+
+ Ignite node0 = startGrid(0);
+
+ ZkTestClientCnxnSocketNIO c0 = ZkTestClientCnxnSocketNIO.forNode(node0);
+
+ c0.closeSocket(false);
+
+ startGrid(1);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testConnectionRestore2() throws Exception {
+ testSockNio = true;
+
+ Ignite node0 = startGrid(0);
+
+ ZkTestClientCnxnSocketNIO c0 = ZkTestClientCnxnSocketNIO.forNode(node0);
+
+ c0.closeSocket(false);
+
+ startGridsMultiThreaded(1, 5);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testConnectionRestore_NonCoordinator1() throws Exception {
+ connectionRestore_NonCoordinator(false);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testConnectionRestore_NonCoordinator2() throws Exception {
+ connectionRestore_NonCoordinator(true);
+ }
+
+ /**
+ * @param failWhenDisconnected {@code True} if fail node while another node is disconnected.
+ * @throws Exception If failed.
+ */
+ private void connectionRestore_NonCoordinator(boolean failWhenDisconnected) throws Exception {
+ testSockNio = true;
+
+ Ignite node0 = startGrid(0);
+ Ignite node1 = startGrid(1);
+
+ ZkTestClientCnxnSocketNIO c1 = ZkTestClientCnxnSocketNIO.forNode(node1);
+
+ c1.closeSocket(true);
+
+ IgniteInternalFuture<?> fut = GridTestUtils.runAsync(new Callable<Void>() {
+ @Override public Void call() {
+ try {
+ startGrid(2);
+ }
+ catch (Exception e) {
+ info("Start error: " + e);
+ }
+
+ return null;
+ }
+ }, "start-node");
+
+ checkEvents(node0, joinEvent(3));
+
+ if (failWhenDisconnected) {
+ ZookeeperDiscoverySpi spi = spis.get(getTestIgniteInstanceName(2));
+
+ closeZkClient(spi);
+
+ checkEvents(node0, failEvent(4));
+ }
+
+ c1.allowConnect();
+
+ checkEvents(ignite(1), joinEvent(3));
+
+ if (failWhenDisconnected) {
+ checkEvents(ignite(1), failEvent(4));
+
+ IgnitionEx.stop(getTestIgniteInstanceName(2), true, true);
+ }
+
+ fut.get();
+
+ waitForTopology(failWhenDisconnected ? 2 : 3);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testConnectionRestore_Coordinator1() throws Exception {
+ connectionRestore_Coordinator(1, 1, 0);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testConnectionRestore_Coordinator1_1() throws Exception {
+ connectionRestore_Coordinator(1, 1, 1);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testConnectionRestore_Coordinator2() throws Exception {
+ connectionRestore_Coordinator(1, 3, 0);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testConnectionRestore_Coordinator3() throws Exception {
+ connectionRestore_Coordinator(3, 3, 0);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testConnectionRestore_Coordinator4() throws Exception {
+ connectionRestore_Coordinator(3, 3, 1);
+ }
+
+ /**
+ * @param initNodes Number of initially started nodes.
+ * @param startNodes Number of nodes to start after coordinator loose connection.
+ * @param failCnt Number of nodes to stop after coordinator loose connection.
+ * @throws Exception If failed.
+ */
+ private void connectionRestore_Coordinator(final int initNodes, int startNodes, int failCnt) throws Exception {
+ sesTimeout = 30_000;
+ testSockNio = true;
+
+ Ignite node0 = startGrids(initNodes);
+
+ ZkTestClientCnxnSocketNIO c0 = ZkTestClientCnxnSocketNIO.forNode(node0);
+
+ c0.closeSocket(true);
+
+ final AtomicInteger nodeIdx = new AtomicInteger(initNodes);
+
+ IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new Callable<Void>() {
+ @Override public Void call() {
+ try {
+ startGrid(nodeIdx.getAndIncrement());
+ }
+ catch (Exception e) {
+ error("Start failed: " + e);
+ }
+
+ return null;
+ }
+ }, startNodes, "start-node");
+
+ int cnt = 0;
+
+ DiscoveryEvent[] expEvts = new DiscoveryEvent[startNodes - failCnt];
+
+ int expEvtCnt = 0;
+
+ sesTimeout = 1000;
+
+ List<ZkTestClientCnxnSocketNIO> blockedC = new ArrayList<>();
+
+ final List<String> failedZkNodes = new ArrayList<>(failCnt);
+
+ for (int i = initNodes; i < initNodes + startNodes; i++) {
+ final ZookeeperDiscoverySpi spi = waitSpi(getTestIgniteInstanceName(i));
+
+ assertTrue(GridTestUtils.waitForCondition(new GridAbsPredicate() {
+ @Override public boolean apply() {
+ Object spiImpl = GridTestUtils.getFieldValue(spi, "impl");
+
+ if (spiImpl == null)
+ return false;
+
+ long internalOrder = GridTestUtils.getFieldValue(spiImpl, "rtState", "internalOrder");
+
+ return internalOrder > 0;
+ }
+ }, 10_000));
+
+ if (cnt++ < failCnt) {
+ ZkTestClientCnxnSocketNIO c = ZkTestClientCnxnSocketNIO.forNode(getTestIgniteInstanceName(i));
+
+ c.closeSocket(true);
+
+ blockedC.add(c);
+
+ failedZkNodes.add(aliveZkNodePath(spi));
+ }
+ else {
+ expEvts[expEvtCnt] = joinEvent(initNodes + expEvtCnt + 1);
+
+ expEvtCnt++;
+ }
+ }
+
+ waitNoAliveZkNodes(log, zkCluster.getConnectString(), failedZkNodes, 30_000);
+
+ c0.allowConnect();
+
+ for (ZkTestClientCnxnSocketNIO c : blockedC)
+ c.allowConnect();
+
+ if (expEvts.length > 0) {
+ for (int i = 0; i < initNodes; i++)
+ checkEvents(ignite(i), expEvts);
+ }
+
+ fut.get();
+
+ waitForTopology(initNodes + startNodes - failCnt);
+ }
+
+ /**
+ * @param node Node.
+ * @return Corresponding znode.
+ */
+ private static String aliveZkNodePath(Ignite node) {
+ return aliveZkNodePath(node.configuration().getDiscoverySpi());
+ }
+
+ /**
+ * @param spi SPI.
+ * @return Znode related to given SPI.
+ */
+ private static String aliveZkNodePath(DiscoverySpi spi) {
+ String path = GridTestUtils.getFieldValue(spi, "impl", "rtState", "locNodeZkPath");
+
+ return path.substring(path.lastIndexOf('/') + 1);
+ }
+
+ /**
+ * @param log Logger.
+ * @param connectString Zookeeper connect string.
+ * @param failedZkNodes Znodes which should be removed.
+ * @param timeout Timeout.
+ * @throws Exception If failed.
+ */
+ private static void waitNoAliveZkNodes(final IgniteLogger log,
+ String connectString,
+ final List<String> failedZkNodes,
+ long timeout)
+ throws Exception
+ {
+ final ZookeeperClient zkClient = new ZookeeperClient(log, connectString, 10_000, null);
+
+ try {
+ assertTrue(GridTestUtils.waitForCondition(new GridAbsPredicate() {
+ @Override public boolean apply() {
+ try {
+ List<String> c = zkClient.getChildren(IGNITE_ZK_ROOT + "/" + ZkIgnitePaths.ALIVE_NODES_DIR);
+
+ for (String failedZkNode : failedZkNodes) {
+ if (c.contains(failedZkNode)) {
+ log.info("Alive node is not removed [node=" + failedZkNode + ", all=" + c + ']');
+
+ return false;
+ }
+ }
+
+ return true;
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+
+ fail();
+
+ return true;
+ }
+ }
+ }, timeout));
+ }
+ finally {
+ zkClient.close();
+ }
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testConcurrentStartWithClient() throws Exception {
+ final int NODES = 20;
+
+ for (int i = 0; i < 3; i++) {
+ info("Iteration: " + i);
+
+ final int srvIdx = ThreadLocalRandom.current().nextInt(NODES);
+
+ final AtomicInteger idx = new AtomicInteger();
+
+ GridTestUtils.runMultiThreaded(new Callable<Void>() {
+ @Override public Void call() throws Exception {
+ int threadIdx = idx.getAndIncrement();
+
+ clientModeThreadLocal(threadIdx == srvIdx || ThreadLocalRandom.current().nextBoolean());
+
+ startGrid(threadIdx);
+
+ return null;
+ }
+ }, NODES, "start-node");
+
+ waitForTopology(NODES);
+
+ stopAllGrids();
+
+ checkEventsConsistency();
+
+ evts.clear();
+ }
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testConcurrentStart() throws Exception {
+ final int NODES = 20;
+
+ for (int i = 0; i < 3; i++) {
+ info("Iteration: " + i);
+
+ final AtomicInteger idx = new AtomicInteger();
+
+ final CyclicBarrier b = new CyclicBarrier(NODES);
+
+ GridTestUtils.runMultiThreaded(new Callable<Void>() {
+ @Override public Void call() throws Exception {
+ b.await();
+
+ int threadIdx = idx.getAndIncrement();
+
+ startGrid(threadIdx);
+
+ return null;
+ }
+ }, NODES, "start-node");
+
+ waitForTopology(NODES);
+
+ stopAllGrids();
+
+ checkEventsConsistency();
+
+ evts.clear();
+ }
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testConcurrentStartStop1() throws Exception {
+ concurrentStartStop(1);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testConcurrentStartStop2() throws Exception {
+ concurrentStartStop(5);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testConcurrentStartStop2_EventsThrottle() throws Exception {
+ System.setProperty(ZookeeperDiscoveryImpl.IGNITE_ZOOKEEPER_DISCOVERY_SPI_MAX_EVTS, "1");
+
+ try {
+ concurrentStartStop(5);
+ }
+ finally {
+ System.clearProperty(ZookeeperDiscoveryImpl.IGNITE_ZOOKEEPER_DISCOVERY_SPI_MAX_EVTS);
+ }
+ }
+
+ /**
+ * @param initNodes Number of initially started nnodes.
+ * @throws Exception If failed.
+ */
+ private void concurrentStartStop(final int initNodes) throws Exception {
+ startGrids(initNodes);
+
+ final int NODES = 5;
+
+ long topVer = initNodes;
+
+ for (int i = 0; i < 10; i++) {
+ info("Iteration: " + i);
+
+ DiscoveryEvent[] expEvts = new DiscoveryEvent[NODES];
+
+ startGridsMultiThreaded(initNodes, NODES);
+
+ for (int j = 0; j < NODES; j++)
+ expEvts[j] = joinEvent(++topVer);
+
+ checkEvents(ignite(0), expEvts);
+
+ checkEventsConsistency();
+
+ final CyclicBarrier b = new CyclicBarrier(NODES);
+
+ GridTestUtils.runMultiThreaded(new IgniteInClosure<Integer>() {
+ @Override public void apply(Integer idx) {
+ try {
+ b.await();
+
+ stopGrid(initNodes + idx);
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+
+ fail();
+ }
+ }
+ }, NODES, "stop-node");
+
+ for (int j = 0; j < NODES; j++)
+ expEvts[j] = failEvent(++topVer);
+
+ checkEventsConsistency();
+ }
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testClusterRestart() throws Exception {
+ startGridsMultiThreaded(3, false);
+
+ stopAllGrids();
+
+ evts.clear();
+
+ startGridsMultiThreaded(3, false);
+
+ waitForTopology(3);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testConnectionRestore4() throws Exception {
+ testSockNio = true;
+
+ Ignite node0 = startGrid(0);
+
+ ZkTestClientCnxnSocketNIO c0 = ZkTestClientCnxnSocketNIO.forNode(node0);
+
+ c0.closeSocket(false);
+
+ startGrid(1);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testStartStop_1_Node() throws Exception {
+ startGrid(0);
+
+ waitForTopology(1);
+
+ stopGrid(0);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testRestarts_2_Nodes() throws Exception {
+ startGrid(0);
+
+ for (int i = 0; i < 10; i++) {
+ info("Iteration: " + i);
+
+ startGrid(1);
+
+ waitForTopology(2);
+
+ stopGrid(1);
+ }
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testStartStop_2_Nodes_WithCache() throws Exception {
+ startGrids(2);
+
+ for (Ignite node : G.allGrids()) {
+ IgniteCache<Object, Object> cache = node.cache(DEFAULT_CACHE_NAME);
+
+ assertNotNull(cache);
+
+ for (int i = 0; i < 100; i++) {
+ cache.put(i, node.name());
+
+ assertEquals(node.name(), cache.get(i));
+ }
+ }
+
+ awaitPartitionMapExchange();
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testStartStop_2_Nodes() throws Exception {
+ ackEveryEventSystemProperty();
+
+ startGrid(0);
+
+ waitForTopology(1);
+
+ startGrid(1);
+
+ waitForTopology(2);
+
+ for (Ignite node : G.allGrids())
+ node.compute().broadcast(new DummyCallable(null));
+
+ awaitPartitionMapExchange();
+
+ waitForEventsAcks(ignite(0));
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testMultipleClusters() throws Exception {
+ Ignite c0 = startGrid(0);
+
+ zkRootPath = "/cluster2";
+
+ Ignite c1 = startGridsMultiThreaded(1, 5);
+
+ zkRootPath = "/cluster3";
+
+ Ignite c2 = startGridsMultiThreaded(6, 3);
+
+ checkNodesNumber(c0, 1);
+ checkNodesNumber(c1, 5);
+ checkNodesNumber(c2, 3);
+
+ stopGrid(2);
+
+ checkNodesNumber(c0, 1);
+ checkNodesNumber(c1, 4);
+ checkNodesNumber(c2, 3);
+
+ for (int i = 0; i < 3; i++)
+ stopGrid(i + 6);
+
+ checkNodesNumber(c0, 1);
+ checkNodesNumber(c1, 4);
+
+ c2 = startGridsMultiThreaded(6, 2);
+
+ checkNodesNumber(c0, 1);
+ checkNodesNumber(c1, 4);
+ checkNodesNumber(c2, 2);
+
+ evts.clear();
+ }
+
+ /**
+ * @param node Node.
+ * @param expNodes Expected node in cluster.
+ * @throws Exception If failed.
+ */
+ private void checkNodesNumber(final Ignite node, final int expNodes) throws Exception {
+ GridTestUtils.waitForCondition(new GridAbsPredicate() {
+ @Override public boolean apply() {
+ return node.cluster().nodes().size() == expNodes;
+ }
+ }, 5000);
+
+ assertEquals(expNodes, node.cluster().nodes().size());
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testStartStop1() throws Exception {
+ ackEveryEventSystemProperty();
+
+ startGridsMultiThreaded(5, false);
+
+ waitForTopology(5);
+
+ awaitPartitionMapExchange();
+
+ waitForEventsAcks(ignite(0));
+
+ stopGrid(0);
+
+ waitForTopology(4);
+
+ for (Ignite node : G.allGrids())
+ node.compute().broadcast(new DummyCallable(null));
+
+ startGrid(0);
+
+ waitForTopology(5);
+
+ awaitPartitionMapExchange();
+
+ waitForEventsAcks(grid(CU.oldest(ignite(1).cluster().nodes())));
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testStartStop3() throws Exception {
+ startGrids(4);
+
+ awaitPartitionMapExchange();
+
+ stopGrid(0);
+
+ startGrid(5);
+
+ awaitPartitionMapExchange();
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testStartStop4() throws Exception {
+ startGrids(6);
+
+ awaitPartitionMapExchange();
+
+ stopGrid(2);
+
+ if (ThreadLocalRandom.current().nextBoolean())
+ awaitPartitionMapExchange();
+
+ stopGrid(1);
+
+ if (ThreadLocalRandom.current().nextBoolean())
+ awaitPartitionMapExchange();
+
+ stopGrid(0);
+
+ if (ThreadLocalRandom.current().nextBoolean())
+ awaitPartitionMapExchange();
+
+ startGrid(7);
+
+ awaitPartitionMapExchange();
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testStartStop2() throws Exception {
+ startGridsMultiThreaded(10, false);
+
+ GridTestUtils.runMultiThreaded(new IgniteInClosure<Integer>() {
+ @Override public void apply(Integer idx) {
+ stopGrid(idx);
+ }
+ }, 3, "stop-node-thread");
+
+ waitForTopology(7);
+
+ startGridsMultiThreaded(0, 3);
+
+ waitForTopology(10);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testStartStopWithClients() throws Exception {
+ final int SRVS = 3;
+
+ startGrids(SRVS);
+
+ clientMode(true);
+
+ final int THREADS = 30;
+
+ for (int i = 0; i < 5; i++) {
+ info("Iteration: " + i);
+
+ startGridsMultiThreaded(SRVS, THREADS);
+
+ waitForTopology(SRVS + THREADS);
+
+ GridTestUtils.runMultiThreaded(new IgniteInClosure<Integer>() {
+ @Override public void apply(Integer idx) {
+ stopGrid(idx + SRVS);
+ }
+ }, THREADS, "stop-node");
+
+ waitForTopology(SRVS);
+
+ checkEventsConsistency();
+ }
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testTopologyChangeMultithreaded() throws Exception {
+ topologyChangeWithRestarts(false, false);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testTopologyChangeMultithreaded_RestartZk() throws Exception {
+ fail("https://issues.apache.org/jira/browse/IGNITE-8184");
+
+ try {
+ topologyChangeWithRestarts(true, false);
+ }
+ finally {
+ zkCluster.stop();
+
+ zkCluster = null;
+ }
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testTopologyChangeMultithreaded_RestartZk_CloseClients() throws Exception {
+ fail("https://issues.apache.org/jira/browse/IGNITE-8184");
+
+ try {
+ topologyChangeWithRestarts(true, true);
+ }
+ finally {
+ zkCluster.stop();
+
+ zkCluster = null;
+ }
+ }
+
+ /**
+ * @param restartZk If {@code true} in background restarts on of ZK servers.
+ * @param closeClientSock If {@code true} in background closes zk clients' sockets.
+ * @throws Exception If failed.
+ */
+ private void topologyChangeWithRestarts(boolean restartZk, boolean closeClientSock) throws Exception {
+ sesTimeout = 30_000;
+
+ if (closeClientSock)
+ testSockNio = true;
+
+ long stopTime = System.currentTimeMillis() + 60_000;
+
+ AtomicBoolean stop = new AtomicBoolean();
+
+ IgniteInternalFuture<?> fut1 = null;
+
+ IgniteInternalFuture<?> fut2 = null;
+
+ try {
+ fut1 = restartZk ? startRestartZkServers(stopTime, stop) : null;
+ fut2 = closeClientSock ? startCloseZkClientSocket(stopTime, stop) : null;
+
+ int INIT_NODES = 10;
+
+ startGridsMultiThreaded(INIT_NODES);
+
+ final int MAX_NODES = 20;
+
+ final List<Integer> startedNodes = new ArrayList<>();
+
+ for (int i = 0; i < INIT_NODES; i++)
+ startedNodes.add(i);
+
+ ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+ final AtomicInteger startIdx = new AtomicInteger(INIT_NODES);
+
+ while (System.currentTimeMillis() < stopTime) {
+ if (startedNodes.size() >= MAX_NODES) {
+ int stopNodes = rnd.nextInt(5) + 1;
+
+ log.info("Next, stop nodes: " + stopNodes);
+
+ final List<Integer> idxs = new ArrayList<>();
+
+ while (idxs.size() < stopNodes) {
+ Integer stopIdx = rnd.nextInt(startedNodes.size());
+
+ if (!idxs.contains(stopIdx))
+ idxs.add(startedNodes.get(stopIdx));
+ }
+
+ GridTestUtils.runMultiThreaded(new IgniteInClosure<Integer>() {
+ @Override public void apply(Integer threadIdx) {
+ int stopNodeIdx = idxs.get(threadIdx);
+
+ info("Stop node: " + stopNodeIdx);
+
+ stopGrid(stopNodeIdx);
+ }
+ }, stopNodes, "stop-node");
+
+ startedNodes.removeAll(idxs);
+ }
+ else {
+ int startNodes = rnd.nextInt(5) + 1;
+
+ log.info("Next, start nodes: " + startNodes);
+
+ GridTestUtils.runMultiThreaded(new Callable<Void>() {
+ @Override public Void call() throws Exception {
+ int idx = startIdx.incrementAndGet();
+
+ log.info("Start node: " + idx);
+
+ startGrid(idx);
+
+ synchronized (startedNodes) {
+ startedNodes.add(idx);
+ }
+
+ return null;
+ }
+ }, startNodes, "start-node");
+ }
+
+ U.sleep(rnd.nextInt(100) + 1);
+ }
+ }
+ finally {
+ stop.set(true);
+ }
+
+ if (fut1 != null)
+ fut1.get();
+
+ if (fut2 != null)
+ fut2.get();
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testRandomTopologyChanges() throws Exception {
+ randomTopologyChanges(false, false);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ private void checkZkNodesCleanup() throws Exception {
+ final ZookeeperClient zkClient = new ZookeeperClient(getTestResources().getLogger(),
+ zkCluster.getConnectString(),
+ 30_000,
+ null);
+
+ final String basePath = IGNITE_ZK_ROOT + "/";
+
+ final String aliveDir = basePath + ZkIgnitePaths.ALIVE_NODES_DIR + "/";
+
+ try {
+ List<String> znodes = listSubTree(zkClient.zk(), IGNITE_ZK_ROOT);
+
+ boolean foundAlive = false;
+
+ for (String znode : znodes) {
+ if (znode.startsWith(aliveDir)) {
+ foundAlive = true;
+
+ break;
+ }
+ }
+
+ assertTrue(foundAlive); // Sanity check to make sure we check correct directory.
+
+ assertTrue("Failed to wait for unused znodes cleanup", GridTestUtils.waitForCondition(new GridAbsPredicate() {
+ @Override public boolean apply() {
+ try {
+ List<String> znodes = listSubTree(zkClient.zk(), IGNITE_ZK_ROOT);
+
+ for (String znode : znodes) {
+ if (znode.startsWith(aliveDir) || znode.length() < basePath.length())
+ continue;
+
+ znode = znode.substring(basePath.length());
+
+ if (!znode.contains("/")) // Ignore roots.
+ continue;
+
+ // TODO ZK: https://issues.apache.org/jira/browse/IGNITE-8193
+ if (znode.startsWith("jd/"))
+ continue;
+
+ log.info("Found unexpected znode: " + znode);
+
+ return false;
+ }
+
+ return true;
+ }
+ catch (Exception e) {
+ error("Unexpected error: " + e, e);
+
+ fail("Unexpected error: " + e);
+ }
+
+ return false;
+ }
+ }, 10_000));
+ }
+ finally {
+ zkClient.close();
+ }
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testRandomTopologyChanges_RestartZk() throws Exception {
+ randomTopologyChanges(true, false);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testRandomTopologyChanges_CloseClients() throws Exception {
+ fail("https://issues.apache.org/jira/browse/IGNITE-8182");
+
+ randomTopologyChanges(false, true);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testDeployService1() throws Exception {
+ startGridsMultiThreaded(3);
+
+ grid(0).services(grid(0).cluster()).deployNodeSingleton("test", new GridCacheAbstractFullApiSelfTest.DummyServiceImpl());
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testDeployService2() throws Exception {
+ clientMode(false);
+
+ startGrid(0);
+
+ clientMode(true);
+
+ startGrid(1);
+
+ grid(0).services(grid(0).cluster()).deployNodeSingleton("test", new GridCacheAbstractFullApiSelfTest.DummyServiceImpl());
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testDeployService3() throws Exception {
+ IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() {
+ @Override public Object call() throws Exception {
+ clientModeThreadLocal(true);
+
+ startGrid(0);
+
+ return null;
+ }
+ }, "start-node");
+
+ clientModeThreadLocal(false);
+
+ startGrid(1);
+
+ fut.get();
+
+ grid(0).services(grid(0).cluster()).deployNodeSingleton("test", new GridCacheAbstractFullApiSelfTest.DummyServiceImpl());
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testLargeUserAttribute1() throws Exception {
+ initLargeAttribute();
+
+ startGrid(0);
+
+ checkZkNodesCleanup();
+
+ userAttrs = null;
+
+ startGrid(1);
+
+ waitForEventsAcks(ignite(0));
+
+ waitForTopology(2);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testLargeUserAttribute2() throws Exception {
+ startGrid(0);
+
+ initLargeAttribute();
+
+ startGrid(1);
+
+ waitForEventsAcks(ignite(0));
+
+ checkZkNodesCleanup();
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testLargeUserAttribute3() throws Exception {
+ ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+ long stopTime = System.currentTimeMillis() + 60_000;
+
+ int nodes = 0;
+
+ for (int i = 0; i < 25; i++) {
+ info("Iteration: " + i);
+
+ if (rnd.nextBoolean())
+ initLargeAttribute();
+ else
+ userAttrs = null;
+
+ clientMode(i > 5);
+
+ startGrid(i);
+
+ nodes++;
+
+ if (System.currentTimeMillis() >= stopTime)
+ break;
+ }
+
+ waitForTopology(nodes);
+ }
+
+ /**
+ *
+ */
+ private void initLargeAttribute() {
+ userAttrs = new HashMap<>();
+
+ int[] attr = new int[1024 * 1024 + ThreadLocalRandom.current().nextInt(1024)];
+
+ for (int i = 0; i < attr.length; i++)
+ attr[i] = i;
+
+ userAttrs.put("testAttr", attr);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testLargeCustomEvent() throws Exception {
+ Ignite srv0 = startGrid(0);
+
+ // Send large message, single node in topology.
+ IgniteCache<Object, Object> cache = srv0.createCache(largeCacheConfiguration("c1"));
+
+ for (int i = 0; i < 100; i++)
+ cache.put(i, i);
+
+ assertEquals(1, cache.get(1));
+
+ waitForEventsAcks(ignite(0));
+
+ startGridsMultiThreaded(1, 3);
+
+ srv0.destroyCache("c1");
+
+ // Send large message, multiple nodes in topology.
+ cache = srv0.createCache(largeCacheConfiguration("c1"));
+
+ for (int i = 0; i < 100; i++)
+ cache.put(i, i);
+
+ waitForTopology(4);
+
+ ignite(3).createCache(largeCacheConfiguration("c2"));
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testClientReconnectSessionExpire1_1() throws Exception {
+ fail("https://issues.apache.org/jira/browse/IGNITE-8131");
+
+ clientReconnectSessionExpire(false);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testClientReconnectSessionExpire1_2() throws Exception {
+ fail("https://issues.apache.org/jira/browse/IGNITE-8131");
+
+ clientReconnectSessionExpire(true);
+ }
+
+ /**
+ * @param closeSock Test mode flag.
+ * @throws Exception If failed.
+ */
+ private void clientReconnectSessionExpire(boolean closeSock) throws Exception {
+ startGrid(0);
+
+ sesTimeout = 2000;
+ clientMode(true);
+ testSockNio = true;
+
+ Ignite client = startGrid(1);
+
+ client.cache(DEFAULT_CACHE_NAME).put(1, 1);
+
+ reconnectClientNodes(log, Collections.singletonList(client), closeSock);
+
+ assertEquals(1, client.cache(DEFAULT_CACHE_NAME).get(1));
+
+ client.compute().broadcast(new DummyCallable(null));
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testForceClientReconnect() throws Exception {
+ final int SRVS = 3;
+
+ startGrids(SRVS);
+
+ clientMode(true);
+
+ startGrid(SRVS);
+
+ reconnectClientNodes(Collections.singletonList(ignite(SRVS)), new Callable<Void>() {
+ @Override public Void call() throws Exception {
+ ZookeeperDiscoverySpi spi = waitSpi(getTestIgniteInstanceName(SRVS));
+
+ spi.clientReconnect();
+
+ return null;
+ }
+ });
+
+ waitForTopology(SRVS + 1);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testForcibleClientFail() throws Exception {
+ final int SRVS = 3;
+
+ startGrids(SRVS);
+
+ clientMode(true);
+
+ startGrid(SRVS);
+
+ reconnectClientNodes(Collections.singletonList(ignite(SRVS)), new Callable<Void>() {
+ @Override public Void call() throws Exception {
+ ZookeeperDiscoverySpi spi = waitSpi(getTestIgniteInstanceName(0));
+
+ spi.failNode(ignite(SRVS).cluster().localNode().id(), "Test forcible node fail");
+
+ return null;
+ }
+ });
+
+ waitForTopology(SRVS + 1);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testDuplicatedNodeId() throws Exception {
+ UUID nodeId0 = nodeId = UUID.randomUUID();
+
+ startGrid(0);
+
+ int failingNodeIdx = 100;
+
+ for (int i = 0; i < 5; i++) {
+ final int idx = failingNodeIdx++;
+
+ nodeId = nodeId0;
+
+ info("Start node with duplicated ID [iter=" + i + ", nodeId=" + nodeId + ']');
+
+ Throwable err = GridTestUtils.assertThrows(log, new Callable<Void>() {
+ @Override public Void call() throws Exception {
+ startGrid(idx);
+
+ return null;
+ }
+ }, IgniteCheckedException.class, null);
+
+ IgniteSpiException spiErr = X.cause(err, IgniteSpiException.class);
+
+ assertNotNull(spiErr);
+ assertTrue(spiErr.getMessage().contains("Node with the same ID already exists"));
+
+ nodeId = null;
+
+ info("Start node with unique ID [iter=" + i + ']');
+
+ Ignite ignite = startGrid(idx);
+
+ nodeId0 = ignite.cluster().localNode().id();
+
+ waitForTopology(i + 2);
+ }
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testPing() throws Exception {
+ sesTimeout = 5000;
+
+ startGrids(3);
+
+ final ZookeeperDiscoverySpi spi = waitSpi(getTestIgniteInstanceName(1));
+
+ final UUID nodeId = ignite(2).cluster().localNode().id();
+
+ IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new Runnable() {
+ @Override public void run() {
+ assertTrue(spi.pingNode(nodeId));
+ }
+ }, 32, "ping");
+
+ fut.get();
+
+ fut = GridTestUtils.runMultiThreadedAsync(new Runnable() {
+ @Override public void run() {
+ spi.pingNode(nodeId);
+ }
+ }, 32, "ping");
+
+ U.sleep(100);
+
+ stopGrid(2);
+
+ fut.get();
+
+ fut = GridTestUtils.runMultiThreadedAsync(new Runnable() {
+ @Override public void run() {
+ assertFalse(spi.pingNode(nodeId));
+ }
+ }, 32, "ping");
+
+ fut.get();
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testWithPersistence1() throws Exception {
+ startWithPersistence(false);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testWithPersistence2() throws Exception {
+ startWithPersistence(true);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testNoOpCommunicationFailureResolve_1() throws Exception {
+ communicationFailureResolve_Simple(2);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testNoOpCommunicationErrorResolve_2() throws Exception {
+ communicationFailureResolve_Simple(10);
+ }
+
+ /**
+ * @param nodes Nodes number.
+ * @throws Exception If failed.
+ */
+ private void communicationFailureResolve_Simple(int nodes) throws Exception {
+ assert nodes > 1;
+
+ sesTimeout = 2000;
+ commFailureRslvr = NoOpCommunicationFailureResolver.FACTORY;
+
+ startGridsMultiThreaded(nodes);
+
+ ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+ for (int i = 0; i < 3; i++) {
+ info("Iteration: " + i);
+
+ int idx1 = rnd.nextInt(nodes);
+
+ int idx2;
+
+ do {
+ idx2 = rnd.nextInt(nodes);
+ }
+ while (idx1 == idx2);
+
+ ZookeeperDiscoverySpi spi = spi(ignite(idx1));
+
+ spi.resolveCommunicationFailure(ignite(idx2).cluster().localNode(), new Exception("test"));
+
+ checkInternalStructuresCleanup();
+ }
+ }
+
+ /**
+ * Tests case when one node fails before sending communication status.
+ *
+ * @throws Exception If failed.
+ */
+ public void testNoOpCommunicationErrorResolve_3() throws Exception {
+ sesTimeout = 2000;
+ commFailureRslvr = NoOpCommunicationFailureResolver.FACTORY;
+
+ startGridsMultiThreaded(3);
+
+ sesTimeout = 10_000;
+
+ testSockNio = true;
+ sesTimeout = 5000;
+
+ startGrid(3);
+
+ IgniteInternalFuture<?> fut = GridTestUtils.runAsync(new Callable<Object>() {
+ @Override public Object call() {
+ ZookeeperDiscoverySpi spi = spi(ignite(0));
+
+ spi.resolveCommunicationFailure(ignite(1).cluster().localNode(), new Exception("test"));
+
+ return null;
+ }
+ });
+
+ U.sleep(1000);
+
+ ZkTestClientCnxnSocketNIO nio = ZkTestClientCnxnSocketNIO.forNode(ignite(3));
+
+ nio.closeSocket(true);
+
+ try {
+ stopGrid(3);
+
+ fut.get();
+ }
+ finally {
+ nio.allowConnect();
+ }
+
+ waitForTopology(3);
+ }
+
+ /**
+ * Tests case when Coordinator fails while resolve process is in progress.
+ *
+ * @throws Exception If failed.
+ */
+ public void testNoOpCommunicationErrorResolve_4() throws Exception {
+ testCommSpi = true;
+
+ sesTimeout = 2000;
+ commFailureRslvr = NoOpCommunicationFailureResolver.FACTORY;
+
+ startGrid(0);
+
+ startGridsMultiThreaded(1, 3);
+
+ ZkTestCommunicationSpi commSpi = ZkTestCommunicationSpi.testSpi(ignite(3));
+
+ commSpi.pingLatch = new CountDownLatch(1);
+
+ IgniteInternalFuture<?> fut = GridTestUtils.runAsync(new Callable<Object>() {
+ @Override public Object call() {
+ ZookeeperDiscoverySpi spi = spi(ignite(1));
+
+ spi.resolveCommunicationFailure(ignite(2).cluster().localNode(), new Exception("test"));
+
+ return null;
+ }
+ });
+
+ U.sleep(1000);
+
+ assertFalse(fut.isDone());
+
+ stopGrid(0);
+
+ commSpi.pingLatch.countDown();
+
+ fut.get();
+
+ waitForTopology(3);
+ }
+
+ /**
+ * Tests that nodes join is delayed while resolve is in progress.
+ *
+ * @throws Exception If failed.
+ */
+ public void testNoOpCommunicationErrorResolve_5() throws Exception {
+ testCommSpi = true;
+
+ sesTimeout = 2000;
+ commFailureRslvr = NoOpCommunicationFailureResolver.FACTORY;
+
+ startGrid(0);
+
+ startGridsMultiThreaded(1, 3);
+
+ ZkTestCommunicationSpi commSpi = ZkTestCommunicationSpi.testSpi(ignite(3));
+
+ commSpi.pingStartLatch = new CountDownLatch(1);
+ commSpi.pingLatch = new CountDownLatch(1);
+
+ IgniteInternalFuture<?> fut = GridTestUtils.runAsync(new Callable<Object>() {
+ @Override public Object call() {
+ ZookeeperDiscoverySpi spi = spi(ignite(1));
+
+ spi.resolveCommunicationFailure(ignite(2).cluster().localNode(), new Exception("test"));
+
+ return null;
+ }
+ });
+
+ assertTrue(commSpi.pingStartLatch.await(10, SECONDS));
+
+ try {
+ assertFalse(fut.isDone());
+
+ final AtomicInteger nodeIdx = new AtomicInteger(3);
+
+ IgniteInternalFuture<?> startFut = GridTestUtils.runMultiThreadedAsync(new Callable<Void>() {
+ @Override public Void call() throws Exception {
+ startGrid(nodeIdx.incrementAndGet());
+
+ return null;
+ }
+ }, 3, "start-node");
+
+ U.sleep(1000);
+
+ assertFalse(startFut.isDone());
+
+ assertEquals(4, ignite(0).cluster().nodes().size());
+
+ commSpi.pingLatch.countDown();
+
+ startFut.get();
+ fut.get();
+
+ waitForTopology(7);
+ }
+ finally {
+ commSpi.pingLatch.countDown();
+ }
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testCommunicationErrorResolve_KillNode_1() throws Exception {
+ communicationFailureResolve_KillNodes(2, Collections.singleton(2L));
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testCommunicationErrorResolve_KillNode_2() throws Exception {
+ communicationFailureResolve_KillNodes(3, Collections.singleton(2L));
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testCommunicationErrorResolve_KillNode_3() throws Exception {
+ communicationFailureResolve_KillNodes(10, Arrays.asList(2L, 4L, 6L));
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testCommunicationErrorResolve_KillCoordinator_1() throws Exception {
+ communicationFailureResolve_KillNodes(2, Collections.singleton(1L));
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testCommunicationErrorResolve_KillCoordinator_2() throws Exception {
+ communicationFailureResolve_KillNodes(3, Collections.singleton(1L));
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testCommunicationErrorResolve_KillCoordinator_3() throws Exception {
+ communicationFailureResolve_KillNodes(10, Arrays.asList(1L, 4L, 6L));
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testCommunicationErrorResolve_KillCoordinator_4() throws Exception {
+ communicationFailureResolve_KillNodes(10, Arrays.asList(1L, 2L, 3L));
+ }
+
+ /**
+ * @param startNodes Number of nodes to start.
+ * @param killNodes Nodes to kill by resolve process.
+ * @throws Exception If failed.
+ */
+ private void communicationFailureResolve_KillNodes(int startNodes, Collection<Long> killNodes) throws Exception {
+ testCommSpi = true;
+
+ commFailureRslvr = TestNodeKillCommunicationFailureResolver.factory(killNodes);
+
+ startGrids(startNodes);
+
+ ZkTestCommunicationSpi commSpi = ZkTestCommunicationSpi.testSpi(ignite(0));
+
+ commSpi.checkRes = new BitSet(startNodes);
+
+ ZookeeperDiscoverySpi spi = null;
+ UUID killNodeId = null;
+
+ for (Ignite node : G.allGrids()) {
+ ZookeeperDiscoverySpi spi0 = spi(node);
+
+ if (!killNodes.contains(node.cluster().localNode().order()))
+ spi = spi0;
+ else
+ killNodeId = node.cluster().localNode().id();
+ }
+
+ assertNotNull(spi);
+ assertNotNull(killNodeId);
+
+ try {
+ spi.resolveCommunicationFailure(spi.getNode(killNodeId), new Exception("test"));
+
+ fail("Exception is not thrown");
+ }
+ catch (IgniteSpiException e) {
+ assertTrue("Unexpected exception: " + e, e.getCause() instanceof ClusterTopologyCheckedException);
+ }
+
+ int expNodes = startNodes - killNodes.size();
+
+ waitForTopology(expNodes);
+
+ for (Ignite node : G.allGrids())
+ assertFalse(killNodes.contains(node.cluster().localNode().order()));
+
+ startGrid(startNodes);
+
+ waitForTopology(expNodes + 1);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testCommunicationFailureResolve_KillCoordinator_5() throws Exception {
+ sesTimeout = 2000;
+
+ testCommSpi = true;
+ commFailureRslvr = KillCoordinatorCommunicationFailureResolver.FACTORY;
+
+ startGrids(10);
+
+ int crd = 0;
+
+ int nodeIdx = 10;
+
+ for (int i = 0; i < 10; i++) {
+ info("Iteration: " + i);
+
+ for (Ignite node : G.allGrids())
+ ZkTestCommunicationSpi.testSpi(node).initCheckResult(10);
+
+ UUID crdId = ignite(crd).cluster().localNode().id();
+
+ ZookeeperDiscoverySpi spi = spi(ignite(crd + 1));
+
+ try {
+ spi.resolveCommunicationFailure(spi.getNode(crdId), new Exception("test"));
+
+ fail("Exception is not thrown");
+ }
+ catch (IgniteSpiException e) {
+ assertTrue("Unexpected exception: " + e, e.getCause() instanceof ClusterTopologyCheckedException);
+ }
+
+ waitForTopology(9);
+
+ startGrid(nodeIdx++);
+
+ waitForTopology(10);
+
+ crd++;
+ }
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testCommunicationFailureResolve_KillRandom() throws Exception {
+ fail("https://issues.apache.org/jira/browse/IGNITE-8179");
+
+ sesTimeout = 2000;
+
+ testCommSpi = true;
+ commFailureRslvr = KillRandomCommunicationFailureResolver.FACTORY;
+
+ startGridsMultiThreaded(10);
+
+ clientMode(true);
+
+ startGridsMultiThreaded(10, 5);
+
+ int nodeIdx = 15;
+
+ for (int i = 0; i < 10; i++) {
+ info("Iteration: " + i);
+
+ ZookeeperDiscoverySpi spi = null;
+
+ for (Ignite node : G.allGrids()) {
+ ZkTestCommunicationSpi.testSpi(node).initCheckResult(100);
+
+ spi = spi(node);
+ }
+
+ assert spi != null;
+
+ try {
+ spi.resolveCommunicationFailure(spi.getRemoteNodes().iterator().next(), new Exception("test"));
+ }
+ catch (IgniteSpiException ignore) {
+ // No-op.
+ }
+
+ clientMode(ThreadLocalRandom.current().nextBoolean());
+
+ startGrid(nodeIdx++);
+
+ awaitPartitionMapExchange();
+ }
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testDefaultCommunicationFailureResolver1() throws Exception {
+ testCommSpi = true;
+ sesTimeout = 5000;
+
+ startGrids(3);
+
+ ZkTestCommunicationSpi.testSpi(ignite(0)).initCheckResult(3, 0, 1);
+ ZkTestCommunicationSpi.testSpi(ignite(1)).initCheckResult(3, 0, 1);
+ ZkTestCommunicationSpi.testSpi(ignite(2)).initCheckResult(3, 2);
+
+ UUID killedId = nodeId(2);
+
+ assertNotNull(ignite(0).cluster().node(killedId));
+
+ ZookeeperDiscoverySpi spi = spi(ignite(0));
+
+ spi.resolveCommunicationFailure(spi.getNode(ignite(1).cluster().localNode().id()), new Exception("test"));
+
+ waitForTopology(2);
+
+ assertNull(ignite(0).cluster().node(killedId));
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testDefaultCommunicationFailureResolver2() throws Exception {
+ testCommSpi = true;
+ sesTimeout = 5000;
+
+ startGrids(3);
+
+ clientMode(true);
+
+ startGridsMultiThreaded(3, 2);
+
+ ZkTestCommunicationSpi.testSpi(ignite(0)).initCheckResult(5, 0, 1);
+ ZkTestCommunicationSpi.testSpi(ignite(1)).initCheckResult(5, 0, 1);
+ ZkTestCommunicationSpi.testSpi(ignite(2)).initCheckResult(5, 2, 3, 4);
+ ZkTestCommunicationSpi.testSpi(ignite(3)).initCheckResult(5, 2, 3, 4);
+ ZkTestCommunicationSpi.testSpi(ignite(4)).initCheckResult(5, 2, 3, 4);
+
+ ZookeeperDiscoverySpi spi = spi(ignite(0));
+
+ spi.resolveCommunicationFailure(spi.getNode(ignite(1).cluster().localNode().id()), new Exception("test"));
+
+ waitForTopology(2);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testDefaultCommunicationFailureResolver3() throws Exception {
+ defaultCommunicationFailureResolver_BreakCommunication(3, 1);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testDefaultCommunicationFailureResolver4() throws Exception {
+ defaultCommunicationFailureResolver_BreakCommunication(3, 0);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testDefaultCommunicationFailureResolver5() throws Exception {
+ defaultCommunicationFailureResolver_BreakCommunication(10, 1, 3, 6);
+ }
+
+ /**
+ * @param startNodes Initial nodes number.
+ * @param breakNodes Node indices where communication server is closed.
+ * @throws Exception If failed.
+ */
+ private void defaultCommunicationFailureResolver_BreakCommunication(int startNodes, final int...breakNodes) throws Exception {
+ sesTimeout = 5000;
+
+ startGridsMultiThreaded(startNodes);
+
+ final CyclicBarrier b = new CyclicBarrier(breakNodes.length);
+
+ GridTestUtils.runMultiThreaded(new IgniteInClosure<Integer>() {
+ @Override public void apply(Integer threadIdx) {
+ try {
+ b.await();
+
+ int nodeIdx = breakNodes[threadIdx];
+
+ info("Close communication: " + nodeIdx);
+
+ ((TcpCommunicationSpi)ignite(nodeIdx).configuration().getCommunicationSpi()).simulateNodeFailure();
+ }
+ catch (Exception e) {
+ fail("Unexpected error: " + e);
+ }
+ }
+ }, breakNodes.length, "break-communication");
+
+ waitForTopology(startNodes - breakNodes.length);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testCommunicationFailureResolve_CachesInfo1() throws Exception {
+ testCommSpi = true;
+ sesTimeout = 5000;
+
+ final CacheInfoCommunicationFailureResolver rslvr = new CacheInfoCommunicationFailureResolver();
+
+ commFailureRslvr = new IgniteOutClosure<CommunicationFailureResolver>() {
+ @Override public CommunicationFailureResolver apply() {
+ return rslvr;
+ }
+ };
+
+ startGrids(2);
+
+ awaitPartitionMapExchange();
+
+ Map<String, T3<Integer, Integer, Integer>> expCaches = new HashMap<>();
+
+ expCaches.put(DEFAULT_CACHE_NAME, new T3<>(RendezvousAffinityFunction.DFLT_PARTITION_COUNT, 0, 1));
+
+ checkResolverCachesInfo(ignite(0), expCaches);
+
+ List<CacheConfiguration> caches = new ArrayList<>();
+
+ CacheConfiguration c1 = new CacheConfiguration("c1");
+ c1.setBackups(1);
+ c1.setAffinity(new RendezvousAffinityFunction(false, 64));
+ caches.add(c1);
+
+ CacheConfiguration c2 = new CacheConfiguration("c2");
+ c2.setBackups(2);
+ c2.setAffinity(new RendezvousAffinityFunction(false, 128));
+ caches.add(c2);
+
+ CacheConfiguration c3 = new CacheConfiguration("c3");
+ c3.setCacheMode(CacheMode.REPLICATED);
+ c3.setAffinity(new RendezvousAffinityFunction(false, 256));
+ caches.add(c3);
+
+ ignite(0).createCaches(caches);
+
+ expCaches.put("c1", new T3<>(64, 1, 2));
+ expCaches.put("c2", new T3<>(128, 2, 2));
+ expCaches.put("c3", new T3<>(256, 1, 2));
+
+ checkResolverCachesInfo(ignite(0), expCaches);
+
+ startGrid(2);
+ startGrid(3);
+
+ awaitPartitionMapExchange();
+
+ expCaches.put("c2", new T3<>(128, 2, 3));
+ expCaches.put("c3", new T3<>(256, 1, 4));
+
+ checkResolverCachesInfo(ignite(0), expCaches);
+
+ CacheConfiguration c4 = new CacheConfiguration("c4");
+ c4.setCacheMode(CacheMode.PARTITIONED);
+ c4.setBackups(0);
+ c4.setAffinity(new RendezvousAffinityFunction(false, 256));
+ c4.setNodeFilter(new TestCacheNodeExcludingFilter(getTestIgniteInstanceName(0), getTestIgniteInstanceName(1)));
+
+ ignite(2).createCache(c4);
+
+ expCaches.put("c4", new T3<>(256, 0, 1));
+
+ checkResolverCachesInfo(ignite(0), expCaches);
+
+ stopGrid(0); // Stop current coordinator, check new coordinator will initialize required caches information.
+
+ awaitPartitionMapExchange();
+
+ expCaches.put("c3", new T3<>(256, 1, 3));
+
+ checkResolverCachesInfo(ignite(1), expCaches);
+
+ startGrid(0);
+
+ expCaches.put("c3", new T3<>(256, 1, 4));
+
+ checkResolverCachesInfo(ignite(1), expCaches);
+
+ stopGrid(1);
+
+ expCaches.put("c3", new T3<>(256, 1, 3));
+
+ checkResolverCachesInfo(ignite(3), expCaches);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testCommunicationFailureResolve_CachesInfo2() throws Exception {
+ testCommSpi = true;
+ sesTimeout = 5000;
+
+ final CacheInfoCommunicationFailureResolver rslvr = new CacheInfoCommunicationFailureResolver();
+
+ commFailureRslvr = new IgniteOutClosure<CommunicationFailureResolver>() {
+ @Override public CommunicationFailureResolver apply() {
+ return rslvr;
+ }
+ };
+
+ Ignite srv0 = startGrid(0);
+
+ CacheConfiguration ccfg = new CacheConfiguration("c1
<TRUNCATED>
[11/12] ignite git commit: IGNITE-7222 Added ZooKeeper discovery SPI
Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataTransport.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataTransport.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataTransport.java
index 9402a32..38450df 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataTransport.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataTransport.java
@@ -155,22 +155,30 @@ final class BinaryMetadataTransport {
* @param metadata Metadata proposed for update.
* @return Future to wait for update result on.
*/
- GridFutureAdapter<MetadataUpdateResult> requestMetadataUpdate(BinaryMetadata metadata) throws IgniteCheckedException {
+ GridFutureAdapter<MetadataUpdateResult> requestMetadataUpdate(BinaryMetadata metadata) {
MetadataUpdateResultFuture resFut = new MetadataUpdateResultFuture();
if (log.isDebugEnabled())
log.debug("Requesting metadata update for " + metadata.typeId() + "; caller thread is blocked on future "
+ resFut);
- synchronized (this) {
- unlabeledFutures.add(resFut);
+ try {
+ synchronized (this) {
+ unlabeledFutures.add(resFut);
- if (!stopping)
- discoMgr.sendCustomEvent(new MetadataUpdateProposedMessage(metadata, ctx.localNodeId()));
- else
- resFut.onDone(MetadataUpdateResult.createUpdateDisabledResult());
+ if (!stopping)
+ discoMgr.sendCustomEvent(new MetadataUpdateProposedMessage(metadata, ctx.localNodeId()));
+ else
+ resFut.onDone(MetadataUpdateResult.createUpdateDisabledResult());
+ }
+ }
+ catch (Exception e) {
+ resFut.onDone(MetadataUpdateResult.createUpdateDisabledResult(), e);
}
+ if (ctx.clientDisconnected())
+ onDisconnected();
+
return resFut;
}
@@ -237,6 +245,8 @@ final class BinaryMetadataTransport {
for (MetadataUpdateResultFuture fut : unlabeledFutures)
fut.onDone(res);
+ unlabeledFutures.clear();
+
for (MetadataUpdateResultFuture fut : syncMap.values())
fut.onDone(res);
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/MetadataUpdateAcceptedMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/MetadataUpdateAcceptedMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/MetadataUpdateAcceptedMessage.java
index 0416746..df64613 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/MetadataUpdateAcceptedMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/MetadataUpdateAcceptedMessage.java
@@ -71,6 +71,11 @@ public class MetadataUpdateAcceptedMessage implements DiscoveryCustomMessage {
}
/** {@inheritDoc} */
+ @Override public boolean stopProcess() {
+ return false;
+ }
+
+ /** {@inheritDoc} */
@Nullable @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr,
AffinityTopologyVersion topVer, DiscoCache discoCache) {
throw new UnsupportedOperationException();
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/MetadataUpdateProposedMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/MetadataUpdateProposedMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/MetadataUpdateProposedMessage.java
index f9bd660..84e32e1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/MetadataUpdateProposedMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/MetadataUpdateProposedMessage.java
@@ -134,6 +134,11 @@ public final class MetadataUpdateProposedMessage implements DiscoveryCustomMessa
}
/** {@inheritDoc} */
+ @Override public boolean stopProcess() {
+ return false;
+ }
+
+ /** {@inheritDoc} */
@Nullable @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr,
AffinityTopologyVersion topVer, DiscoCache discoCache) {
throw new UnsupportedOperationException();
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
index 9b3c1ec..5bbbb31 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
@@ -598,6 +598,45 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
}
/** {@inheritDoc} */
+ @Override public List<List<ClusterNode>> allOwners() {
+ lock.readLock().lock();
+
+ try {
+ int parts = partitions();
+
+ List<List<ClusterNode>> res = new ArrayList<>(parts);
+
+ for (int i = 0; i < parts; i++)
+ res.add(new ArrayList<>());
+
+ List<ClusterNode> allNodes = discoCache.cacheGroupAffinityNodes(grpId);
+
+ for (int i = 0; i < allNodes.size(); i++) {
+ ClusterNode node = allNodes.get(i);
+
+ GridDhtPartitionMap nodeParts = node2part.get(node.id());
+
+ if (nodeParts != null) {
+ for (Map.Entry<Integer, GridDhtPartitionState> e : nodeParts.map().entrySet()) {
+ if (e.getValue() == OWNING) {
+ int part = e.getKey();
+
+ List<ClusterNode> owners = res.get(part);
+
+ owners.add(node);
+ }
+ }
+ }
+ }
+
+ return res;
+ }
+ finally {
+ lock.readLock().unlock();
+ }
+ }
+
+ /** {@inheritDoc} */
@Override public List<ClusterNode> moving(int p) {
return nodes(p, AffinityTopologyVersion.NONE, MOVING, null);
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
index ba55543..ea99f5d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
@@ -970,6 +970,10 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
try {
ctx.io().send(nodeId, res, ctx.ioPolicy());
}
+ catch (ClusterTopologyCheckedException e) {
+ if (log.isDebugEnabled())
+ log.debug("Failed to send get response to node, node failed: " + nodeId);
+ }
catch (IgniteCheckedException e) {
U.error(log, "Failed to send get response to node (is node still alive?) [nodeId=" + nodeId +
",req=" + req + ", res=" + res + ']', e);
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
index 13564c2..7f900cb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
@@ -237,6 +237,12 @@ public interface GridDhtPartitionTopology {
public List<ClusterNode> owners(int p);
/**
+ * @return List indexed by partition number, each list element is collection of all nodes who
+ * owns corresponding partition.
+ */
+ public List<List<ClusterNode>> allOwners();
+
+ /**
* @param p Partition ID.
* @param topVer Topology version.
* @return Collection of all nodes who {@code own} this partition.
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
index 528f0a6..538c57e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
@@ -1217,6 +1217,45 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
}
/** {@inheritDoc} */
+ @Override public List<List<ClusterNode>> allOwners() {
+ lock.readLock().lock();
+
+ try {
+ int parts = partitions();
+
+ List<List<ClusterNode>> res = new ArrayList<>(parts);
+
+ for (int i = 0; i < parts; i++)
+ res.add(new ArrayList<>());
+
+ List<ClusterNode> allNodes = discoCache.cacheGroupAffinityNodes(grp.groupId());
+
+ for (int i = 0; i < allNodes.size(); i++) {
+ ClusterNode node = allNodes.get(i);
+
+ GridDhtPartitionMap nodeParts = node2part.get(node.id());
+
+ if (nodeParts != null) {
+ for (Map.Entry<Integer, GridDhtPartitionState> e : nodeParts.map().entrySet()) {
+ if (e.getValue() == OWNING) {
+ int part = e.getKey();
+
+ List<ClusterNode> owners = res.get(part);
+
+ owners.add(node);
+ }
+ }
+ }
+ }
+
+ return res;
+ }
+ finally {
+ lock.readLock().unlock();
+ }
+ }
+
+ /** {@inheritDoc} */
@Override public List<ClusterNode> moving(int p) {
if (!grp.rebalanceEnabled())
return ownersAndMoving(p, AffinityTopologyVersion.NONE);
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
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 8da91a8..cbb4985 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
@@ -1506,12 +1506,16 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
}
catch (ClusterTopologyCheckedException ignore) {
if (log.isDebugEnabled())
- log.debug("Oldest node left during partition exchange [nodeId=" + oldestNode.id() +
+ log.debug("Coordinator left during partition exchange [nodeId=" + oldestNode.id() +
", exchId=" + exchId + ']');
}
catch (IgniteCheckedException e) {
- U.error(log, "Failed to send local partitions to oldest node (will retry after timeout) [oldestNodeId=" +
- oldestNode.id() + ", exchId=" + exchId + ']', e);
+ if (reconnectOnError(e))
+ onDone(new IgniteNeedReconnectException(cctx.localNode(), e));
+ else {
+ U.error(log, "Failed to send local partitions to coordinator [crd=" + oldestNode.id() +
+ ", exchId=" + exchId + ']', e);
+ }
}
}
@@ -3369,9 +3373,13 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
}
if (allReceived) {
- awaitSingleMapUpdates();
+ cctx.kernalContext().getSystemExecutorService().submit(new Runnable() {
+ @Override public void run() {
+ awaitSingleMapUpdates();
- onAllReceived(null);
+ onAllReceived(null);
+ }
+ });
}
}
else {
@@ -3399,7 +3407,13 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
", newCrd=" + crd0.id() + ']');
}
- sendPartitions(crd0);
+ final ClusterNode newCrd = crd0;
+
+ cctx.kernalContext().getSystemExecutorService().submit(new Runnable() {
+ @Override public void run() {
+ sendPartitions(newCrd);
+ }
+ });
}
}
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateFinishMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateFinishMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateFinishMessage.java
index d7dfa16..bbbd999 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateFinishMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateFinishMessage.java
@@ -94,6 +94,11 @@ public class ChangeGlobalStateFinishMessage implements DiscoveryCustomMessage {
}
/** {@inheritDoc} */
+ @Override public boolean stopProcess() {
+ return false;
+ }
+
+ /** {@inheritDoc} */
@Nullable @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr,
AffinityTopologyVersion topVer, DiscoCache discoCache) {
throw new UnsupportedOperationException();
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateMessage.java
index 50fc022..81855fc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateMessage.java
@@ -131,6 +131,11 @@ public class ChangeGlobalStateMessage implements DiscoveryCustomMessage {
}
/** {@inheritDoc} */
+ @Override public boolean stopProcess() {
+ return false;
+ }
+
+ /** {@inheritDoc} */
@Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr, AffinityTopologyVersion topVer,
DiscoCache discoCache) {
return mgr.createDiscoCacheOnCacheChange(topVer, discoCache);
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterMetricsUpdateMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterMetricsUpdateMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterMetricsUpdateMessage.java
new file mode 100644
index 0000000..5c3044b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterMetricsUpdateMessage.java
@@ -0,0 +1,158 @@
+/*
+ * 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.processors.cluster;
+
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.internal.GridDirectMap;
+import org.apache.ignite.internal.managers.communication.GridIoMessageFactory;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
+import org.apache.ignite.plugin.extensions.communication.MessageReader;
+import org.apache.ignite.plugin.extensions.communication.MessageWriter;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ *
+ */
+public class ClusterMetricsUpdateMessage implements Message {
+ /** */
+ private static final long serialVersionUID = 0L;
+
+ /** */
+ private byte[] nodeMetrics;
+
+ /** */
+ @GridDirectMap(keyType = UUID.class, valueType = byte[].class)
+ private Map<UUID, byte[]> allNodesMetrics;
+
+ /**
+ * Required by {@link GridIoMessageFactory}.
+ */
+ public ClusterMetricsUpdateMessage() {
+ // No-op.
+ }
+
+ /**
+ * @param nodeMetrics Node metrics.
+ */
+ ClusterMetricsUpdateMessage(byte[] nodeMetrics) {
+ this.nodeMetrics = nodeMetrics;
+ }
+
+ /**
+ * @param allNodesMetrics All nodes metrcis.
+ */
+ ClusterMetricsUpdateMessage(Map<UUID, byte[]> allNodesMetrics) {
+ this.allNodesMetrics = allNodesMetrics;
+ }
+
+ /**
+ * @return Node metrics.
+ */
+ @Nullable byte[] nodeMetrics() {
+ return nodeMetrics;
+ }
+
+ /**
+ * @return All nodes metrics.
+ */
+ @Nullable Map<UUID, byte[]> allNodesMetrics() {
+ return allNodesMetrics;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+ writer.setBuffer(buf);
+
+ if (!writer.isHeaderWritten()) {
+ if (!writer.writeHeader(directType(), fieldsCount()))
+ return false;
+
+ writer.onHeaderWritten();
+ }
+
+ switch (writer.state()) {
+ case 0:
+ if (!writer.writeMap("allNodesMetrics", allNodesMetrics, MessageCollectionItemType.UUID, MessageCollectionItemType.BYTE_ARR))
+ return false;
+
+ writer.incrementState();
+
+ case 1:
+ if (!writer.writeByteArray("nodeMetrics", nodeMetrics))
+ return false;
+
+ writer.incrementState();
+
+ }
+
+ return true;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+ reader.setBuffer(buf);
+
+ if (!reader.beforeMessageRead())
+ return false;
+
+ switch (reader.state()) {
+ case 0:
+ allNodesMetrics = reader.readMap("allNodesMetrics", MessageCollectionItemType.UUID, MessageCollectionItemType.BYTE_ARR, false);
+
+ if (!reader.isLastRead())
+ return false;
+
+ reader.incrementState();
+
+ case 1:
+ nodeMetrics = reader.readByteArray("nodeMetrics");
+
+ if (!reader.isLastRead())
+ return false;
+
+ reader.incrementState();
+
+ }
+
+ return reader.afterMessageRead(ClusterMetricsUpdateMessage.class);
+ }
+
+ /** {@inheritDoc} */
+ @Override public short directType() {
+ return 133;
+ }
+
+ /** {@inheritDoc} */
+ @Override public byte fieldsCount() {
+ return 2;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void onAckReceived() {
+ // No-op.
+ }
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return S.toString(ClusterMetricsUpdateMessage.class, this);
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterNodeMetrics.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterNodeMetrics.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterNodeMetrics.java
new file mode 100644
index 0000000..22a385f
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterNodeMetrics.java
@@ -0,0 +1,62 @@
+/*
+ * 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.processors.cluster;
+
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.Map;
+import org.apache.ignite.cache.CacheMetrics;
+import org.apache.ignite.cluster.ClusterMetrics;
+import org.apache.ignite.internal.ClusterMetricsSnapshot;
+
+/**
+ *
+ */
+class ClusterNodeMetrics implements Serializable {
+ /** */
+ private static final long serialVersionUID = 0L;
+
+ /** */
+ private final byte[] metrics;
+
+ /** */
+ private final Map<Integer, CacheMetrics> cacheMetrics;
+
+ /**
+ * @param metrics Metrics.
+ * @param cacheMetrics Cache metrics.
+ */
+ ClusterNodeMetrics(ClusterMetrics metrics, Map<Integer, CacheMetrics> cacheMetrics) {
+ this.metrics = ClusterMetricsSnapshot.serialize(metrics);
+ this.cacheMetrics = cacheMetrics;
+ }
+
+ /**
+ * @return Metrics.
+ */
+ byte[] metrics() {
+ return metrics;
+ }
+
+ /**
+ * @return Cache metrics.
+ */
+ Map<Integer, CacheMetrics> cacheMetrics() {
+ return cacheMetrics != null ? cacheMetrics : Collections.<Integer, CacheMetrics>emptyMap();
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java
index 5f2c66c..8796302 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java
@@ -33,6 +33,8 @@ import org.apache.ignite.IgniteSystemProperties;
import org.apache.ignite.cluster.ClusterNode;
import org.apache.ignite.events.DiscoveryEvent;
import org.apache.ignite.events.Event;
+import org.apache.ignite.internal.ClusterMetricsSnapshot;
+import org.apache.ignite.internal.GridDirectMap;
import org.apache.ignite.internal.GridKernalContext;
import org.apache.ignite.internal.IgniteDiagnosticInfo;
import org.apache.ignite.internal.IgniteDiagnosticMessage;
@@ -42,21 +44,29 @@ import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
import org.apache.ignite.internal.cluster.IgniteClusterImpl;
import org.apache.ignite.internal.managers.communication.GridIoPolicy;
import org.apache.ignite.internal.managers.communication.GridMessageListener;
+import org.apache.ignite.internal.managers.discovery.DiscoCache;
+import org.apache.ignite.internal.managers.discovery.IgniteClusterNode;
import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
import org.apache.ignite.internal.processors.GridProcessorAdapter;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
import org.apache.ignite.internal.util.GridTimerTask;
import org.apache.ignite.internal.util.future.GridFinishedFuture;
import org.apache.ignite.internal.util.future.GridFutureAdapter;
import org.apache.ignite.internal.util.future.IgniteFinishedFutureImpl;
import org.apache.ignite.internal.util.tostring.GridToStringExclude;
import org.apache.ignite.internal.util.typedef.CI1;
+import org.apache.ignite.internal.util.typedef.internal.LT;
import org.apache.ignite.internal.util.typedef.internal.S;
import org.apache.ignite.internal.util.typedef.internal.U;
import org.apache.ignite.lang.IgniteClosure;
import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.lang.IgniteUuid;
import org.apache.ignite.marshaller.jdk.JdkMarshaller;
import org.apache.ignite.spi.discovery.DiscoveryDataBag;
import org.apache.ignite.spi.discovery.DiscoveryDataBag.GridDiscoveryData;
+import org.apache.ignite.spi.discovery.DiscoveryMetricsProvider;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
import org.jetbrains.annotations.Nullable;
import static org.apache.ignite.IgniteSystemProperties.IGNITE_DIAGNOSTIC_ENABLED;
@@ -66,6 +76,7 @@ import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
import static org.apache.ignite.internal.GridComponent.DiscoveryDataExchangeType.CLUSTER_PROC;
import static org.apache.ignite.internal.GridTopic.TOPIC_INTERNAL_DIAGNOSTIC;
+import static org.apache.ignite.internal.GridTopic.TOPIC_METRICS;
import static org.apache.ignite.internal.IgniteVersionUtils.VER_STR;
/**
@@ -102,6 +113,18 @@ public class ClusterProcessor extends GridProcessorAdapter {
/** */
private final AtomicLong diagFutId = new AtomicLong();
+ /** */
+ private final Map<UUID, byte[]> allNodesMetrics = new ConcurrentHashMap<>();
+
+ /** */
+ private final JdkMarshaller marsh = new JdkMarshaller();
+
+ /** */
+ private DiscoveryMetricsProvider metricsProvider;
+
+ /** */
+ private boolean sndMetrics;
+
/**
* @param ctx Kernal context.
*/
@@ -111,6 +134,8 @@ public class ClusterProcessor extends GridProcessorAdapter {
notifyEnabled.set(IgniteSystemProperties.getBoolean(IGNITE_UPDATE_NOTIFIER, true));
cluster = new IgniteClusterImpl(ctx);
+
+ sndMetrics = !(ctx.config().getDiscoverySpi() instanceof TcpDiscoverySpi);
}
/**
@@ -120,33 +145,31 @@ public class ClusterProcessor extends GridProcessorAdapter {
return getBoolean(IGNITE_DIAGNOSTIC_ENABLED, true);
}
- /** */
- private final JdkMarshaller marsh = new JdkMarshaller();
-
/**
* @throws IgniteCheckedException If failed.
*/
public void initDiagnosticListeners() throws IgniteCheckedException {
ctx.event().addLocalEventListener(new GridLocalEventListener() {
- @Override public void onEvent(Event evt) {
- assert evt instanceof DiscoveryEvent;
- assert evt.type() == EVT_NODE_FAILED || evt.type() == EVT_NODE_LEFT;
+ @Override public void onEvent(Event evt) {
+ assert evt instanceof DiscoveryEvent;
+ assert evt.type() == EVT_NODE_FAILED || evt.type() == EVT_NODE_LEFT;
- DiscoveryEvent discoEvt = (DiscoveryEvent)evt;
+ DiscoveryEvent discoEvt = (DiscoveryEvent)evt;
- UUID nodeId = discoEvt.eventNode().id();
+ UUID nodeId = discoEvt.eventNode().id();
- ConcurrentHashMap<Long, InternalDiagnosticFuture> futs = diagnosticFutMap.get();
+ ConcurrentHashMap<Long, InternalDiagnosticFuture> futs = diagnosticFutMap.get();
- if (futs != null) {
- for (InternalDiagnosticFuture fut : futs.values()) {
- if (fut.nodeId.equals(nodeId))
- fut.onDone(new IgniteDiagnosticInfo("Target node failed: " + nodeId));
- }
+ if (futs != null) {
+ for (InternalDiagnosticFuture fut : futs.values()) {
+ if (fut.nodeId.equals(nodeId))
+ fut.onDone(new IgniteDiagnosticInfo("Target node failed: " + nodeId));
}
}
- },
- EVT_NODE_FAILED, EVT_NODE_LEFT);
+
+ allNodesMetrics.remove(nodeId);
+ }
+ }, EVT_NODE_FAILED, EVT_NODE_LEFT);
ctx.io().addMessageListener(TOPIC_INTERNAL_DIAGNOSTIC, new GridMessageListener() {
@Override public void onMessage(UUID nodeId, Object msg, byte plc) {
@@ -233,6 +256,17 @@ public class ClusterProcessor extends GridProcessorAdapter {
U.warn(diagnosticLog, "Received unexpected message: " + msg);
}
});
+
+ if (sndMetrics) {
+ ctx.io().addMessageListener(TOPIC_METRICS, new GridMessageListener() {
+ @Override public void onMessage(UUID nodeId, Object msg, byte plc) {
+ if (msg instanceof ClusterMetricsUpdateMessage)
+ processMetricsUpdateMessage(nodeId, (ClusterMetricsUpdateMessage)msg);
+ else
+ U.warn(log, "Received unexpected message for TOPIC_METRICS: " + msg);
+ }
+ });
+ }
}
/**
@@ -296,7 +330,6 @@ public class ClusterProcessor extends GridProcessorAdapter {
}
}
-
/**
* @param vals collection to seek through.
*/
@@ -334,6 +367,14 @@ public class ClusterProcessor extends GridProcessorAdapter {
log.debug("Failed to create GridUpdateNotifier: " + e);
}
}
+
+ if (sndMetrics) {
+ metricsProvider = ctx.discovery().createMetricsProvider();
+
+ long updateFreq = ctx.config().getMetricsUpdateFrequency();
+
+ ctx.timeout().addTimeoutObject(new MetricsUpdateTimeoutObject(updateFreq));
+ }
}
/** {@inheritDoc} */
@@ -352,6 +393,133 @@ public class ClusterProcessor extends GridProcessorAdapter {
}
/**
+ * @param sndNodeId Sender node ID.
+ * @param msg Message.
+ */
+ private void processMetricsUpdateMessage(UUID sndNodeId, ClusterMetricsUpdateMessage msg) {
+ byte[] nodeMetrics = msg.nodeMetrics();
+
+ if (nodeMetrics != null) {
+ assert msg.allNodesMetrics() == null;
+
+ allNodesMetrics.put(sndNodeId, nodeMetrics);
+
+ updateNodeMetrics(ctx.discovery().discoCache(), sndNodeId, nodeMetrics);
+ }
+ else {
+ Map<UUID, byte[]> allNodesMetrics = msg.allNodesMetrics();
+
+ assert allNodesMetrics != null;
+
+ DiscoCache discoCache = ctx.discovery().discoCache();
+
+ for (Map.Entry<UUID, byte[]> e : allNodesMetrics.entrySet()) {
+ if (!ctx.localNodeId().equals(e.getKey()))
+ updateNodeMetrics(discoCache, e.getKey(), e.getValue());
+ }
+ }
+ }
+
+ /**
+ * @param discoCache Discovery data cache.
+ * @param nodeId Node ID.
+ * @param metricsBytes Marshalled metrics.
+ */
+ private void updateNodeMetrics(DiscoCache discoCache, UUID nodeId, byte[] metricsBytes) {
+ ClusterNode node = discoCache.node(nodeId);
+
+ if (node == null || !discoCache.alive(nodeId))
+ return;
+
+ try {
+ ClusterNodeMetrics metrics = U.unmarshalZip(ctx.config().getMarshaller(), metricsBytes, null);
+
+ assert node instanceof IgniteClusterNode : node;
+
+ IgniteClusterNode node0 = (IgniteClusterNode)node;
+
+ node0.setMetrics(ClusterMetricsSnapshot.deserialize(metrics.metrics(), 0));
+ node0.setCacheMetrics(metrics.cacheMetrics());
+
+ ctx.discovery().metricsUpdateEvent(discoCache, node0);
+ }
+ catch (IgniteCheckedException e) {
+ U.warn(log, "Failed to unmarshal node metrics: " + e);
+ }
+ }
+
+ /**
+ *
+ */
+ private void updateMetrics() {
+ if (ctx.isStopping() || ctx.clientDisconnected())
+ return;
+
+ ClusterNode oldest = ctx.discovery().oldestAliveServerNode(AffinityTopologyVersion.NONE);
+
+ if (oldest == null)
+ return;
+
+ if (ctx.localNodeId().equals(oldest.id())) {
+ IgniteClusterNode locNode = (IgniteClusterNode)ctx.discovery().localNode();
+
+ locNode.setMetrics(metricsProvider.metrics());
+ locNode.setCacheMetrics(metricsProvider.cacheMetrics());
+
+ ClusterNodeMetrics metrics = new ClusterNodeMetrics(locNode.metrics(), locNode.cacheMetrics());
+
+ try {
+ byte[] metricsBytes = U.zip(U.marshal(ctx.config().getMarshaller(), metrics));
+
+ allNodesMetrics.put(ctx.localNodeId(), metricsBytes);
+ }
+ catch (IgniteCheckedException e) {
+ U.warn(log, "Failed to marshal local node metrics: " + e, e);
+ }
+
+ ctx.discovery().metricsUpdateEvent(ctx.discovery().discoCache(), locNode);
+
+ Collection<ClusterNode> allNodes = ctx.discovery().allNodes();
+
+ ClusterMetricsUpdateMessage msg = new ClusterMetricsUpdateMessage(new HashMap<>(allNodesMetrics));
+
+ for (ClusterNode node : allNodes) {
+ if (ctx.localNodeId().equals(node.id()) || !ctx.discovery().alive(node.id()))
+ continue;
+
+ try {
+ ctx.io().sendToGridTopic(node, TOPIC_METRICS, msg, GridIoPolicy.SYSTEM_POOL);
+ }
+ catch (ClusterTopologyCheckedException e) {
+ if (log.isDebugEnabled())
+ log.debug("Failed to send metrics update, node failed: " + e);
+ }
+ catch (IgniteCheckedException e) {
+ U.warn(log, "Failed to send metrics update: " + e, e);
+ }
+ }
+ }
+ else {
+ ClusterNodeMetrics metrics = new ClusterNodeMetrics(metricsProvider.metrics(), metricsProvider.cacheMetrics());
+
+ try {
+ byte[] metricsBytes = U.zip(U.marshal(ctx.config().getMarshaller(), metrics));
+
+ ClusterMetricsUpdateMessage msg = new ClusterMetricsUpdateMessage(metricsBytes);
+
+ ctx.io().sendToGridTopic(oldest, TOPIC_METRICS, msg, GridIoPolicy.SYSTEM_POOL);
+ }
+ catch (ClusterTopologyCheckedException e) {
+ if (log.isDebugEnabled())
+ log.debug("Failed to send metrics update to oldest, node failed: " + e);
+ }
+ catch (IgniteCheckedException e) {
+ LT.warn(log, e, "Failed to send metrics update to oldest: " + e, false, false);
+ }
+ }
+ }
+
+ /**
* Disables update notifier.
*/
public void disableUpdateNotifier() {
@@ -571,4 +739,51 @@ public class ClusterProcessor extends GridProcessorAdapter {
return S.toString(InternalDiagnosticFuture.class, this);
}
}
+
+ /**
+ *
+ */
+ private class MetricsUpdateTimeoutObject implements GridTimeoutObject, Runnable {
+ /** */
+ private final IgniteUuid id = IgniteUuid.randomUuid();
+
+ /** */
+ private long endTime;
+
+ /** */
+ private final long timeout;
+
+ /**
+ * @param timeout Timeout.
+ */
+ MetricsUpdateTimeoutObject(long timeout) {
+ this.timeout = timeout;
+
+ endTime = U.currentTimeMillis() + timeout;
+ }
+
+ /** {@inheritDoc} */
+ @Override public IgniteUuid timeoutId() {
+ return id;
+ }
+
+ /** {@inheritDoc} */
+ @Override public long endTime() {
+ return endTime;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void run() {
+ updateMetrics();
+
+ endTime = U.currentTimeMillis() + timeout;
+
+ ctx.timeout().addTimeoutObject(this);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void onTimeout() {
+ ctx.getSystemExecutorService().execute(this);
+ }
+ }
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/AbstractContinuousMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/AbstractContinuousMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/AbstractContinuousMessage.java
index e9754d1..928c619 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/AbstractContinuousMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/AbstractContinuousMessage.java
@@ -63,6 +63,11 @@ public abstract class AbstractContinuousMessage implements DiscoveryCustomMessag
}
/** {@inheritDoc} */
+ @Override public boolean stopProcess() {
+ return false;
+ }
+
+ /** {@inheritDoc} */
@Nullable @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr,
AffinityTopologyVersion topVer, DiscoCache discoCache) {
throw new UnsupportedOperationException();
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutineInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutineInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutineInfo.java
new file mode 100644
index 0000000..fc0f181
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutineInfo.java
@@ -0,0 +1,100 @@
+/*
+ * 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.processors.continuous;
+
+import java.io.Serializable;
+import java.util.UUID;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ *
+ */
+class ContinuousRoutineInfo implements Serializable {
+ /** */
+ private static final long serialVersionUID = 0L;
+
+ /** */
+ UUID srcNodeId;
+
+ /** */
+ final UUID routineId;
+
+ /** */
+ final byte[] hnd;
+
+ /** */
+ final byte[] nodeFilter;
+
+ /** */
+ final int bufSize;
+
+ /** */
+ final long interval;
+
+ /** */
+ final boolean autoUnsubscribe;
+
+ /** */
+ transient boolean disconnected;
+
+ /**
+ * @param srcNodeId Source node ID.
+ * @param routineId Routine ID.
+ * @param hnd Marshalled handler.
+ * @param nodeFilter Marshalled node filter.
+ * @param bufSize Handler buffer size.
+ * @param interval Time interval.
+ * @param autoUnsubscribe Auto unsubscribe flag.
+ */
+ ContinuousRoutineInfo(
+ UUID srcNodeId,
+ UUID routineId,
+ byte[] hnd,
+ byte[] nodeFilter,
+ int bufSize,
+ long interval,
+ boolean autoUnsubscribe)
+ {
+ this.srcNodeId = srcNodeId;
+ this.routineId = routineId;
+ this.hnd = hnd;
+ this.nodeFilter = nodeFilter;
+ this.bufSize = bufSize;
+ this.interval = interval;
+ this.autoUnsubscribe = autoUnsubscribe;
+ }
+
+ /**
+ * @param srcNodeId Source node ID.
+ */
+ void sourceNodeId(UUID srcNodeId) {
+ this.srcNodeId = srcNodeId;
+ }
+
+ /**
+ *
+ */
+ void onDisconnected() {
+ disconnected = true;
+ }
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return S.toString(ContinuousRoutineInfo.class, this);
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutineStartResultMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutineStartResultMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutineStartResultMessage.java
new file mode 100644
index 0000000..581ac60
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutineStartResultMessage.java
@@ -0,0 +1,206 @@
+/*
+ * 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.processors.continuous;
+
+import java.nio.ByteBuffer;
+import java.util.UUID;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.CachePartitionPartialCountersMap;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.plugin.extensions.communication.MessageReader;
+import org.apache.ignite.plugin.extensions.communication.MessageWriter;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ *
+ */
+public class ContinuousRoutineStartResultMessage implements Message {
+ /** */
+ private static final long serialVersionUID = 0L;
+
+ /** */
+ private static final int ERROR_FLAG = 0x01;
+
+ /** */
+ private UUID routineId;
+
+ /** */
+ private byte[] errBytes;
+
+ /** */
+ private byte[] cntrsMapBytes;
+
+ /** */
+ private int flags;
+
+ /**
+ *
+ */
+ public ContinuousRoutineStartResultMessage() {
+ // No-op.
+ }
+
+ /**
+ * @param routineId Routine ID.
+ * @param cntrsMapBytes Marshalled {@link CachePartitionPartialCountersMap}.
+ * @param errBytes Error bytes.
+ * @param err {@code True} if failed to start routine.
+ */
+ ContinuousRoutineStartResultMessage(UUID routineId, byte[] cntrsMapBytes, byte[] errBytes, boolean err) {
+ this.routineId = routineId;
+ this.cntrsMapBytes = cntrsMapBytes;
+ this.errBytes = errBytes;
+
+ if (err)
+ flags |= ERROR_FLAG;
+ }
+
+ /**
+ * @return Marshalled {@link CachePartitionPartialCountersMap}.
+ */
+ @Nullable byte[] countersMapBytes() {
+ return cntrsMapBytes;
+ }
+
+ /**
+ * @return {@code True} if failed to start routine.
+ */
+ boolean error() {
+ return (flags & ERROR_FLAG) != 0;
+ }
+
+ /**
+ * @return Routine ID.
+ */
+ UUID routineId() {
+ return routineId;
+ }
+
+ /**
+ * @return Error bytes.
+ */
+ @Nullable byte[] errorBytes() {
+ return errBytes;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+ writer.setBuffer(buf);
+
+ if (!writer.isHeaderWritten()) {
+ if (!writer.writeHeader(directType(), fieldsCount()))
+ return false;
+
+ writer.onHeaderWritten();
+ }
+
+ switch (writer.state()) {
+ case 0:
+ if (!writer.writeByteArray("cntrsMapBytes", cntrsMapBytes))
+ return false;
+
+ writer.incrementState();
+
+ case 1:
+ if (!writer.writeByteArray("errBytes", errBytes))
+ return false;
+
+ writer.incrementState();
+
+ case 2:
+ if (!writer.writeInt("flags", flags))
+ return false;
+
+ writer.incrementState();
+
+ case 3:
+ if (!writer.writeUuid("routineId", routineId))
+ return false;
+
+ writer.incrementState();
+
+ }
+
+ return true;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+ reader.setBuffer(buf);
+
+ if (!reader.beforeMessageRead())
+ return false;
+
+ switch (reader.state()) {
+ case 0:
+ cntrsMapBytes = reader.readByteArray("cntrsMapBytes");
+
+ if (!reader.isLastRead())
+ return false;
+
+ reader.incrementState();
+
+ case 1:
+ errBytes = reader.readByteArray("errBytes");
+
+ if (!reader.isLastRead())
+ return false;
+
+ reader.incrementState();
+
+ case 2:
+ flags = reader.readInt("flags");
+
+ if (!reader.isLastRead())
+ return false;
+
+ reader.incrementState();
+
+ case 3:
+ routineId = reader.readUuid("routineId");
+
+ if (!reader.isLastRead())
+ return false;
+
+ reader.incrementState();
+
+ }
+
+ return reader.afterMessageRead(ContinuousRoutineStartResultMessage.class);
+ }
+
+ /** {@inheritDoc} */
+ @Override public short directType() {
+ return 134;
+ }
+
+ /** {@inheritDoc} */
+ @Override public byte fieldsCount() {
+ return 4;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void onAckReceived() {
+ // No-op.
+ }
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return S.toString(ContinuousRoutineStartResultMessage.class, this);
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutinesCommonDiscoveryData.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutinesCommonDiscoveryData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutinesCommonDiscoveryData.java
new file mode 100644
index 0000000..d29de89
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutinesCommonDiscoveryData.java
@@ -0,0 +1,45 @@
+/*
+ * 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.processors.continuous;
+
+import java.io.Serializable;
+import java.util.List;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ *
+ */
+public class ContinuousRoutinesCommonDiscoveryData implements Serializable {
+ /** */
+ private static final long serialVersionUID = 0L;
+
+ /** */
+ final List<ContinuousRoutineInfo> startedRoutines;
+
+ /**
+ * @param startedRoutines Routines started in cluster.
+ */
+ ContinuousRoutinesCommonDiscoveryData(List<ContinuousRoutineInfo> startedRoutines) {
+ this.startedRoutines = startedRoutines;
+ }
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return S.toString(ContinuousRoutinesCommonDiscoveryData.class, this);
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutinesInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutinesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutinesInfo.java
new file mode 100644
index 0000000..ad24ff1
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutinesInfo.java
@@ -0,0 +1,132 @@
+/*
+ * 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.processors.continuous;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag;
+
+import static org.apache.ignite.internal.GridComponent.DiscoveryDataExchangeType.CONTINUOUS_PROC;
+
+/**
+ *
+ */
+class ContinuousRoutinesInfo {
+ /** */
+ private final Map<UUID, ContinuousRoutineInfo> startedRoutines = new HashMap<>();
+
+ /**
+ * @param dataBag Discovery data bag.
+ */
+ void collectGridNodeData(DiscoveryDataBag dataBag) {
+ synchronized (startedRoutines) {
+ if (!dataBag.commonDataCollectedFor(CONTINUOUS_PROC.ordinal()))
+ dataBag.addGridCommonData(CONTINUOUS_PROC.ordinal(),
+ new ContinuousRoutinesCommonDiscoveryData(new ArrayList<>(startedRoutines.values())));
+ }
+ }
+
+ /**
+ * @param dataBag Discovery data bag.
+ */
+ void collectJoiningNodeData(DiscoveryDataBag dataBag) {
+ synchronized (startedRoutines) {
+ for (ContinuousRoutineInfo info : startedRoutines.values()) {
+ if (info.disconnected)
+ info.sourceNodeId(dataBag.joiningNodeId());
+ }
+
+ dataBag.addJoiningNodeData(CONTINUOUS_PROC.ordinal(),
+ new ContinuousRoutinesJoiningNodeDiscoveryData(new ArrayList<>(startedRoutines.values())));
+ }
+ }
+
+ /**
+ * @param info Routine info.
+ */
+ void addRoutineInfo(ContinuousRoutineInfo info) {
+ synchronized (startedRoutines) {
+ startedRoutines.put(info.routineId, info);
+ }
+ }
+
+ /**
+ * @param routineId Routine ID.
+ * @return {@code True} if routine exists.
+ */
+ boolean routineExists(UUID routineId) {
+ synchronized (startedRoutines) {
+ return startedRoutines.containsKey(routineId);
+ }
+ }
+
+ /**
+ * @param routineId Routine ID.
+ */
+ void removeRoutine(UUID routineId) {
+ synchronized (startedRoutines) {
+ startedRoutines.remove(routineId);
+ }
+ }
+
+ /**
+ * @param locRoutines Routines IDs which can survive reconnect.
+ */
+ void onClientDisconnected(Collection<UUID> locRoutines) {
+ synchronized (startedRoutines) {
+ for (Iterator<Map.Entry<UUID, ContinuousRoutineInfo>> it = startedRoutines.entrySet().iterator(); it.hasNext();) {
+ Map.Entry<UUID, ContinuousRoutineInfo> e = it.next();
+
+ ContinuousRoutineInfo info = e.getValue();
+
+ if (!locRoutines.contains(info.routineId))
+ it.remove();
+ else
+ info.onDisconnected();
+ }
+ }
+ }
+
+ /**
+ * Removes all routines with autoUnsubscribe=false started by given node.
+ *
+ * @param nodeId Node ID.
+ */
+ void onNodeFail(UUID nodeId) {
+ synchronized (startedRoutines) {
+ for (Iterator<Map.Entry<UUID, ContinuousRoutineInfo>> it = startedRoutines.entrySet().iterator(); it.hasNext();) {
+ Map.Entry<UUID, ContinuousRoutineInfo> e = it.next();
+
+ ContinuousRoutineInfo info = e.getValue();
+
+ if (info.autoUnsubscribe && info.srcNodeId.equals(nodeId))
+ it.remove();
+ }
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return S.toString(ContinuousRoutinesInfo.class, this);
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutinesJoiningNodeDiscoveryData.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutinesJoiningNodeDiscoveryData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutinesJoiningNodeDiscoveryData.java
new file mode 100644
index 0000000..9be6ef8
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutinesJoiningNodeDiscoveryData.java
@@ -0,0 +1,45 @@
+/*
+ * 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.processors.continuous;
+
+import java.io.Serializable;
+import java.util.List;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ *
+ */
+public class ContinuousRoutinesJoiningNodeDiscoveryData implements Serializable {
+ /** */
+ private static final long serialVersionUID = 0L;
+
+ /** */
+ final List<ContinuousRoutineInfo> startedRoutines;
+
+ /**
+ * @param startedRoutines Routines registered on nodes, to be started in cluster.
+ */
+ ContinuousRoutinesJoiningNodeDiscoveryData(List<ContinuousRoutineInfo> startedRoutines) {
+ this.startedRoutines = startedRoutines;
+ }
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return S.toString(ContinuousRoutinesJoiningNodeDiscoveryData.class, this);
+ }
+}
[04/12] ignite git commit: IGNITE-7222 Added ZooKeeper discovery SPI
Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoveryImpl.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoveryImpl.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoveryImpl.java
new file mode 100644
index 0000000..7708358
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoveryImpl.java
@@ -0,0 +1,4464 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.io.ByteArrayInputStream;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.BitSet;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.zip.DataFormatException;
+import java.util.zip.Deflater;
+import java.util.zip.Inflater;
+import java.util.zip.InflaterInputStream;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteClientDisconnectedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteInterruptedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CommunicationFailureResolver;
+import org.apache.ignite.events.EventType;
+import org.apache.ignite.internal.ClusterMetricsSnapshot;
+import org.apache.ignite.internal.IgniteClientDisconnectedCheckedException;
+import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.IgnitionEx;
+import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.events.DiscoveryCustomEvent;
+import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpiInternalListener;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.security.SecurityContext;
+import org.apache.ignite.internal.util.GridLongList;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.io.GridByteArrayOutputStream;
+import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.LT;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.lang.IgniteRunnable;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.marshaller.MarshallerUtils;
+import org.apache.ignite.marshaller.jdk.JdkMarshaller;
+import org.apache.ignite.plugin.security.SecurityCredentials;
+import org.apache.ignite.spi.IgniteNodeValidationResult;
+import org.apache.ignite.spi.IgniteSpiException;
+import org.apache.ignite.spi.IgniteSpiTimeoutObject;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag;
+import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
+import org.apache.ignite.spi.discovery.DiscoverySpiDataExchange;
+import org.apache.ignite.spi.discovery.DiscoverySpiListener;
+import org.apache.ignite.spi.discovery.DiscoverySpiNodeAuthenticator;
+import org.apache.ignite.spi.discovery.zk.ZookeeperDiscoverySpi;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+import org.apache.zookeeper.AsyncCallback;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.data.Stat;
+import org.jboss.netty.util.internal.ConcurrentHashMap;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.events.EventType.EVT_CLIENT_NODE_DISCONNECTED;
+import static org.apache.ignite.events.EventType.EVT_CLIENT_NODE_RECONNECTED;
+import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
+import static org.apache.ignite.events.EventType.EVT_NODE_JOINED;
+import static org.apache.ignite.events.EventType.EVT_NODE_SEGMENTED;
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_IGNITE_INSTANCE_NAME;
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS;
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_SECURITY_SUBJECT_V2;
+import static org.apache.zookeeper.CreateMode.EPHEMERAL_SEQUENTIAL;
+import static org.apache.zookeeper.CreateMode.PERSISTENT;
+
+/**
+ * Zookeeper Discovery Impl.
+ */
+public class ZookeeperDiscoveryImpl {
+ /** */
+ static final String IGNITE_ZOOKEEPER_DISCOVERY_SPI_ACK_THRESHOLD = "IGNITE_ZOOKEEPER_DISCOVERY_SPI_ACK_THRESHOLD";
+
+ /** */
+ static final String IGNITE_ZOOKEEPER_DISCOVERY_SPI_ACK_TIMEOUT = "IGNITE_ZOOKEEPER_DISCOVERY_SPI_ACK_TIMEOUT";
+
+ /** */
+ static final String IGNITE_ZOOKEEPER_DISCOVERY_SPI_MAX_EVTS = "IGNITE_ZOOKEEPER_DISCOVERY_SPI_MAX_EVTS";
+
+ /** */
+ private static final String IGNITE_ZOOKEEPER_DISCOVERY_SPI_EVTS_THROTTLE = "IGNITE_ZOOKEEPER_DISCOVERY_SPI_EVTS_THROTTLE";
+
+ /** */
+ final ZookeeperDiscoverySpi spi;
+
+ /** */
+ private final String igniteInstanceName;
+
+ /** */
+ private final String connectString;
+
+ /** */
+ private final int sesTimeout;
+
+ /** */
+ private final JdkMarshaller marsh = new JdkMarshaller();
+
+ /** */
+ private final ZkIgnitePaths zkPaths;
+
+ /** */
+ private final IgniteLogger log;
+
+ /** */
+ final GridSpinBusyLock busyLock = new GridSpinBusyLock();
+
+ /** */
+ private final ZookeeperClusterNode locNode;
+
+ /** */
+ private final DiscoverySpiListener lsnr;
+
+ /** */
+ private final DiscoverySpiDataExchange exchange;
+
+ /** */
+ private final boolean clientReconnectEnabled;
+
+ /** */
+ private final GridFutureAdapter<Void> joinFut = new GridFutureAdapter<>();
+
+ /** */
+ private final int evtsAckThreshold;
+
+ /** */
+ private IgniteThreadPoolExecutor utilityPool;
+
+ /** */
+ private ZkRuntimeState rtState;
+
+ /** */
+ private volatile ConnectionState connState = ConnectionState.STARTED;
+
+ /** */
+ private final AtomicBoolean stop = new AtomicBoolean();
+
+ /** */
+ private final Object stateMux = new Object();
+
+ /** */
+ public volatile IgniteDiscoverySpiInternalListener internalLsnr;
+
+ /** */
+ private final ConcurrentHashMap<Long, PingFuture> pingFuts = new ConcurrentHashMap<>();
+
+ /** */
+ private final AtomicReference<ZkCommunicationErrorProcessFuture> commErrProcFut = new AtomicReference<>();
+
+ /** */
+ private long prevSavedEvtsTopVer;
+
+ /**
+ * @param spi Discovery SPI.
+ * @param igniteInstanceName Instance name.
+ * @param log Logger.
+ * @param zkRootPath Zookeeper base path node all nodes.
+ * @param locNode Local node instance.
+ * @param lsnr Discovery events listener.
+ * @param exchange Discovery data exchange.
+ * @param internalLsnr Internal listener (used for testing only).
+ */
+ public ZookeeperDiscoveryImpl(
+ ZookeeperDiscoverySpi spi,
+ String igniteInstanceName,
+ IgniteLogger log,
+ String zkRootPath,
+ ZookeeperClusterNode locNode,
+ DiscoverySpiListener lsnr,
+ DiscoverySpiDataExchange exchange,
+ IgniteDiscoverySpiInternalListener internalLsnr) {
+ assert locNode.id() != null && locNode.isLocal() : locNode;
+
+ MarshallerUtils.setNodeName(marsh, igniteInstanceName);
+
+ zkPaths = new ZkIgnitePaths(zkRootPath);
+
+ this.spi = spi;
+ this.igniteInstanceName = igniteInstanceName;
+ this.connectString = spi.getZkConnectionString();
+ this.sesTimeout = (int)spi.getSessionTimeout();
+ this.log = log.getLogger(getClass());
+ this.locNode = locNode;
+ this.lsnr = lsnr;
+ this.exchange = exchange;
+ this.clientReconnectEnabled = locNode.isClient() && !spi.isClientReconnectDisabled();
+
+ int evtsAckThreshold = IgniteSystemProperties.getInteger(IGNITE_ZOOKEEPER_DISCOVERY_SPI_ACK_THRESHOLD, 5);
+
+ if (evtsAckThreshold <= 0)
+ evtsAckThreshold = 1;
+
+ this.evtsAckThreshold = evtsAckThreshold;
+
+ if (internalLsnr != null)
+ this.internalLsnr = internalLsnr;
+ }
+
+ /**
+ * @return Exception.
+ */
+ private static IgniteClientDisconnectedCheckedException disconnectError() {
+ return new IgniteClientDisconnectedCheckedException(null, "Client node disconnected.");
+ }
+
+ /**
+ * @return Logger.
+ */
+ IgniteLogger log() {
+ return log;
+ }
+
+ /**
+ * @return Local node instance.
+ */
+ public ClusterNode localNode() {
+ return locNode;
+ }
+
+ /**
+ * @param nodeId Node ID.
+ * @return Node instance.
+ */
+ @Nullable public ZookeeperClusterNode node(UUID nodeId) {
+ assert nodeId != null;
+
+ return rtState.top.nodesById.get(nodeId);
+ }
+
+ /**
+ * @param nodeOrder Node order.
+ * @return Node instance.
+ */
+ @Nullable public ZookeeperClusterNode node(long nodeOrder) {
+ assert nodeOrder > 0 : nodeOrder;
+
+ return rtState.top.nodesByOrder.get(nodeOrder);
+ }
+
+ /**
+ * @param fut Future to remove.
+ */
+ void clearCommunicationErrorProcessFuture(ZkCommunicationErrorProcessFuture fut) {
+ assert fut.isDone() : fut;
+
+ commErrProcFut.compareAndSet(fut, null);
+ }
+
+ /**
+ * @param node0 Problem node ID
+ * @param err Connect error.
+ */
+ public void resolveCommunicationError(ClusterNode node0, Exception err) {
+ ZookeeperClusterNode node = node(node0.id());
+
+ if (node == null)
+ throw new IgniteSpiException(new ClusterTopologyCheckedException("Node failed: " + node0.id()));
+
+ IgniteInternalFuture<Boolean> nodeStatusFut;
+
+ for (;;) {
+ checkState();
+
+ ZkCommunicationErrorProcessFuture fut = commErrProcFut.get();
+
+ if (fut == null || fut.isDone()) {
+ ZkCommunicationErrorProcessFuture newFut = ZkCommunicationErrorProcessFuture.createOnCommunicationError(
+ this,
+ node.sessionTimeout() + 1000);
+
+ if (commErrProcFut.compareAndSet(fut, newFut)) {
+ fut = newFut;
+
+ if (log.isInfoEnabled()) {
+ log.info("Created new communication error process future [errNode=" + node0.id() +
+ ", err=" + err + ']');
+ }
+
+ try {
+ checkState();
+ }
+ catch (Exception e) {
+ fut.onError(e);
+
+ throw e;
+ }
+
+ fut.scheduleCheckOnTimeout();
+ }
+ else {
+ fut = commErrProcFut.get();
+
+ if (fut == null)
+ continue;
+ }
+ }
+
+ nodeStatusFut = fut.nodeStatusFuture(node);
+
+ if (nodeStatusFut != null)
+ break;
+ else {
+ try {
+ fut.get();
+ }
+ catch (IgniteCheckedException e) {
+ U.warn(log, "Previous communication error process future failed: " + e);
+ }
+ }
+ }
+
+ try {
+ if (!nodeStatusFut.get())
+ throw new IgniteSpiException(new ClusterTopologyCheckedException("Node failed: " + node0.id()));
+ }
+ catch (IgniteCheckedException e) {
+ throw new IgniteSpiException(e);
+ }
+ }
+
+ /**
+ * @param nodeId Node ID.
+ * @return Ping result.
+ */
+ public boolean pingNode(UUID nodeId) {
+ checkState();
+
+ ZkRuntimeState rtState = this.rtState;
+
+ ZookeeperClusterNode node = rtState.top.nodesById.get(nodeId);
+
+ if (node == null)
+ return false;
+
+ if (node.isLocal())
+ return true;
+
+ PingFuture fut = pingFuts.get(node.order());
+
+ if (fut == null) {
+ fut = new PingFuture(rtState, node);
+
+ PingFuture old = pingFuts.putIfAbsent(node.order(), fut);
+
+ if (old == null) {
+ if (fut.checkNodeAndState())
+ spi.getSpiContext().addTimeoutObject(fut);
+ else
+ assert fut.isDone();
+ }
+ else
+ fut = old;
+ }
+
+ try {
+ return fut.get();
+ }
+ catch (IgniteCheckedException e) {
+ throw new IgniteSpiException(e);
+ }
+ }
+
+ /**
+ * @param nodeId Node ID.
+ * @param warning Warning.
+ */
+ public void failNode(UUID nodeId, @Nullable String warning) {
+ ZookeeperClusterNode node = rtState.top.nodesById.get(nodeId);
+
+ if (node == null) {
+ if (log.isDebugEnabled())
+ log.debug("Ignore forcible node fail request, node does not exist: " + nodeId);
+
+ return;
+ }
+
+ if (!node.isClient()) {
+ U.warn(log, "Ignore forcible node fail request for non-client node: " + node);
+
+ return;
+ }
+
+ sendCustomMessage(new ZkForceNodeFailMessage(node.internalId(), warning));
+ }
+
+ /**
+ *
+ */
+ public void reconnect() {
+ assert clientReconnectEnabled;
+
+ synchronized (stateMux) {
+ if (connState == ConnectionState.STARTED) {
+ connState = ConnectionState.DISCONNECTED;
+
+ rtState.onCloseStart(disconnectError());
+ }
+ else
+ return;
+ }
+
+ busyLock.block();
+
+ busyLock.unblock();
+
+ rtState.zkClient.close();
+
+ UUID newId = UUID.randomUUID();
+
+ U.quietAndWarn(log, "Local node will try to reconnect to cluster with new id due to network problems [" +
+ "newId=" + newId +
+ ", prevId=" + locNode.id() +
+ ", locNode=" + locNode + ']');
+
+ runInWorkerThread(new ReconnectClosure(newId));
+ }
+
+ /**
+ * @param newId New ID.
+ */
+ private void doReconnect(UUID newId) {
+ if (rtState.joined) {
+ assert rtState.evtsData != null;
+
+ lsnr.onDiscovery(EVT_CLIENT_NODE_DISCONNECTED,
+ rtState.evtsData.topVer,
+ locNode,
+ rtState.top.topologySnapshot(),
+ Collections.<Long, Collection<ClusterNode>>emptyMap(),
+ null);
+ }
+
+ try {
+ locNode.onClientDisconnected(newId);
+
+ joinTopology(rtState);
+ }
+ catch (Exception e) {
+ if (stopping()) {
+ if (log.isDebugEnabled())
+ log.debug("Reconnect failed, node is stopping [err=" + e + ']');
+
+ return;
+ }
+
+ U.error(log, "Failed to reconnect: " + e, e);
+
+ onSegmented(e);
+ }
+ }
+
+ /**
+ * @return {@code True} if started to stop.
+ */
+ private boolean stopping() {
+ if (stop.get())
+ return true;
+
+ synchronized (stateMux) {
+ if (connState == ConnectionState.STOPPED)
+ return true;
+ }
+
+ return false;
+ }
+
+ /**
+ * @param e Error.
+ */
+ private void onSegmented(Exception e) {
+ rtState.errForClose = e;
+
+ if (rtState.joined || joinFut.isDone()) {
+ synchronized (stateMux) {
+ connState = ConnectionState.STOPPED;
+ }
+
+ notifySegmented();
+ }
+ else
+ joinFut.onDone(e);
+ }
+
+ /**
+ *
+ */
+ private void notifySegmented() {
+ List<ClusterNode> nodes = rtState.top.topologySnapshot();
+
+ if (nodes.isEmpty())
+ nodes = Collections.singletonList((ClusterNode)locNode);
+
+ lsnr.onDiscovery(EVT_NODE_SEGMENTED,
+ rtState.evtsData != null ? rtState.evtsData.topVer : 1L,
+ locNode,
+ nodes,
+ Collections.<Long, Collection<ClusterNode>>emptyMap(),
+ null);
+ }
+
+ /**
+ * @return Remote nodes.
+ */
+ public Collection<ClusterNode> remoteNodes() {
+ checkState();
+
+ return rtState.top.remoteNodes();
+ }
+
+ /**
+ *
+ */
+ private void checkState() {
+ switch (connState) {
+ case STARTED:
+ break;
+
+ case STOPPED:
+ throw new IgniteSpiException("Node stopped.");
+
+ case DISCONNECTED:
+ throw new IgniteClientDisconnectedException(null, "Client is disconnected.");
+ }
+ }
+
+ /**
+ * @param nodeId Node ID.
+ * @return {@code True} if node joined or joining topology.
+ */
+ public boolean knownNode(UUID nodeId) {
+ while (!busyLock.enterBusy())
+ checkState();
+
+ try {
+ List<String> children = rtState.zkClient.getChildren(zkPaths.aliveNodesDir);
+
+ for (int i = 0; i < children.size(); i++) {
+ UUID id = ZkIgnitePaths.aliveNodeId(children.get(i));
+
+ if (nodeId.equals(id))
+ return true;
+ }
+
+ return false;
+ }
+ catch (ZookeeperClientFailedException e) {
+ if (clientReconnectEnabled)
+ throw new IgniteClientDisconnectedException(null, "Client is disconnected.");
+
+ throw new IgniteException(e);
+ }
+ catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+
+ throw new IgniteInterruptedException(e);
+ }
+ finally {
+ busyLock.leaveBusy();
+ }
+ }
+
+ /**
+ * @param msg Message.
+ */
+ public void sendCustomMessage(DiscoverySpiCustomMessage msg) {
+ assert msg != null;
+
+ byte[] msgBytes;
+
+ try {
+ msgBytes = marshalZip(msg);
+ }
+ catch (IgniteCheckedException e) {
+ throw new IgniteSpiException("Failed to marshal custom message: " + msg, e);
+ }
+
+ while (!busyLock.enterBusy())
+ checkState();
+
+ try {
+ ZookeeperClient zkClient = rtState.zkClient;
+
+ saveCustomMessage(zkClient, msgBytes);
+ }
+ catch (ZookeeperClientFailedException e) {
+ if (clientReconnectEnabled)
+ throw new IgniteClientDisconnectedException(null, "Client is disconnected.");
+
+ throw new IgniteException(e);
+ }
+ catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+
+ throw new IgniteInterruptedException(e);
+ }
+ finally {
+ busyLock.leaveBusy();
+ }
+ }
+
+ /**
+ * @param zkClient Client.
+ * @param msgBytes Marshalled message.
+ * @throws ZookeeperClientFailedException If connection to zk was lost.
+ * @throws InterruptedException If interrupted.
+ */
+ private void saveCustomMessage(ZookeeperClient zkClient, byte[] msgBytes)
+ throws ZookeeperClientFailedException, InterruptedException
+ {
+ String prefix = UUID.randomUUID().toString();
+
+ int partCnt = 1;
+
+ int overhead = 10;
+
+ UUID locId = locNode.id();
+
+ String path = zkPaths.createCustomEventPath(prefix, locId, partCnt);
+
+ if (zkClient.needSplitNodeData(path, msgBytes, overhead)) {
+ List<byte[]> parts = zkClient.splitNodeData(path, msgBytes, overhead);
+
+ String partsBasePath = zkPaths.customEventPartsBasePath(prefix, locId);
+
+ saveMultipleParts(zkClient, partsBasePath, parts);
+
+ msgBytes = null;
+
+ partCnt = parts.size();
+ }
+
+ zkClient.createSequential(prefix,
+ zkPaths.customEvtsDir,
+ zkPaths.createCustomEventPath(prefix, locId, partCnt),
+ msgBytes,
+ CreateMode.PERSISTENT_SEQUENTIAL);
+ }
+
+ /**
+ * @return Cluster start time.
+ */
+ public long gridStartTime() {
+ return rtState.gridStartTime;
+ }
+
+ /**
+ * Starts join procedure and waits for {@link EventType#EVT_NODE_JOINED} event for local node.
+ *
+ * @throws InterruptedException If interrupted.
+ */
+ public void startJoinAndWait() throws InterruptedException {
+ joinTopology(null);
+
+ for (;;) {
+ try {
+ joinFut.get(10_000);
+
+ break;
+ }
+ catch (IgniteFutureTimeoutCheckedException e) {
+ U.warn(log, "Waiting for local join event [nodeId=" + locNode.id() + ", name=" + igniteInstanceName + ']');
+ }
+ catch (Exception e) {
+ IgniteSpiException spiErr = X.cause(e, IgniteSpiException.class);
+
+ if (spiErr != null)
+ throw spiErr;
+
+ throw new IgniteSpiException("Failed to join cluster", e);
+ }
+ }
+ }
+
+ /**
+ * @param prevState Previous state in case of connect retry.
+ * @throws InterruptedException If interrupted.
+ */
+ private void joinTopology(@Nullable ZkRuntimeState prevState) throws InterruptedException {
+ if (!busyLock.enterBusy())
+ return;
+
+ try {
+ boolean reconnect = prevState != null;
+
+ // Need fire EVT_CLIENT_NODE_RECONNECTED event if reconnect after already joined.
+ boolean prevJoined = prevState != null && prevState.joined;
+
+ IgniteDiscoverySpiInternalListener internalLsnr = this.internalLsnr;
+
+ if (internalLsnr != null)
+ internalLsnr.beforeJoin(locNode, log);
+
+ if (locNode.isClient() && reconnect)
+ locNode.setAttributes(spi.getSpiContext().nodeAttributes());
+
+ marshalCredentialsOnJoin(locNode);
+
+ synchronized (stateMux) {
+ if (connState == ConnectionState.STOPPED)
+ return;
+
+ connState = ConnectionState.STARTED;
+ }
+
+ ZkRuntimeState rtState = this.rtState = new ZkRuntimeState(prevJoined);
+
+ DiscoveryDataBag discoDataBag = new DiscoveryDataBag(locNode.id(), locNode.isClient());
+
+ exchange.collect(discoDataBag);
+
+ ZkJoiningNodeData joinData = new ZkJoiningNodeData(locNode, discoDataBag.joiningNodeData());
+
+ byte[] joinDataBytes;
+
+ try {
+ joinDataBytes = marshalZip(joinData);
+ }
+ catch (Exception e) {
+ throw new IgniteSpiException("Failed to marshal joining node data", e);
+ }
+
+ try {
+ rtState.zkClient = new ZookeeperClient(
+ igniteInstanceName,
+ log,
+ connectString,
+ sesTimeout,
+ new ConnectionLossListener());
+ }
+ catch (Exception e) {
+ throw new IgniteSpiException("Failed to create Zookeeper client", e);
+ }
+
+ startJoin(rtState, prevState, joinDataBytes);
+ }
+ finally {
+ busyLock.leaveBusy();
+ }
+ }
+
+ /**
+ * @throws InterruptedException If interrupted.
+ */
+ private void initZkNodes() throws InterruptedException {
+ try {
+ ZookeeperClient client = rtState.zkClient;
+
+ if (!client.exists(zkPaths.clusterDir)) {
+ createRootPathParents(zkPaths.clusterDir, client);
+
+ client.createIfNeeded(zkPaths.clusterDir, null, PERSISTENT);
+ }
+
+ List<String> createdDirs = client.getChildren(zkPaths.clusterDir);
+
+ String[] requiredDirs = {
+ zkPaths.evtsPath,
+ zkPaths.joinDataDir,
+ zkPaths.customEvtsDir,
+ zkPaths.customEvtsPartsDir,
+ zkPaths.customEvtsAcksDir,
+ zkPaths.aliveNodesDir};
+
+ List<String> dirs = new ArrayList<>();
+
+ for (String dir : requiredDirs) {
+ String dir0 = dir.substring(zkPaths.clusterDir.length() + 1);
+
+ if (!createdDirs.contains(dir0))
+ dirs.add(dir);
+ }
+
+ try {
+ if (!dirs.isEmpty())
+ client.createAll(dirs, PERSISTENT);
+ }
+ catch (KeeperException.NodeExistsException e) {
+ if (log.isDebugEnabled())
+ log.debug("Failed to create nodes using bulk operation: " + e);
+
+ for (String dir : dirs)
+ client.createIfNeeded(dir, null, PERSISTENT);
+ }
+ }
+ catch (ZookeeperClientFailedException e) {
+ throw new IgniteSpiException("Failed to initialize Zookeeper nodes", e);
+ }
+ }
+
+ /**
+ * @param rootDir Root directory.
+ * @param client Client.
+ * @throws ZookeeperClientFailedException If connection to zk was lost.
+ * @throws InterruptedException If interrupted.
+ */
+ private void createRootPathParents(String rootDir, ZookeeperClient client)
+ throws ZookeeperClientFailedException, InterruptedException {
+ int startIdx = 0;
+
+ for (;;) {
+ int separatorIdx = rootDir.indexOf(ZkIgnitePaths.PATH_SEPARATOR, startIdx);
+
+ if (separatorIdx == -1)
+ break;
+
+ if (separatorIdx > 0) {
+ String path = rootDir.substring(0, separatorIdx);
+
+ client.createIfNeeded(path, null, CreateMode.PERSISTENT);
+ }
+
+ startIdx = separatorIdx + 1;
+ }
+ }
+
+ /**
+ * @param zkClient Client.
+ * @param basePath Base path.
+ * @param partCnt Parts count.
+ */
+ private void deleteMultiplePartsAsync(ZookeeperClient zkClient, String basePath, int partCnt) {
+ for (int i = 0; i < partCnt; i++) {
+ String path = multipartPathName(basePath, i);
+
+ zkClient.deleteIfExistsAsync(path);
+ }
+ }
+
+ /**
+ * @param zkClient Client.
+ * @param basePath Base path.
+ * @param partCnt Parts count.
+ * @return Read parts.
+ * @throws Exception If failed.
+ */
+ private byte[] readMultipleParts(ZookeeperClient zkClient, String basePath, int partCnt)
+ throws Exception {
+ assert partCnt >= 1;
+
+ if (partCnt > 1) {
+ List<byte[]> parts = new ArrayList<>(partCnt);
+
+ int totSize = 0;
+
+ for (int i = 0; i < partCnt; i++) {
+ byte[] part = zkClient.getData(multipartPathName(basePath, i));
+
+ parts.add(part);
+
+ totSize += part.length;
+ }
+
+ byte[] res = new byte[totSize];
+
+ int pos = 0;
+
+ for (int i = 0; i < partCnt; i++) {
+ byte[] part = parts.get(i);
+
+ System.arraycopy(part, 0, res, pos, part.length);
+
+ pos += part.length;
+ }
+
+ return res;
+ }
+ else
+ return zkClient.getData(multipartPathName(basePath, 0));
+ }
+
+ /**
+ * @param zkClient Client.
+ * @param basePath Base path.
+ * @param parts Data parts.
+ * @return Number of parts.
+ * @throws ZookeeperClientFailedException If client failed.
+ * @throws InterruptedException If interrupted.
+ */
+ private int saveMultipleParts(ZookeeperClient zkClient, String basePath, List<byte[]> parts)
+ throws ZookeeperClientFailedException, InterruptedException
+ {
+ assert parts.size() > 1;
+
+ for (int i = 0; i < parts.size(); i++) {
+ byte[] part = parts.get(i);
+
+ String path = multipartPathName(basePath, i);
+
+ zkClient.createIfNeeded(path, part, PERSISTENT);
+ }
+
+ return parts.size();
+ }
+
+ /**
+ * @param basePath Base path.
+ * @param part Part number.
+ * @return Path.
+ */
+ private static String multipartPathName(String basePath, int part) {
+ return basePath + String.format("%04d", part);
+ }
+
+ /**
+ * @param rtState Runtime state.
+ * @param joinDataBytes Joining node data.
+ * @param prevState Previous state in case of connect retry.
+ * @throws InterruptedException If interrupted.
+ */
+ private void startJoin(ZkRuntimeState rtState, @Nullable ZkRuntimeState prevState, final byte[] joinDataBytes)
+ throws InterruptedException
+ {
+ try {
+ long startTime = System.currentTimeMillis();
+
+ initZkNodes();
+
+ String prefix = UUID.randomUUID().toString();
+
+ rtState.init(new ZkWatcher(rtState), new AliveNodeDataWatcher(rtState));
+
+ ZookeeperClient zkClient = rtState.zkClient;
+
+ final int OVERHEAD = 5;
+
+ // TODO ZK: https://issues.apache.org/jira/browse/IGNITE-8193
+ String joinDataPath = zkPaths.joinDataDir + "/" + prefix + ":" + locNode.id();
+
+ if (zkClient.needSplitNodeData(joinDataPath, joinDataBytes, OVERHEAD)) {
+ List<byte[]> parts = zkClient.splitNodeData(joinDataPath, joinDataBytes, OVERHEAD);
+
+ rtState.joinDataPartCnt = parts.size();
+
+ saveMultipleParts(zkClient, joinDataPath + ":", parts);
+
+ joinDataPath = zkClient.createIfNeeded(
+ joinDataPath,
+ marshalZip(new ZkJoiningNodeData(parts.size())),
+ PERSISTENT);
+ }
+ else {
+ joinDataPath = zkClient.createIfNeeded(
+ joinDataPath,
+ joinDataBytes,
+ PERSISTENT);
+ }
+
+ rtState.locNodeZkPath = zkClient.createSequential(
+ prefix,
+ zkPaths.aliveNodesDir,
+ zkPaths.aliveNodePathForCreate(prefix, locNode),
+ null,
+ EPHEMERAL_SEQUENTIAL);
+
+ rtState.internalOrder = ZkIgnitePaths.aliveInternalId(rtState.locNodeZkPath);
+
+ if (log.isInfoEnabled()) {
+ log.info("Node started join [nodeId=" + locNode.id() +
+ ", instanceName=" + locNode.attribute(ATTR_IGNITE_INSTANCE_NAME) +
+ ", zkSessionId=0x" + Long.toHexString(rtState.zkClient.zk().getSessionId()) +
+ ", joinDataSize=" + joinDataBytes.length +
+ (rtState.joinDataPartCnt > 1 ? (", joinDataPartCnt=" + rtState.joinDataPartCnt) : "") +
+ ", consistentId=" + locNode.consistentId() +
+ ", initTime=" + (System.currentTimeMillis() - startTime) +
+ ", nodePath=" + rtState.locNodeZkPath + ']');
+ }
+
+ /*
+ If node can not join due to validation error this error is reported in join data,
+ As a minor optimization do not start watch join data immediately, but only if do not receive
+ join event after some timeout.
+ */
+ CheckJoinErrorWatcher joinErrorWatcher = new CheckJoinErrorWatcher(5000, joinDataPath, rtState);
+
+ rtState.joinErrTo = joinErrorWatcher.timeoutObj;
+
+ if (locNode.isClient() && spi.getJoinTimeout() > 0) {
+ ZkTimeoutObject joinTimeoutObj = prevState != null ? prevState.joinTo : null;
+
+ if (joinTimeoutObj == null) {
+ joinTimeoutObj = new JoinTimeoutObject(spi.getJoinTimeout());
+
+ spi.getSpiContext().addTimeoutObject(joinTimeoutObj);
+ }
+
+ rtState.joinTo = joinTimeoutObj;
+ }
+
+ if (!locNode.isClient())
+ zkClient.getChildrenAsync(zkPaths.aliveNodesDir, null, new CheckCoordinatorCallback(rtState));
+
+ zkClient.getDataAsync(zkPaths.evtsPath, rtState.watcher, rtState.watcher);
+
+ spi.getSpiContext().addTimeoutObject(rtState.joinErrTo);
+ }
+ catch (IgniteCheckedException | ZookeeperClientFailedException e) {
+ throw new IgniteSpiException("Failed to initialize Zookeeper nodes", e);
+ }
+ }
+
+ /**
+ * Authenticate local node.
+ *
+ * @param nodeAuth Authenticator.
+ * @param locCred Local security credentials for authentication.
+ * @throws IgniteSpiException If any error occurs.
+ */
+ private void localAuthentication(DiscoverySpiNodeAuthenticator nodeAuth, SecurityCredentials locCred){
+ assert nodeAuth != null;
+ assert locCred != null;
+
+ try {
+ SecurityContext subj = nodeAuth.authenticateNode(locNode, locCred);
+
+ // Note: exception message is checked in tests.
+ if (subj == null)
+ throw new IgniteSpiException("Authentication failed for local node.");
+
+ if (!(subj instanceof Serializable))
+ throw new IgniteSpiException("Authentication subject is not Serializable.");
+
+ Map<String, Object> attrs = new HashMap<>(locNode.attributes());
+
+ attrs.put(ATTR_SECURITY_SUBJECT_V2, U.marshal(marsh, subj));
+
+ locNode.setAttributes(attrs);
+ }
+ catch (Exception e) {
+ throw new IgniteSpiException("Failed to authenticate local node (will shutdown local node).", e);
+ }
+ }
+
+ /**
+ * @param node Node.
+ * @param zipBytes Zip-compressed marshalled security subject.
+ * @throws Exception If failed.
+ */
+ private void setNodeSecuritySubject(ZookeeperClusterNode node, byte[] zipBytes) throws Exception {
+ assert zipBytes != null;
+
+ Map<String, Object> attrs = new HashMap<>(node.getAttributes());
+
+ attrs.put(ATTR_SECURITY_SUBJECT_V2, unzip(zipBytes));
+
+ node.setAttributes(attrs);
+ }
+
+ /**
+ * @param node Node.
+ * @return Credentials.
+ * @throws IgniteCheckedException If failed to unmarshal.
+ */
+ private SecurityCredentials unmarshalCredentials(ZookeeperClusterNode node) throws Exception {
+ byte[] credBytes = (byte[])node.getAttributes().get(ATTR_SECURITY_CREDENTIALS);
+
+ if (credBytes == null)
+ return null;
+
+ return unmarshalZip(credBytes);
+ }
+
+ /**
+ * Marshalls credentials with discovery SPI marshaller (will replace attribute value).
+ *
+ * @param node Node to marshall credentials for.
+ * @throws IgniteSpiException If marshalling failed.
+ */
+ private void marshalCredentialsOnJoin(ZookeeperClusterNode node) throws IgniteSpiException {
+ try {
+ // Use security-unsafe getter.
+ Map<String, Object> attrs0 = node.getAttributes();
+
+ Object creds = attrs0.get(ATTR_SECURITY_CREDENTIALS);
+
+ if (creds != null) {
+ Map<String, Object> attrs = new HashMap<>(attrs0);
+
+ assert !(creds instanceof byte[]);
+
+ attrs.put(ATTR_SECURITY_CREDENTIALS, marshalZip(creds));
+
+ node.setAttributes(attrs);
+ }
+ }
+ catch (IgniteCheckedException e) {
+ throw new IgniteSpiException("Failed to marshal node security credentials: " + node.id(), e);
+ }
+ }
+
+ /**
+ *
+ */
+ private class UpdateProcessedEventsTimeoutObject extends ZkTimeoutObject {
+ /** */
+ private final ZkRuntimeState rtState;
+
+ /**
+ * @param rtState Runtime state.
+ * @param timeout Timeout.
+ */
+ UpdateProcessedEventsTimeoutObject(ZkRuntimeState rtState, long timeout) {
+ super(timeout);
+
+ this.rtState = rtState;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void onTimeout() {
+ runInWorkerThread(new ZkRunnable(rtState, ZookeeperDiscoveryImpl.this) {
+ @Override protected void run0() throws Exception {
+ updateProcessedEventsOnTimeout(rtState, UpdateProcessedEventsTimeoutObject.this);
+ }
+ });
+ }
+ }
+
+ /**
+ *
+ */
+ private class JoinTimeoutObject extends ZkTimeoutObject {
+ /**
+ * @param timeout Timeout.
+ */
+ JoinTimeoutObject(long timeout) {
+ super(timeout);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void onTimeout() {
+ if (cancelled || rtState.joined)
+ return;
+
+ runInWorkerThread(new Runnable() {
+ @Override public void run() {
+ synchronized (stateMux) {
+ if (cancelled || rtState.joined)
+ return;
+
+ if (connState == ConnectionState.STOPPED)
+ return;
+
+ connState = ConnectionState.STOPPED;
+ }
+
+ U.warn(log, "Failed to connect to cluster, either connection to ZooKeeper can not be established or there " +
+ "are no alive server nodes (consider increasing 'joinTimeout' configuration property) [" +
+ "joinTimeout=" + spi.getJoinTimeout() + ']');
+
+ // Note: exception message is checked in tests.
+ onSegmented(new IgniteSpiException("Failed to connect to cluster within configured timeout"));
+ }
+ });
+ }
+ }
+
+ /**
+ *
+ */
+ private class CheckJoinErrorWatcher extends ZkAbstractWatcher implements AsyncCallback.DataCallback {
+ /** */
+ private final String joinDataPath;
+
+ /** */
+ private ZkTimeoutObject timeoutObj;
+
+ /**
+ * @param timeout Timeout.
+ * @param joinDataPath0 Node joined data path.
+ * @param rtState0 State.
+ */
+ CheckJoinErrorWatcher(long timeout, String joinDataPath0, ZkRuntimeState rtState0) {
+ super(rtState0, ZookeeperDiscoveryImpl.this);
+
+ this.joinDataPath = joinDataPath0;
+
+ timeoutObj = new ZkTimeoutObject(timeout) {
+ @Override public void onTimeout() {
+ if (rtState.errForClose != null || rtState.joined)
+ return;
+
+ synchronized (stateMux) {
+ if (connState != ConnectionState.STARTED)
+ return;
+ }
+
+ rtState.zkClient.getDataAsync(joinDataPath,
+ CheckJoinErrorWatcher.this,
+ CheckJoinErrorWatcher.this);
+ }
+ };
+ }
+
+ /** {@inheritDoc} */
+ @Override public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat) {
+ if (rc != 0)
+ return;
+
+ if (!onProcessStart())
+ return;
+
+ try {
+ Object obj = unmarshalZip(data);
+
+ if (obj instanceof ZkInternalJoinErrorMessage) {
+ ZkInternalJoinErrorMessage joinErr = (ZkInternalJoinErrorMessage)obj;
+
+ onSegmented(new IgniteSpiException(joinErr.err));
+ }
+
+ onProcessEnd();
+ }
+ catch (Throwable e) {
+ onProcessError(e);
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override public void process0(WatchedEvent evt) {
+ if (rtState.errForClose != null || rtState.joined)
+ return;
+
+ if (evt.getType() == Event.EventType.NodeDataChanged)
+ rtState.zkClient.getDataAsync(evt.getPath(), this, this);
+ }
+ }
+
+ /**
+ * @param aliveNodes Alive nodes.
+ * @throws Exception If failed.
+ */
+ private void checkIsCoordinator(final List<String> aliveNodes) throws Exception {
+ assert !locNode.isClient();
+
+ TreeMap<Long, String> aliveSrvs = new TreeMap<>();
+
+ long locInternalOrder = rtState.internalOrder;
+
+ for (String aliveNodePath : aliveNodes) {
+ if (ZkIgnitePaths.aliveNodeClientFlag(aliveNodePath))
+ continue;
+
+ Long internalId = ZkIgnitePaths.aliveInternalId(aliveNodePath);
+
+ aliveSrvs.put(internalId, aliveNodePath);
+ }
+
+ assert !aliveSrvs.isEmpty();
+
+ Map.Entry<Long, String> crdE = aliveSrvs.firstEntry();
+
+ if (locInternalOrder == crdE.getKey())
+ onBecomeCoordinator(aliveNodes);
+ else {
+ assert aliveSrvs.size() > 1 : aliveSrvs;
+
+ Map.Entry<Long, String> prevE = aliveSrvs.floorEntry(locInternalOrder - 1);
+
+ assert prevE != null;
+
+ if (log.isInfoEnabled()) {
+ log.info("Discovery coordinator already exists, watch for previous server node [" +
+ "locId=" + locNode.id() +
+ ", watchPath=" + prevE.getValue() + ']');
+ }
+
+ PreviousNodeWatcher watcher = new ServerPreviousNodeWatcher(rtState);
+
+ rtState.zkClient.existsAsync(zkPaths.aliveNodesDir + "/" + prevE.getValue(), watcher, watcher);
+ }
+ }
+
+ /**
+ * @param aliveNodes Alive nodes.
+ * @throws Exception If failed.
+ */
+ private void checkClientsStatus(final List<String> aliveNodes) throws Exception {
+ assert locNode.isClient() : locNode;
+ assert rtState.joined;
+ assert rtState.evtsData != null;
+
+ TreeMap<Long, String> aliveClients = new TreeMap<>();
+
+ String srvPath = null;
+ Long srvInternalOrder = null;
+
+ long locInternalOrder = rtState.internalOrder;
+
+ for (String aliveNodePath : aliveNodes) {
+ Long internalId = ZkIgnitePaths.aliveInternalId(aliveNodePath);
+
+ if (ZkIgnitePaths.aliveNodeClientFlag(aliveNodePath))
+ aliveClients.put(internalId, aliveNodePath);
+ else {
+ if (srvInternalOrder == null || internalId < srvInternalOrder) {
+ srvPath = aliveNodePath;
+ srvInternalOrder = internalId;
+ }
+ }
+ }
+
+ assert !aliveClients.isEmpty();
+
+ Map.Entry<Long, String> oldest = aliveClients.firstEntry();
+
+ boolean oldestClient = locInternalOrder == oldest.getKey();
+
+ if (srvPath == null) {
+ if (oldestClient) {
+ Stat stat = new Stat();
+
+ ZkDiscoveryEventsData prevEvts = rtState.evtsData;
+
+ byte[] evtsBytes = rtState.zkClient.getData(zkPaths.evtsPath, stat);
+
+ assert evtsBytes.length > 0;
+
+ ZkDiscoveryEventsData newEvts = unmarshalZip(evtsBytes);
+
+ if (prevEvts.clusterId.equals(newEvts.clusterId)) {
+ U.warn(log, "All server nodes failed, notify all clients [locId=" + locNode.id() + ']');
+
+ generateNoServersEvent(newEvts, stat);
+ }
+ else
+ U.warn(log, "All server nodes failed (received events from new cluster).");
+ }
+ }
+ else {
+ String watchPath;
+
+ if (oldestClient) {
+ watchPath = srvPath;
+
+ if (log.isInfoEnabled()) {
+ log.info("Servers exists, watch for server node [locId=" + locNode.id() +
+ ", watchPath=" + watchPath + ']');
+ }
+ }
+ else {
+ assert aliveClients.size() > 1 : aliveClients;
+
+ Map.Entry<Long, String> prevE = aliveClients.floorEntry(locInternalOrder - 1);
+
+ assert prevE != null;
+
+ watchPath = prevE.getValue();
+
+ if (log.isInfoEnabled()) {
+ log.info("Servers exists, watch for previous node [locId=" + locNode.id() +
+ ", watchPath=" + watchPath + ']');
+ }
+ }
+
+ PreviousNodeWatcher watcher = new ClientPreviousNodeWatcher(rtState);
+
+ rtState.zkClient.existsAsync(zkPaths.aliveNodesDir + "/" + watchPath, watcher, watcher);
+ }
+ }
+
+ /**
+ * @param evtsData Events data.
+ * @param evtsStat Events zookeeper state.
+ * @throws Exception If failed.
+ */
+ private void generateNoServersEvent(ZkDiscoveryEventsData evtsData, Stat evtsStat) throws Exception {
+ evtsData.evtIdGen++;
+
+ ZkDiscoveryCustomEventData evtData = new ZkDiscoveryCustomEventData(
+ evtsData.evtIdGen,
+ 0L,
+ evtsData.topVer,
+ locNode.id(),
+ new ZkNoServersMessage(),
+ null);
+
+ Collection<ZookeeperClusterNode> nodesToAck = Collections.emptyList();
+
+ evtsData.addEvent(nodesToAck, evtData);
+
+ byte[] newEvtsBytes = marshalZip(evtsData);
+
+ try {
+ rtState.zkClient.setData(zkPaths.evtsPath, newEvtsBytes, evtsStat.getVersion());
+ }
+ catch (KeeperException.BadVersionException e) {
+ // Version can change if new cluster started and saved new events.
+ if (log.isDebugEnabled())
+ log.debug("Failed to save no servers message");
+ }
+ }
+
+ /**
+ * @param lastEvts Last events from previous coordinator.
+ * @throws Exception If failed.
+ */
+ private void previousCoordinatorCleanup(ZkDiscoveryEventsData lastEvts) throws Exception {
+ for (ZkDiscoveryEventData evtData : lastEvts.evts.values()) {
+ if (evtData instanceof ZkDiscoveryCustomEventData) {
+ ZkDiscoveryCustomEventData evtData0 = (ZkDiscoveryCustomEventData)evtData;
+
+ // It is possible previous coordinator failed before finished cleanup.
+ if (evtData0.msg instanceof ZkCommunicationErrorResolveFinishMessage) {
+ try {
+ ZkCommunicationErrorResolveFinishMessage msg =
+ (ZkCommunicationErrorResolveFinishMessage)evtData0.msg;
+
+ ZkCommunicationErrorResolveResult res = unmarshalZip(
+ ZkDistributedCollectDataFuture.readResult(rtState.zkClient, zkPaths, msg.futId));
+
+ deleteAliveNodes(res.killedNodes);
+ }
+ catch (KeeperException.NoNodeException ignore) {
+ // No-op.
+ }
+ }
+ else if (evtData0.resolvedMsg instanceof ZkForceNodeFailMessage)
+ deleteAliveNode(((ZkForceNodeFailMessage)evtData0.resolvedMsg).nodeInternalId);
+ }
+ }
+ }
+
+ /**
+ * @param aliveNodes Alive nodes paths.
+ * @throws Exception If failed.
+ */
+ private void onBecomeCoordinator(List<String> aliveNodes) throws Exception {
+ ZkDiscoveryEventsData prevEvts = processNewEvents(rtState.zkClient.getData(zkPaths.evtsPath));
+
+ rtState.crd = true;
+
+ if (rtState.joined) {
+ if (log.isInfoEnabled())
+ log.info("Node is new discovery coordinator [locId=" + locNode.id() + ']');
+
+ assert locNode.order() > 0 : locNode;
+ assert rtState.evtsData != null;
+
+ previousCoordinatorCleanup(rtState.evtsData);
+
+ UUID futId = rtState.evtsData.communicationErrorResolveFutureId();
+
+ if (futId != null) {
+ if (log.isInfoEnabled()) {
+ log.info("New discovery coordinator will handle already started cluster-wide communication " +
+ "error resolve [reqId=" + futId + ']');
+ }
+
+ ZkCommunicationErrorProcessFuture fut = commErrProcFut.get();
+
+ ZkDistributedCollectDataFuture collectResFut = collectCommunicationStatusFuture(futId);
+
+ if (fut != null)
+ fut.nodeResultCollectFuture(collectResFut);
+ }
+
+ for (ZkDiscoveryEventData evtData : rtState.evtsData.evts.values())
+ evtData.initRemainingAcks(rtState.top.nodesByOrder.values());
+
+ handleProcessedEvents("crd");
+ }
+ else {
+ String locAlivePath = rtState.locNodeZkPath.substring(rtState.locNodeZkPath.lastIndexOf('/') + 1);
+
+ deleteJoiningNodeData(locNode.id(),
+ ZkIgnitePaths.aliveNodePrefixId(locAlivePath),
+ rtState.joinDataPartCnt);
+
+ DiscoverySpiNodeAuthenticator nodeAuth = spi.getAuthenticator();
+
+ if (nodeAuth != null) {
+ try {
+ if (log.isInfoEnabled()) {
+ log.info("Node is first server node in cluster, try authenticate local node " +
+ "[locId=" + locNode.id() + ']');
+ }
+
+ localAuthentication(nodeAuth, unmarshalCredentials(locNode));
+ }
+ catch (Exception e) {
+ U.warn(log, "Local node authentication failed: " + e, e);
+
+ onSegmented(e);
+
+ // Stop any further processing.
+ throw new ZookeeperClientFailedException("Local node authentication failed: " + e);
+ }
+ }
+
+ newClusterStarted(prevEvts);
+ }
+
+ rtState.zkClient.getChildrenAsync(zkPaths.aliveNodesDir, rtState.watcher, rtState.watcher);
+ rtState.zkClient.getChildrenAsync(zkPaths.customEvtsDir, rtState.watcher, rtState.watcher);
+
+ for (String alivePath : aliveNodes)
+ watchAliveNodeData(alivePath);
+ }
+
+ /**
+ * @param alivePath Node path.
+ */
+ private void watchAliveNodeData(String alivePath) {
+ assert rtState.locNodeZkPath != null;
+
+ String path = zkPaths.aliveNodesDir + "/" + alivePath;
+
+ if (!path.equals(rtState.locNodeZkPath))
+ rtState.zkClient.getDataAsync(path, rtState.aliveNodeDataWatcher, rtState.aliveNodeDataWatcher);
+ }
+
+ /**
+ * @param aliveNodes ZK nodes representing alive cluster nodes.
+ * @throws Exception If failed.
+ */
+ private void generateTopologyEvents(List<String> aliveNodes) throws Exception {
+ assert rtState.crd;
+
+ if (log.isInfoEnabled())
+ log.info("Process alive nodes change [alives=" + aliveNodes.size() + "]");
+
+ if (rtState.updateAlives) {
+ aliveNodes = rtState.zkClient.getChildren(zkPaths.aliveNodesDir);
+
+ rtState.updateAlives = false;
+ }
+
+ TreeMap<Long, String> alives = new TreeMap<>();
+
+ for (String child : aliveNodes) {
+ Long internalId = ZkIgnitePaths.aliveInternalId(child);
+
+ Object old = alives.put(internalId, child);
+
+ assert old == null;
+ }
+
+ TreeMap<Long, ZookeeperClusterNode> curTop = new TreeMap<>(rtState.top.nodesByOrder);
+
+ int newEvts = 0;
+
+ final int MAX_NEW_EVTS = IgniteSystemProperties.getInteger(IGNITE_ZOOKEEPER_DISCOVERY_SPI_MAX_EVTS, 100);
+
+ List<ZookeeperClusterNode> failedNodes = null;
+
+ for (Map.Entry<Long, ZookeeperClusterNode> e : rtState.top.nodesByInternalId.entrySet()) {
+ if (!alives.containsKey(e.getKey())) {
+ ZookeeperClusterNode failedNode = e.getValue();
+
+ if (failedNodes == null)
+ failedNodes = new ArrayList<>();
+
+ failedNodes.add(failedNode);
+
+ generateNodeFail(curTop, failedNode);
+
+ newEvts++;
+
+ if (newEvts == MAX_NEW_EVTS) {
+ saveAndProcessNewEvents();
+
+ if (log.isInfoEnabled()) {
+ log.info("Delay alive nodes change process, max event threshold reached [newEvts=" + newEvts +
+ ", totalEvts=" + rtState.evtsData.evts.size() + ']');
+ }
+
+ handleProcessedEventsOnNodesFail(failedNodes);
+
+ throttleNewEventsGeneration();
+
+ rtState.zkClient.getChildrenAsync(zkPaths.aliveNodesDir, rtState.watcher, rtState.watcher);
+
+ return;
+ }
+ }
+ }
+
+ // Process failures before processing join, otherwise conflicts are possible in case of fast node stop/re-start.
+ if (newEvts > 0) {
+ saveAndProcessNewEvents();
+
+ handleProcessedEventsOnNodesFail(failedNodes);
+
+ rtState.zkClient.getChildrenAsync(zkPaths.aliveNodesDir, rtState.watcher, rtState.watcher);
+
+ return;
+ }
+
+ generateJoinEvents(curTop, alives, MAX_NEW_EVTS);
+
+ if (failedNodes != null)
+ handleProcessedEventsOnNodesFail(failedNodes);
+ }
+
+ /**
+ * @param curTop Current topology.
+ * @param alives Alive znodes.
+ * @param MAX_NEW_EVTS Max event to process.
+ * @throws Exception If failed.
+ */
+ private void generateJoinEvents(TreeMap<Long, ZookeeperClusterNode> curTop,
+ TreeMap<Long, String> alives,
+ final int MAX_NEW_EVTS) throws Exception
+ {
+ ZkBulkJoinContext joinCtx = new ZkBulkJoinContext();
+
+ for (Map.Entry<Long, String> e : alives.entrySet()) {
+ Long internalId = e.getKey();
+
+ if (!rtState.top.nodesByInternalId.containsKey(internalId)) {
+ UUID rslvFutId = rtState.evtsData.communicationErrorResolveFutureId();
+
+ if (rslvFutId != null) {
+ if (log.isInfoEnabled()) {
+ log.info("Delay alive nodes change process while communication error resolve " +
+ "is in progress [reqId=" + rslvFutId + ']');
+ }
+
+ break;
+ }
+
+ processJoinOnCoordinator(joinCtx, curTop, internalId, e.getValue());
+
+ if (joinCtx.nodes() == MAX_NEW_EVTS) {
+ generateBulkJoinEvent(curTop, joinCtx);
+
+ if (log.isInfoEnabled()) {
+ log.info("Delay alive nodes change process, max event threshold reached [" +
+ "newEvts=" + joinCtx.nodes() +
+ ", totalEvts=" + rtState.evtsData.evts.size() + ']');
+ }
+
+ throttleNewEventsGeneration();
+
+ rtState.zkClient.getChildrenAsync(zkPaths.aliveNodesDir, rtState.watcher, rtState.watcher);
+
+ return;
+ }
+ }
+ }
+
+ if (joinCtx.nodes() > 0)
+ generateBulkJoinEvent(curTop, joinCtx);
+ }
+
+ /**
+ * @param curTop Current topology.
+ * @param joinCtx Joined nodes context.
+ * @throws Exception If failed.
+ */
+ private void generateBulkJoinEvent(TreeMap<Long, ZookeeperClusterNode> curTop, ZkBulkJoinContext joinCtx)
+ throws Exception
+ {
+ rtState.evtsData.evtIdGen++;
+
+ long evtId = rtState.evtsData.evtIdGen;
+
+ List<T2<ZkJoinedNodeEvtData, Map<Integer, Serializable>>> nodes = joinCtx.nodes;
+
+ assert nodes != null && nodes.size() > 0;
+
+ int nodeCnt = nodes.size();
+
+ List<ZkJoinedNodeEvtData> joinedNodes = new ArrayList<>(nodeCnt);
+
+ Map<Long, byte[]> discoDataMap = U.newHashMap(nodeCnt);
+ Map<Long, Long> dupDiscoData = null;
+
+ for (int i = 0; i < nodeCnt; i++) {
+ T2<ZkJoinedNodeEvtData, Map<Integer, Serializable>> nodeEvtData = nodes.get(i);
+
+ Map<Integer, Serializable> discoData = nodeEvtData.get2();
+
+ byte[] discoDataBytes = U.marshal(marsh, discoData);
+
+ Long dupDataNode = null;
+
+ for (Map.Entry<Long, byte[]> e : discoDataMap.entrySet()) {
+ if (Arrays.equals(discoDataBytes, e.getValue())) {
+ dupDataNode = e.getKey();
+
+ break;
+ }
+ }
+
+ long nodeTopVer = nodeEvtData.get1().topVer;
+
+ if (dupDataNode != null) {
+ if (dupDiscoData == null)
+ dupDiscoData = new HashMap<>();
+
+ Long old = dupDiscoData.put(nodeTopVer, dupDataNode);
+
+ assert old == null : old;
+ }
+ else
+ discoDataMap.put(nodeTopVer, discoDataBytes);
+
+ joinedNodes.add(nodeEvtData.get1());
+ }
+
+ int overhead = 5;
+
+ ZkJoinEventDataForJoined dataForJoined = new ZkJoinEventDataForJoined(
+ new ArrayList<>(curTop.values()),
+ discoDataMap,
+ dupDiscoData);
+
+ byte[] dataForJoinedBytes = marshalZip(dataForJoined);
+
+ long addDataStart = System.currentTimeMillis();
+
+ int dataForJoinedPartCnt = saveData(zkPaths.joinEventDataPathForJoined(evtId),
+ dataForJoinedBytes,
+ overhead);
+
+ long addDataTime = System.currentTimeMillis() - addDataStart;
+
+ ZkDiscoveryNodeJoinEventData evtData = new ZkDiscoveryNodeJoinEventData(
+ evtId,
+ rtState.evtsData.topVer,
+ joinedNodes,
+ dataForJoinedPartCnt);
+
+ rtState.evtsData.addEvent(curTop.values(), evtData);
+
+ if (log.isInfoEnabled()) {
+ if (nodeCnt > 1) {
+ log.info("Generated NODE_JOINED bulk event [" +
+ "nodeCnt=" + nodeCnt +
+ ", dataForJoinedSize=" + dataForJoinedBytes.length +
+ ", dataForJoinedPartCnt=" + dataForJoinedPartCnt +
+ ", addDataTime=" + addDataTime +
+ ", evt=" + evtData + ']');
+ }
+ else {
+ log.info("Generated NODE_JOINED event [" +
+ "dataForJoinedSize=" + dataForJoinedBytes.length +
+ ", dataForJoinedPartCnt=" + dataForJoinedPartCnt +
+ ", addDataTime=" + addDataTime +
+ ", evt=" + evtData + ']');
+ }
+ }
+
+ saveAndProcessNewEvents();
+ }
+
+ /**
+ *
+ */
+ private void throttleNewEventsGeneration() {
+ long delay = IgniteSystemProperties.getLong(IGNITE_ZOOKEEPER_DISCOVERY_SPI_EVTS_THROTTLE, 0);
+
+ if (delay > 0) {
+ if (log.isInfoEnabled())
+ log.info("Sleep delay before generate new events [delay=" + delay + ']');
+
+ try {
+ Thread.sleep(delay);
+ }
+ catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ }
+ }
+ }
+
+ /**
+ * @param nodeId Node ID.
+ * @param prefixId Path prefix.
+ * @return Join data.
+ * @throws Exception If failed.
+ */
+ private ZkJoiningNodeData unmarshalJoinData(UUID nodeId, UUID prefixId) throws Exception {
+ String joinDataPath = zkPaths.joiningNodeDataPath(nodeId, prefixId);
+
+ byte[] joinData = rtState.zkClient.getData(joinDataPath);
+
+ Object dataObj = unmarshalZip(joinData);
+
+ if (!(dataObj instanceof ZkJoiningNodeData))
+ throw new Exception("Invalid joined node data: " + dataObj);
+
+ ZkJoiningNodeData joiningNodeData = (ZkJoiningNodeData)dataObj;
+
+ if (joiningNodeData.partCount() > 1) {
+ joinData = readMultipleParts(rtState.zkClient, joinDataPath + ":", joiningNodeData.partCount());
+
+ joiningNodeData = unmarshalZip(joinData);
+ }
+
+ return joiningNodeData;
+ }
+
+ /**
+ * @param nodeId Node ID.
+ * @param prefixId Path prefix.
+ * @param aliveNodePath Node path.
+ * @return Join data.
+ * @throws Exception If failed.
+ */
+ private Object unmarshalJoinDataOnCoordinator(UUID nodeId, UUID prefixId, String aliveNodePath) throws Exception {
+ String joinDataPath = zkPaths.joiningNodeDataPath(nodeId, prefixId);
+
+ byte[] joinData = rtState.zkClient.getData(joinDataPath);
+
+ Object dataObj;
+
+ try {
+ dataObj = unmarshalZip(joinData);
+
+ if (dataObj instanceof ZkInternalJoinErrorMessage)
+ return dataObj;
+ }
+ catch (Exception e) {
+ U.error(log, "Failed to unmarshal joining node data [nodePath=" + aliveNodePath + "']", e);
+
+ return new ZkInternalJoinErrorMessage(ZkIgnitePaths.aliveInternalId(aliveNodePath),
+ "Failed to unmarshal join data: " + e);
+ }
+
+ assert dataObj instanceof ZkJoiningNodeData : dataObj;
+
+ ZkJoiningNodeData joiningNodeData = (ZkJoiningNodeData)dataObj;
+
+ if (joiningNodeData.partCount() > 1) {
+ joinData = readMultipleParts(rtState.zkClient, joinDataPath + ":", joiningNodeData.partCount());
+
+ try {
+ joiningNodeData = unmarshalZip(joinData);
+ }
+ catch (Exception e) {
+ U.error(log, "Failed to unmarshal joining node data [nodePath=" + aliveNodePath + "']", e);
+
+ return new ZkInternalJoinErrorMessage(ZkIgnitePaths.aliveInternalId(aliveNodePath),
+ "Failed to unmarshal join data: " + e);
+ }
+ }
+
+ assert joiningNodeData.node() != null : joiningNodeData;
+
+ return joiningNodeData;
+ }
+
+ /**
+ * @param joinCtx Joined nodes context.
+ * @param curTop Current nodes.
+ * @param internalId Joined node internal ID.
+ * @param aliveNodePath Joined node path.
+ * @throws Exception If failed.
+ */
+ private void processJoinOnCoordinator(
+ ZkBulkJoinContext joinCtx,
+ TreeMap<Long, ZookeeperClusterNode> curTop,
+ long internalId,
+ String aliveNodePath)
+ throws Exception
+ {
+ UUID nodeId = ZkIgnitePaths.aliveNodeId(aliveNodePath);
+ UUID prefixId = ZkIgnitePaths.aliveNodePrefixId(aliveNodePath);
+
+ Object data = unmarshalJoinDataOnCoordinator(nodeId, prefixId, aliveNodePath);
+
+ if (data instanceof ZkJoiningNodeData) {
+ ZkJoiningNodeData joiningNodeData = (ZkJoiningNodeData)data;
+
+ ZkNodeValidateResult validateRes = validateJoiningNode(joiningNodeData.node());
+
+ if (validateRes.err == null) {
+ ZookeeperClusterNode joinedNode = joiningNodeData.node();
+
+ assert nodeId.equals(joinedNode.id()) : joiningNodeData.node();
+
+ addJoinedNode(
+ joinCtx,
+ curTop,
+ joiningNodeData,
+ internalId,
+ prefixId,
+ validateRes.secSubjZipBytes);
+
+ watchAliveNodeData(aliveNodePath);
+ }
+ else {
+ ZkInternalJoinErrorMessage joinErr = new ZkInternalJoinErrorMessage(
+ ZkIgnitePaths.aliveInternalId(aliveNodePath),
+ validateRes.err);
+
+ processJoinError(aliveNodePath, nodeId, prefixId, joinErr);
+ }
+ }
+ else {
+ assert data instanceof ZkInternalJoinErrorMessage : data;
+
+ processJoinError(aliveNodePath, nodeId, prefixId, (ZkInternalJoinErrorMessage)data);
+ }
+ }
+
+ /**
+ * @param aliveNodePath Joined node path.
+ * @param nodeId Node ID.
+ * @param prefixId Path prefix ID.
+ * @param joinErr Join error message.
+ * @throws Exception If failed.
+ */
+ private void processJoinError(String aliveNodePath,
+ UUID nodeId,
+ UUID prefixId,
+ ZkInternalJoinErrorMessage joinErr) throws Exception {
+ ZookeeperClient client = rtState.zkClient;
+
+ if (joinErr.notifyNode) {
+ String joinDataPath = zkPaths.joiningNodeDataPath(nodeId, prefixId);
+
+ client.setData(joinDataPath, marshalZip(joinErr), -1);
+
+ client.deleteIfExists(zkPaths.aliveNodesDir + "/" + aliveNodePath, -1);
+ }
+ else {
+ if (log.isInfoEnabled())
+ log.info("Ignore join data, node was failed by previous coordinator: " + aliveNodePath);
+
+ client.deleteIfExists(zkPaths.aliveNodesDir + "/" + aliveNodePath, -1);
+ }
+ }
+
+ /**
+ * @param node Joining node.
+ * @return Validation result.
+ */
+ private ZkNodeValidateResult validateJoiningNode(ZookeeperClusterNode node) {
+ ZookeeperClusterNode node0 = rtState.top.nodesById.get(node.id());
+
+ if (node0 != null) {
+ U.error(log, "Failed to include node in cluster, node with the same ID already exists [joiningNode=" + node +
+ ", existingNode=" + node0 + ']');
+
+ // Note: exception message is checked in tests.
+ return new ZkNodeValidateResult("Node with the same ID already exists: " + node0);
+ }
+
+ ZkNodeValidateResult res = authenticateNode(node);
+
+ if (res.err != null)
+ return res;
+
+ IgniteNodeValidationResult err = spi.getSpiContext().validateNode(node);
+
+ if (err != null) {
+ LT.warn(log, err.message());
+
+ res.err = err.sendMessage();
+ }
+
+ return res;
+ }
+
+ /**
+ * @param node Node.
+ * @return Validation result.
+ */
+ private ZkNodeValidateResult authenticateNode(ZookeeperClusterNode node) {
+ DiscoverySpiNodeAuthenticator nodeAuth = spi.getAuthenticator();
+
+ if (nodeAuth == null)
+ return new ZkNodeValidateResult((byte[])null);
+
+ SecurityCredentials cred;
+
+ try {
+ cred = unmarshalCredentials(node);
+ }
+ catch (Exception e) {
+ U.error(log, "Failed to unmarshal node credentials: " + e, e);
+
+ return new ZkNodeValidateResult("Failed to unmarshal node credentials");
+ }
+
+ SecurityContext subj = nodeAuth.authenticateNode(node, cred);
+
+ if (subj == null) {
+ U.warn(log, "Authentication failed [nodeId=" + node.id() +
+ ", addrs=" + U.addressesAsString(node) + ']',
+ "Authentication failed [nodeId=" + U.id8(node.id()) + ", addrs=" +
+ U.addressesAsString(node) + ']');
+
+ // Note: exception message test is checked in tests.
+ return new ZkNodeValidateResult("Authentication failed");
+ }
+
+ if (!(subj instanceof Serializable)) {
+ U.warn(log, "Authentication subject is not Serializable [nodeId=" + node.id() +
+ ", addrs=" + U.addressesAsString(node) + ']',
+ "Authentication subject is not Serializable [nodeId=" + U.id8(node.id()) +
+ ", addrs=" +
+ U.addressesAsString(node) + ']');
+
+ return new ZkNodeValidateResult("Authentication subject is not serializable");
+ }
+
+ byte[] secSubjZipBytes;
+
+ try {
+ secSubjZipBytes = marshalZip(subj);
+ }
+ catch (Exception e) {
+ U.error(log, "Failed to marshal node security subject: " + e, e);
+
+ return new ZkNodeValidateResult("Failed to marshal node security subject");
+ }
+
+ return new ZkNodeValidateResult(secSubjZipBytes);
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ private void saveAndProcessNewEvents() throws Exception {
+ if (stopping())
+ return;
+
+ long start = System.currentTimeMillis();
+
+ byte[] evtsBytes = marshalZip(rtState.evtsData);
+
+ rtState.zkClient.setData(zkPaths.evtsPath, evtsBytes, -1);
+
+ long time = System.currentTimeMillis() - start;
+
+ if (prevSavedEvtsTopVer != rtState.evtsData.topVer) {
+ if (log.isInfoEnabled()) {
+ log.info("Discovery coordinator saved new topology events [topVer=" + rtState.evtsData.topVer +
+ ", size=" + evtsBytes.length +
+ ", evts=" + rtState.evtsData.evts.size() +
+ ", lastEvt=" + rtState.evtsData.evtIdGen +
+ ", saveTime=" + time + ']');
+ }
+
+ prevSavedEvtsTopVer = rtState.evtsData.topVer;
+ }
+ else if (log.isDebugEnabled()) {
+ log.debug("Discovery coordinator saved new topology events [topVer=" + rtState.evtsData.topVer +
+ ", size=" + evtsBytes.length +
+ ", evts=" + rtState.evtsData.evts.size() +
+ ", lastEvt=" + rtState.evtsData.evtIdGen +
+ ", saveTime=" + time + ']');
+ }
+
+ processNewEvents(rtState.evtsData);
+ }
+
+ /**
+ * @param curTop Current topology.
+ * @param failedNode Failed node.
+ */
+ private void generateNodeFail(TreeMap<Long, ZookeeperClusterNode> curTop, ZookeeperClusterNode failedNode) {
+ Object rmvd = curTop.remove(failedNode.order());
+
+ assert rmvd != null;
+
+ rtState.evtsData.topVer++;
+ rtState.evtsData.evtIdGen++;
+
+ ZkDiscoveryNodeFailEventData evtData = new ZkDiscoveryNodeFailEventData(
+ rtState.evtsData.evtIdGen,
+ rtState.evtsData.topVer,
+ failedNode.internalId());
+
+ rtState.evtsData.addEvent(curTop.values(), evtData);
+
+ if (log.isInfoEnabled())
+ log.info("Generated NODE_FAILED event [evt=" + evtData + ']');
+ }
+
+ /**
+ * @param curTop Current nodes.
+ * @param joiningNodeData Join data.
+ * @param internalId Joined node internal ID.
+ * @param prefixId Unique path prefix.
+ * @param secSubjZipBytes Marshalled security subject.
+ * @throws Exception If failed.
+ */
+ private void addJoinedNode(
+ ZkBulkJoinContext joinCtx,
+ TreeMap<Long, ZookeeperClusterNode> curTop,
+ ZkJoiningNodeData joiningNodeData,
+ long internalId,
+ UUID prefixId,
+ @Nullable byte[] secSubjZipBytes)
+ throws Exception
+ {
+ ZookeeperClusterNode joinedNode = joiningNodeData.node();
+
+ UUID nodeId = joinedNode.id();
+
+ rtState.evtsData.topVer++;
+
+ joinedNode.order(rtState.evtsData.topVer);
+ joinedNode.internalId(internalId);
+
+ DiscoveryDataBag joiningNodeBag = new DiscoveryDataBag(nodeId, joiningNodeData.node().isClient());
+
+ joiningNodeBag.joiningNodeData(joiningNodeData.discoveryData());
+
+ exchange.onExchange(joiningNodeBag);
+
+ DiscoveryDataBag collectBag = new DiscoveryDataBag(nodeId,
+ new HashSet<Integer>(),
+ joiningNodeData.node().isClient());
+
+ collectBag.joiningNodeData(joiningNodeBag.joiningNodeData());
+
+ exchange.collect(collectBag);
+
+ Map<Integer, Serializable> commonData = collectBag.commonData();
+
+ Object old = curTop.put(joinedNode.order(), joinedNode);
+
+ assert old == null;
+
+ int overhead = 5;
+
+ int secSubjPartCnt = 0;
+
+ if (secSubjZipBytes != null) {
+ secSubjPartCnt = saveData(zkPaths.joinEventSecuritySubjectPath(joinedNode.order()),
+ secSubjZipBytes,
+ overhead);
+
+ assert secSubjPartCnt > 0 : secSubjPartCnt;
+
+ setNodeSecuritySubject(joinedNode, secSubjZipBytes);
+ }
+
+ ZkJoinedNodeEvtData nodeEvtData = new ZkJoinedNodeEvtData(
+ rtState.evtsData.topVer,
+ joinedNode.id(),
+ joinedNode.internalId(),
+ prefixId,
+ joiningNodeData.partCount(),
+ secSubjPartCnt);
+
+ nodeEvtData.joiningNodeData = joiningNodeData;
+
+ joinCtx.addJoinedNode(nodeEvtData, commonData);
+
+ rtState.evtsData.onNodeJoin(joinedNode);
+ }
+
+ /**
+ * @param path Path to save.
+ * @param bytes Bytes to save.
+ * @param overhead Extra overhead.
+ * @return Parts count.
+ * @throws Exception If failed.
+ */
+ private int saveData(String path, byte[] bytes, int overhead) throws Exception {
+ int dataForJoinedPartCnt = 1;
+
+ if (rtState.zkClient.needSplitNodeData(path, bytes, overhead)) {
+ dataForJoinedPartCnt = saveMultipleParts(rtState.zkClient,
+ path,
+ rtState.zkClient.splitNodeData(path, bytes, overhead));
+ }
+ else {
+ rtState.zkClient.createIfNeeded(multipartPathName(path, 0),
+ bytes,
+ PERSISTENT);
+ }
+
+ return dataForJoinedPartCnt;
+ }
+
+ /**
+ * @param prevEvts Events from previous cluster.
+ * @throws Exception If failed.
+ */
+ @SuppressWarnings("unchecked")
+ private void newClusterStarted(@Nullable ZkDiscoveryEventsData prevEvts) throws Exception {
+ assert !locNode.isClient() : locNode;
+
+ long locInternalId = rtState.internalOrder;
+
+ assert prevEvts == null || prevEvts.maxInternalOrder < locInternalId;
+
+ spi.getSpiContext().removeTimeoutObject(rtState.joinErrTo);
+
+ cleanupPreviousClusterData(prevEvts != null ? prevEvts.maxInternalOrder + 1 : -1L);
+
+ rtState.joined = true;
+ rtState.gridStartTime = System.currentTimeMillis();
+
+ rtState.evtsData = ZkDiscoveryEventsData.createForNewCluster(rtState.gridStartTime);
+
+ if (log.isInfoEnabled()) {
+ log.info("New cluster started [locId=" + locNode.id() +
+ ", clusterId=" + rtState.evtsData.clusterId +
+ ", startTime=" + rtState.evtsData.clusterStartTime + ']');
+ }
+
+ locNode.internalId(locInternalId);
+ locNode.order(1);
+
+ rtState.evtsData.onNodeJoin(locNode);
+
+ rtState.top.addNode(locNode);
+
+ final List<ClusterNode> topSnapshot = Collections.singletonList((ClusterNode)locNode);
+
+ lsnr.onDiscovery(EVT_NODE_JOINED,
+ 1L,
+ locNode,
+ topSnapshot,
+ Collections.<Long, Collection<ClusterNode>>emptyMap(),
+ null);
+
+ // Reset events (this is also notification for clients left from previous cluster).
+ rtState.zkClient.setData(zkPaths.evtsPath, marshalZip(rtState.evtsData), -1);
+
+ joinFut.onDone();
+ }
+
+ /**
+ * @param startInternalOrder Starting internal order for cluster (znodes having lower order belong
+ * to clients from previous cluster and should be removed).
+
+ * @throws Exception If failed.
+ */
+ private void cleanupPreviousClusterData(long startInternalOrder) throws Exception {
+ long start = System.currentTimeMillis();
+
+ ZookeeperClient client = rtState.zkClient;
+
+ // TODO ZK: use multi, better batching + max-size safe + NoNodeException safe.
+ List<String> evtChildren = rtState.zkClient.getChildren(zkPaths.evtsPath);
+
+ for (String evtPath : evtChildren) {
+ String evtDir = zkPaths.evtsPath + "/" + evtPath;
+
+ removeChildren(evtDir);
+ }
+
+ client.deleteAll(zkPaths.evtsPath, evtChildren, -1);
+
+ client.deleteAll(zkPaths.customEvtsDir,
+ client.getChildren(zkPaths.customEvtsDir),
+ -1);
+
+ rtState.zkClient.deleteAll(zkPaths.customEvtsPartsDir,
+ rtState.zkClient.getChildren(zkPaths.customEvtsPartsDir),
+ -1);
+
+ rtState.zkClient.deleteAll(zkPaths.customEvtsAcksDir,
+ rtState.zkClient.getChildren(zkPaths.customEvtsAcksDir),
+ -1);
+
+ if (startInternalOrder > 0) {
+ for (String alive : rtState.zkClient.getChildren(zkPaths.aliveNodesDir)) {
+ if (ZkIgnitePaths.aliveInternalId(alive) < startInternalOrder)
+ rtState.zkClient.deleteIfExists(zkPaths.aliveNodesDir + "/" + alive, -1);
+ }
+ }
+
+ long time = System.currentTimeMillis() - start;
+
+ if (time > 0) {
+ if (log.isInfoEnabled())
+ log.info("Previous cluster data cleanup time: " + time);
+ }
+ }
+
+ /**
+ * @param path Path.
+ * @throws Exception If failed.
+ */
+ private void removeChildren(String path) throws Exception {
+ rtState.zkClient.deleteAll(path, rtState.zkClient.getChildren(path), -1);
+ }
+
+ /**
+ * @param zkClient Client.
+ * @param evtPath Event path.
+ * @param sndNodeId Sender node ID.
+ * @return Event data.
+ * @throws Exception If failed.
+ */
+ private byte[] readCustomEventData(ZookeeperClient zkClient, String evtPath, UUID sndNodeId) throws Exception {
+ int partCnt = ZkIgnitePaths.customEventPartsCount(evtPath);
+
+ if (partCnt > 1) {
+ String partsBasePath = zkPaths.customEventPartsBasePath(
+ ZkIgnitePaths.customEventPrefix(evtPath), sndNodeId);
+
+ return readMultipleParts(zkClient, partsBasePath, partCnt);
+ }
+ else
+ return zkClient.getData(zkPaths.customEvtsDir + "/" + evtPath);
+ }
+
+ /**
+ * @param customEvtNodes ZK nodes representing custom events to process.
+ * @throws Exception If failed.
+ */
+ private void generateCustomEvents(List<String> customEvtNodes) throws Exception {
+ assert rtState.crd;
+
+ ZookeeperClient zkClient = rtState.zkClient;
+ ZkDiscoveryEventsData evtsData = rtState.evtsData;
+
+ TreeMap<Integer, String> unprocessedEvts = null;
+
+ for (int i = 0; i < customEvtNodes.size(); i++) {
+ String evtPath = customEvtNodes.get(i);
+
+ int evtSeq = ZkIgnitePaths.customEventSequence(evtPath);
+
+ if (evtSeq > evtsData.procCustEvt) {
+ if (unprocessedEvts == null)
+ unprocessedEvts = new TreeMap<>();
+
+ unprocessedEvts.put(evtSeq, evtPath);
+ }
+ }
+
+ if (unprocessedEvts == null)
+ return;
+
+ for (Map.Entry<Integer, String> evtE : unprocessedEvts.entrySet()) {
+ evtsData.procCustEvt = evtE.getKey();
+
+ String evtPath = evtE.getValue();
+
+ UUID sndNodeId = ZkIgnitePaths.customEventSendNodeId(evtPath);
+
+ ZookeeperClusterNode sndNode = rtState.top.nodesById.get(sndNodeId);
+
+ if (sndNode != null) {
+ byte[] evtBytes = readCustomEventData(zkClient, evtPath, sndNodeId);
+
+ DiscoverySpiCustomMessage msg;
+
+ try {
+ msg = unmarshalZip(evtBytes);
+ }
+ catch (Exception e) {
+ U.error(log, "Failed to unmarshal custom discovery message: " + e, e);
+
+ deleteCustomEventDataAsync(rtState.zkClient, evtPath);
+
+ continue;
+ }
+
+ generateAndProcessCustomEventOnCoordinator(evtPath, sndNode, msg);
+ }
+ else {
+ U.warn(log, "Ignore custom event from unknown node: " + sndNodeId);
+
+ deleteCustomEventDataAsync(rtState.zkClient, evtPath);
+ }
+ }
+ }
+
+ /**
+ * @param evtPath Event data path.
+ * @param sndNode Sender node.
+ * @param msg Message instance.
+ * @throws Exception If failed.
+ */
+ private void generateAndProcessCustomEventOnCoordinator(String evtPath,
+ ZookeeperClusterNode sndNode,
+ DiscoverySpiCustomMessage msg) throws Exception
+ {
+ ZookeeperClient zkClient = rtState.zkClient;
+ ZkDiscoveryEventsData evtsData = rtState.evtsData;
+
+ ZookeeperClusterNode failedNode = null;
+
+ if (msg instanceof ZkForceNodeFailMessage) {
+ ZkForceNodeFailMessage msg0 = (ZkForceNodeFailMessage)msg;
+
+ failedNode = rtState.top.nodesByInternalId.get(msg0.nodeInternalId);
+
+ if (failedNode != null)
+ evtsData.topVer++;
+ else {
+ if (log.isDebugEnabled())
+ log.debug("Ignore forcible node fail request for unknown node: " + msg0.nodeInternalId);
+
+ deleteCustomEventDataAsync(zkClient, evtPath);
+
+ return;
+ }
+ }
+ else if (msg instanceof ZkCommunicationErrorResolveStartMessage) {
+ ZkCommunicationErrorResolveStartMessage msg0 =
+ (ZkCommunicationErrorResolveStartMessage)msg;
+
+ if (evtsData.communicationErrorResolveFutureId() != null) {
+ if (log.isInfoEnabled()) {
+ log.info("Ignore communication error resolve message, resolve process " +
+ "already started [sndNode=" + sndNode + ']');
+ }
+
+ deleteCustomEventDataAsync(zkClient, evtPath);
+
+ return;
+ }
+ else {
+ if (log.isInfoEnabled()) {
+ log.info("Start cluster-wide communication error resolve [sndNode=" + sndNode +
+ ", reqId=" + msg0.id +
+ ", topVer=" + evtsData.topVer + ']');
+ }
+
+ zkClient.createIfNeeded(zkPaths.distributedFutureBasePath(msg0.id),
+ null,
+ PERSISTENT);
+
+ evtsData.communicationErrorResolveFutureId(msg0.id);
+ }
+ }
+
+ evtsData.evtIdGen++;
+
+ ZkDiscoveryCustomEventData evtData = new ZkDiscoveryCustomEventData(
+ evtsData.evtIdGen,
+ 0L,
+ evtsData.topVer,
+ sndNode.id(),
+ null,
+ evtPath);
+
+ evtData.resolvedMsg = msg;
+
+ if (log.isDebugEnabled())
+ log.debug("Generated CUSTOM event [evt=" + evtData + ", msg=" + msg + ']');
+
+ boolean fastStopProcess = false;
+
+ if (msg instanceof ZkInternalMessage)
+ processInternalMessage(evtData, (ZkInternalMessage)msg);
+ else {
+ notifyCustomEvent(evtData, msg);
+
+ if (msg.stopProcess()) {
+ if (log.isDebugEnabled())
+ log.debug("Fast stop process custom event [evt=" + evtData + ", msg=" + msg + ']');
+
+ fastStopProcess = true;
+
+ // No need to process this event on others nodes, skip this event.
+ evtsData.evts.remove(evtData.eventId());
+
+ evtsData.evtIdGen--;
+
+ DiscoverySpiCustomMessage ack = msg.ackMessage();
+
+ if (ack != null) {
+ evtData = createAckEvent(ack, evtData);
+
+ if (log.isDebugEnabled())
+ log.debug("Generated CUSTOM event (ack for fast stop process) [evt=" + evtData + ", msg=" + msg + ']');
+
+ notifyCustomEvent(evtData, ack);
+ }
+ else
+ evtData = null;
+ }
+ }
+
+ if (evtData != null) {
+ evtsData.addEvent(rtState.top.nodesByOrder.values(), evtData);
+
+ rtState.locNodeInfo.lastProcEvt = evtData.eventId();
+
+ saveAndProcessNewEvents();
+
+ if (fastStopProcess)
+ deleteCustomEventDataAsync(zkClient, evtPath);
+
+ if (failedNode != null) {
+ deleteAliveNode(failedNode.internalId());
+
+ handleProcessedEventsOnNodesFail(Collections.singletonList(failedNode));
+
+ rtState.updateAlives = true;
+ }
+ }
+ }
+
+ /**
+ * @param internalId Node internal ID.
+ * @throws Exception If failed.
+ */
+ private void deleteAliveNode(long internalId) throws Exception {
+ for (String child : rtState.zkClient.getChildren(zkPaths.aliveNodesDir)) {
+ if (ZkIgnitePaths.aliveInternalId(child) == internalId) {
+ // Need use sync delete to do not process again join of this node again.
+ rtState.zkClient.deleteIfExists(zkPaths.aliveNodesDir + "/" + child, -1);
+
+ return;
+ }
+ }
+ }
+
+ /**
+ * @param zkClient Client.
+ * @param evtPath Event path.
+ */
+ private void deleteCustomEventDataAsync(ZookeeperClient zkClient, String evtPath) {
+ if (log.isDebugEnabled())
+ log.debug("Delete custom event data: " + evtPath);
+
+ String prefix = ZkIgnitePaths.customEventPrefix(evtPath);
+ UUID sndNodeId = ZkIgnitePaths.customEventSendNodeId(evtPath);
+ int partCnt = ZkIgnitePaths.customEventPartsCount(evtPath);
+
+ assert partCnt >= 1 : partCnt;
+
+ if (partCnt > 1) {
+ for (int i = 0; i < partCnt; i++) {
+ String path0 = zkPaths.customEventPartPath(prefix, sndNodeId, i);
+
+ zkClient.deleteIfExistsAsync(path0);
+ }
+ }
+
+ zkClient.deleteIfExistsAsync(zkPaths.customEvtsDir + "/" + evtPath);
+ }
+
+ /**
+ * @param data Marshalled events.
+ * @throws Exception If failed.
+ * @return Events.
+ */
+ @Nullable private ZkDiscoveryEventsData processNewEvents(byte[] data) throws Exception {
+ ZkDiscoveryEventsData newEvts = data.length > 0 ? (ZkDiscoveryEventsData)unmarshalZip(data) : null;
+
+ if (rtState.joined && (newEvts == null || !rtState.evtsData.clusterId.equals(newEvts.clusterId))) {
+ assert locNode.isClient() : locNode;
+
+ throw localNodeFail("All server nodes failed, client node disconnected (received events from new custer) " +
+ "[locId=" + locNode.id() + ']', true);
+ }
+
+ if (newEvts == null)
+ return null;
+
+ assert !rtState.crd;
+
+ // Need keep processed custom events since they contain message object which is needed to create ack.
+ if (!locNode.isClient() && rtState.evtsData != null) {
+ for (Map.Entry<Long, ZkDiscoveryEventData> e : rtState.evtsData.evts.entrySet()) {
+ ZkDiscoveryEventData evtData = e.getValue();
+
+ if (evtData.eventType() == ZkDiscoveryEventData.ZK_EVT_CUSTOM_EVT) {
+ ZkDiscoveryCustomEventData evtData0 =
+ (ZkDiscoveryCustomEventData)newEvts.evts.get(evtData.eventId());
+
+ if (evtData0 != null)
+ evtData0.resolvedMsg = ((ZkDiscoveryCustomEventData)evtData).resolvedMsg;
+ }
+ }
+ }
+
+ processNewEvents(newEvts);
+
+ if (rtState.joined)
+ rtState.evtsData = newEvts;
+
+ return newEvts;
+ }
+
+ /**
+ * @param evtsData Ev
<TRUNCATED>
[06/12] ignite git commit: IGNITE-7222 Added ZooKeeper discovery SPI
Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorResolveStartMessage.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorResolveStartMessage.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorResolveStartMessage.java
new file mode 100644
index 0000000..0c79c36
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorResolveStartMessage.java
@@ -0,0 +1,61 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.util.UUID;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Zk Communication Error Resolve Start Message.
+ */
+public class ZkCommunicationErrorResolveStartMessage implements DiscoverySpiCustomMessage, ZkInternalMessage {
+ /** */
+ private static final long serialVersionUID = 0L;
+
+ /** */
+ final UUID id;
+
+ /**
+ * @param id Unique ID.
+ */
+ ZkCommunicationErrorResolveStartMessage(UUID id) {
+ this.id = id;
+ }
+
+ /** {@inheritDoc} */
+ @Nullable @Override public DiscoverySpiCustomMessage ackMessage() {
+ return null;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean isMutable() {
+ return false;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean stopProcess() {
+ return false;
+ }
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return S.toString(ZkCommunicationErrorResolveStartMessage.class, this);
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationFailureContext.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationFailureContext.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationFailureContext.java
new file mode 100644
index 0000000..d27b717
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationFailureContext.java
@@ -0,0 +1,188 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.CommunicationFailureContext;
+import org.apache.ignite.internal.processors.affinity.GridAffinityAssignmentCache;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ *
+ */
+class ZkCommunicationFailureContext implements CommunicationFailureContext {
+ /** */
+ private static final Comparator<ClusterNode> NODE_ORDER_CMP = new Comparator<ClusterNode>() {
+ @Override public int compare(ClusterNode node1, ClusterNode node2) {
+ return Long.compare(node1.order(), node2.order());
+ }
+ };
+
+ /** */
+ private Set<ClusterNode> killedNodes = new HashSet<>();
+
+ /** */
+ private final Map<UUID, BitSet> nodesState;
+
+ /** */
+ private final List<ClusterNode> initialNodes;
+
+ /** */
+ private final List<ClusterNode> curNodes;
+
+ /** */
+ private final GridCacheSharedContext<?, ?> ctx;
+
+ /**
+ * @param ctx Context.
+ * @param curNodes Current topology snapshot.
+ * @param initialNodes Topology snapshot when communication error resolve started.
+ * @param nodesState Nodes communication state.
+ */
+ ZkCommunicationFailureContext(
+ GridCacheSharedContext<?, ?> ctx,
+ List<ClusterNode> curNodes,
+ List<ClusterNode> initialNodes,
+ Map<UUID, BitSet> nodesState)
+ {
+ this.ctx = ctx;
+ this.curNodes = Collections.unmodifiableList(curNodes);
+ this.initialNodes = initialNodes;
+ this.nodesState = nodesState;
+ }
+
+ /** {@inheritDoc} */
+ @Override public List<ClusterNode> topologySnapshot() {
+ return curNodes;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean connectionAvailable(ClusterNode node1, ClusterNode node2) {
+ BitSet nodeState = nodesState.get(node1.id());
+
+ if (nodeState == null)
+ throw new IllegalArgumentException("Invalid node: " + node1);
+
+ int nodeIdx = Collections.binarySearch(initialNodes, node2, NODE_ORDER_CMP);
+
+ if (nodeIdx < 0)
+ throw new IllegalArgumentException("Invalid node: " + node2);
+
+ assert nodeIdx < nodeState.size() : nodeIdx;
+
+ return nodeState.get(nodeIdx);
+ }
+
+ /** {@inheritDoc} */
+ @Override public Map<String, CacheConfiguration<?, ?>> startedCaches() {
+ Map<Integer, DynamicCacheDescriptor> cachesMap = ctx.affinity().caches();
+
+ Map<String, CacheConfiguration<?, ?>> res = U.newHashMap(cachesMap.size());
+
+ for (DynamicCacheDescriptor desc : cachesMap.values()) {
+ if (desc.cacheType().userCache())
+ res.put(desc.cacheName(), desc.cacheConfiguration());
+ }
+
+ return res;
+ }
+
+ /** {@inheritDoc} */
+ @Override public List<List<ClusterNode>> cacheAffinity(String cacheName) {
+ if (cacheName == null)
+ throw new NullPointerException("Null cache name.");
+
+ DynamicCacheDescriptor cacheDesc = ctx.affinity().caches().get(CU.cacheId(cacheName));
+
+ if (cacheDesc == null)
+ throw new IllegalArgumentException("Invalid cache name: " + cacheName);
+
+ GridAffinityAssignmentCache aff = ctx.affinity().groupAffinity(cacheDesc.groupId());
+
+ assert aff != null : cacheName;
+
+ return aff.readyAssignments(aff.lastVersion());
+ }
+
+ /** {@inheritDoc} */
+ @Override public List<List<ClusterNode>> cachePartitionOwners(String cacheName) {
+ if (cacheName == null)
+ throw new NullPointerException("Null cache name.");
+
+ DynamicCacheDescriptor cacheDesc = ctx.affinity().caches().get(CU.cacheId(cacheName));
+
+ if (cacheDesc == null)
+ throw new IllegalArgumentException("Invalid cache name: " + cacheName);
+
+ if (cacheDesc.cacheConfiguration().getCacheMode() == CacheMode.LOCAL)
+ return Collections.emptyList();
+
+ CacheGroupContext grp = ctx.cache().cacheGroup(cacheDesc.groupId());
+
+ GridDhtPartitionTopology top;
+
+ if (grp == null) {
+ top = ctx.exchange().clientTopologyIfExists(cacheDesc.groupId());
+
+ assert top != null : cacheName;
+ }
+ else
+ top = grp.topology();
+
+ return top.allOwners();
+ }
+
+ /** {@inheritDoc} */
+ @Override public void killNode(ClusterNode node) {
+ if (node == null)
+ throw new NullPointerException();
+
+ if (Collections.binarySearch(curNodes, node, NODE_ORDER_CMP) < 0)
+ throw new IllegalArgumentException("Invalid node: " + node);
+
+ killedNodes.add(node);
+ }
+
+ /**
+ * @return Nodes to fail.
+ */
+ Set<ClusterNode> killedNodes() {
+ return killedNodes;
+ }
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return "ZkCommunicationFailureContext []";
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryCustomEventData.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryCustomEventData.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryCustomEventData.java
new file mode 100644
index 0000000..21dfe62
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryCustomEventData.java
@@ -0,0 +1,89 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.util.UUID;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
+
+/**
+ *
+ */
+class ZkDiscoveryCustomEventData extends ZkDiscoveryEventData {
+ /** */
+ private static final long serialVersionUID = 0L;
+
+ /** */
+ final long origEvtId;
+
+ /** */
+ final UUID sndNodeId;
+
+ /** */
+ final String evtPath;
+
+ /** Message instance (can be marshalled as part of ZkDiscoveryCustomEventData or stored in separate znode. */
+ DiscoverySpiCustomMessage msg;
+
+ /** Unmarshalled message. */
+ transient DiscoverySpiCustomMessage resolvedMsg;
+
+ /**
+ * @param evtId Event ID.
+ * @param origEvtId For acknowledge events ID of original event.
+ * @param topVer Topology version.
+ * @param sndNodeId Sender node ID.
+ * @param msg Message instance.
+ * @param evtPath Event path.
+ */
+ ZkDiscoveryCustomEventData(
+ long evtId,
+ long origEvtId,
+ long topVer,
+ UUID sndNodeId,
+ DiscoverySpiCustomMessage msg,
+ String evtPath)
+ {
+ super(evtId, ZK_EVT_CUSTOM_EVT, topVer);
+
+ assert sndNodeId != null;
+ assert msg != null || origEvtId != 0 || !F.isEmpty(evtPath);
+
+ this.origEvtId = origEvtId;
+ this.msg = msg;
+ this.sndNodeId = sndNodeId;
+ this.evtPath = evtPath;
+ }
+
+ /**
+ * @return {@code True} for custom event ack message.
+ */
+ boolean ackEvent() {
+ return origEvtId != 0;
+ }
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return "ZkDiscoveryCustomEventData [" +
+ "evtId=" + eventId() +
+ ", topVer=" + topologyVersion() +
+ ", sndNode=" + sndNodeId +
+ ", ack=" + ackEvent() +
+ ']';
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryEventData.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryEventData.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryEventData.java
new file mode 100644
index 0000000..d667a17
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryEventData.java
@@ -0,0 +1,165 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Set;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ *
+ */
+abstract class ZkDiscoveryEventData implements Serializable {
+ /** */
+ static final byte ZK_EVT_NODE_JOIN = 1;
+
+ /** */
+ static final byte ZK_EVT_NODE_FAILED = 2;
+
+ /** */
+ static final byte ZK_EVT_CUSTOM_EVT = 3;
+
+ /** */
+ private static final long serialVersionUID = 0L;
+
+ /** */
+ private final long evtId;
+
+ /** */
+ private final byte evtType;
+
+ /** */
+ private final long topVer;
+
+ /** */
+ private transient Set<Long> remainingAcks;
+
+ /** */
+ int flags;
+
+ /**
+ * @param evtId Event ID.
+ * @param evtType Event type.
+ * @param topVer Topology version.
+ */
+ ZkDiscoveryEventData(long evtId, byte evtType, long topVer) {
+ assert evtType == ZK_EVT_NODE_JOIN || evtType == ZK_EVT_NODE_FAILED || evtType == ZK_EVT_CUSTOM_EVT : evtType;
+
+ this.evtId = evtId;
+ this.evtType = evtType;
+ this.topVer = topVer;
+ }
+
+ /**
+ * @param nodes Current nodes in topology.
+ */
+ void initRemainingAcks(Collection<ZookeeperClusterNode> nodes) {
+ assert remainingAcks == null : this;
+
+ remainingAcks = U.newHashSet(nodes.size());
+
+ for (ZookeeperClusterNode node : nodes) {
+ if (!node.isLocal() && node.order() <= topVer) {
+ boolean add = remainingAcks.add(node.internalId());
+
+ assert add : node;
+ }
+ }
+ }
+
+ /**
+ * @param node Node.
+ */
+ void addRemainingAck(ZookeeperClusterNode node) {
+ assert node.order() <= topVer : node;
+
+ boolean add = remainingAcks.add(node.internalId());
+
+ assert add : node;
+ }
+
+ /**
+ * @return {@code True} if all nodes processed event.
+ */
+ boolean allAcksReceived() {
+ return remainingAcks.isEmpty();
+ }
+
+ /**
+ * @return Remaining acks.
+ */
+ Set<Long> remainingAcks() {
+ return remainingAcks;
+ }
+
+ /**
+ * @param nodeInternalId Node ID.
+ * @param ackEvtId Last event ID processed on node.
+ * @return {@code True} if all nodes processed event.
+ */
+ boolean onAckReceived(Long nodeInternalId, long ackEvtId) {
+ assert remainingAcks != null;
+
+ if (ackEvtId >= evtId)
+ remainingAcks.remove(nodeInternalId);
+
+ return remainingAcks.isEmpty();
+ }
+
+ /**
+ * @param node Failed node.
+ * @return {@code True} if all nodes processed event.
+ */
+ boolean onNodeFail(ZookeeperClusterNode node) {
+ assert remainingAcks != null : this;
+
+ remainingAcks.remove(node.internalId());
+
+ return remainingAcks.isEmpty();
+ }
+
+ /**
+ * @param flag Flag mask.
+ * @return {@code True} if flag set.
+ */
+ boolean flagSet(int flag) {
+ return (flags & flag) == flag;
+ }
+
+ /**
+ * @return Event ID.
+ */
+ long eventId() {
+ return evtId;
+ }
+
+ /**
+ * @return Event type.
+ */
+ byte eventType() {
+ return evtType;
+ }
+
+ /**
+ * @return Event topology version.
+ */
+ long topologyVersion() {
+ return topVer;
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryEventsData.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryEventsData.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryEventsData.java
new file mode 100644
index 0000000..dce861b
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryEventsData.java
@@ -0,0 +1,121 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.TreeMap;
+import java.util.UUID;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ *
+ */
+class ZkDiscoveryEventsData implements Serializable {
+ /** */
+ private static final long serialVersionUID = 0L;
+
+ /** Unique cluster ID (generated when first node in cluster starts). */
+ final UUID clusterId;
+
+ /** Internal order of last processed custom event. */
+ long procCustEvt = -1;
+
+ /** Event ID counter. */
+ long evtIdGen;
+
+ /** Current topology version. */
+ long topVer;
+
+ /** Max node internal order in cluster. */
+ long maxInternalOrder;
+
+ /** Cluster start time (recorded when first node in cluster starts). */
+ final long clusterStartTime;
+
+ /** Events to process. */
+ final TreeMap<Long, ZkDiscoveryEventData> evts;
+
+ /** ID of current active communication error resolve process. */
+ private UUID commErrFutId;
+
+ /**
+ * @param clusterStartTime Start time of first node in cluster.
+ * @return Events.
+ */
+ static ZkDiscoveryEventsData createForNewCluster(long clusterStartTime) {
+ return new ZkDiscoveryEventsData(
+ UUID.randomUUID(),
+ clusterStartTime,
+ 1L,
+ new TreeMap<Long, ZkDiscoveryEventData>()
+ );
+ }
+
+ /**
+ * @param clusterId Cluster ID.
+ * @param topVer Current topology version.
+ * @param clusterStartTime Cluster start time.
+ * @param evts Events history.
+ */
+ private ZkDiscoveryEventsData(
+ UUID clusterId,
+ long clusterStartTime,
+ long topVer,
+ TreeMap<Long, ZkDiscoveryEventData> evts)
+ {
+ this.clusterId = clusterId;
+ this.clusterStartTime = clusterStartTime;
+ this.topVer = topVer;
+ this.evts = evts;
+ }
+
+ /**
+ * @param node Joined node.
+ */
+ void onNodeJoin(ZookeeperClusterNode node) {
+ if (node.internalId() > maxInternalOrder)
+ maxInternalOrder = node.internalId();
+ }
+
+ /**
+ * @return Future ID.
+ */
+ @Nullable UUID communicationErrorResolveFutureId() {
+ return commErrFutId;
+ }
+
+ /**
+ * @param id Future ID.
+ */
+ void communicationErrorResolveFutureId(@Nullable UUID id) {
+ commErrFutId = id;
+ }
+
+ /**
+ * @param nodes Current nodes in topology (these nodes should ack that event processed).
+ * @param evt Event.
+ */
+ void addEvent(Collection<ZookeeperClusterNode> nodes, ZkDiscoveryEventData evt) {
+ Object old = evts.put(evt.eventId(), evt);
+
+ assert old == null : old;
+
+ evt.initRemainingAcks(nodes);
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryNodeFailEventData.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryNodeFailEventData.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryNodeFailEventData.java
new file mode 100644
index 0000000..c76158f
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryNodeFailEventData.java
@@ -0,0 +1,55 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+/**
+ *
+ */
+class ZkDiscoveryNodeFailEventData extends ZkDiscoveryEventData {
+ /** */
+ private static final long serialVersionUID = 0L;
+
+ /** */
+ private long failedNodeInternalId;
+
+ /**
+ * @param evtId Event ID.
+ * @param topVer Topology version.
+ * @param failedNodeInternalId Failed node ID.
+ */
+ ZkDiscoveryNodeFailEventData(long evtId, long topVer, long failedNodeInternalId) {
+ super(evtId, ZK_EVT_NODE_FAILED, topVer);
+
+ this.failedNodeInternalId = failedNodeInternalId;
+ }
+
+ /**
+ * @return Failed node ID.
+ */
+ long failedNodeInternalId() {
+ return failedNodeInternalId;
+ }
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return "ZkDiscoveryNodeFailEventData [" +
+ "evtId=" + eventId() +
+ ", topVer=" + topologyVersion() +
+ ", nodeId=" + failedNodeInternalId + ']';
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryNodeJoinEventData.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryNodeJoinEventData.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryNodeJoinEventData.java
new file mode 100644
index 0000000..e46d52d
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryNodeJoinEventData.java
@@ -0,0 +1,60 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.util.List;
+
+/**
+ *
+ */
+class ZkDiscoveryNodeJoinEventData extends ZkDiscoveryEventData {
+ /** */
+ private static final long serialVersionUID = 0L;
+
+ /** */
+ final List<ZkJoinedNodeEvtData> joinedNodes;
+
+ /** */
+ final int dataForJoinedPartCnt;
+
+ /**
+ * @param evtId Event ID.
+ * @param topVer Topology version.
+ * @param joinedNodes Joined nodes data.
+ * @param dataForJoinedPartCnt Data for joined part count.
+ */
+ ZkDiscoveryNodeJoinEventData(
+ long evtId,
+ long topVer,
+ List<ZkJoinedNodeEvtData> joinedNodes,
+ int dataForJoinedPartCnt)
+ {
+ super(evtId, ZK_EVT_NODE_JOIN, topVer);
+
+ this.joinedNodes = joinedNodes;
+ this.dataForJoinedPartCnt = dataForJoinedPartCnt;
+ }
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return "ZkDiscoveryNodeJoinEventData [" +
+ "evtId=" + eventId() +
+ ", topVer=" + topologyVersion() +
+ ", nodes=" + joinedNodes + ']';
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDistributedCollectDataFuture.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDistributedCollectDataFuture.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDistributedCollectDataFuture.java
new file mode 100644
index 0000000..174d698
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDistributedCollectDataFuture.java
@@ -0,0 +1,250 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.zookeeper.AsyncCallback;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.data.Stat;
+
+/**
+ *
+ */
+class ZkDistributedCollectDataFuture extends GridFutureAdapter<Void> {
+ /** */
+ private final IgniteLogger log;
+
+ /** */
+ private final String futPath;
+
+ /** */
+ private final Set<Long> remainingNodes;
+
+ /** */
+ private final Callable<Void> lsnr;
+
+ /**
+ * @param impl Disovery impl
+ * @param rtState Runtime state.
+ * @param futPath Future path.
+ * @param lsnr Future listener.
+ * @throws Exception If listener call failed.
+ */
+ ZkDistributedCollectDataFuture(
+ ZookeeperDiscoveryImpl impl,
+ ZkRuntimeState rtState,
+ String futPath,
+ Callable<Void> lsnr)
+ throws Exception
+ {
+ this.log = impl.log();
+ this.futPath = futPath;
+ this.lsnr = lsnr;
+
+ ZkClusterNodes top = rtState.top;
+
+ // Assume new nodes can not join while future is in progress.
+
+ remainingNodes = U.newHashSet(top.nodesByOrder.size());
+
+ for (ZookeeperClusterNode node : top.nodesByInternalId.values())
+ remainingNodes.add(node.order());
+
+ NodeResultsWatcher watcher = new NodeResultsWatcher(rtState, impl);
+
+ if (remainingNodes.isEmpty())
+ completeAndNotifyListener();
+ else {
+ if (log.isInfoEnabled()) {
+ log.info("Initialize data collect future [futPath=" + futPath + ", " +
+ "remainingNodes=" + remainingNodes.size() + ']');
+ }
+
+ rtState.zkClient.getChildrenAsync(futPath, watcher, watcher);
+ }
+ }
+
+ /**
+ * @throws Exception If listener call failed.
+ */
+ private void completeAndNotifyListener() throws Exception {
+ if (super.onDone())
+ lsnr.call();
+ }
+
+ /**
+ * @param futPath
+ * @param client
+ * @param nodeOrder
+ * @param data
+ * @throws Exception If failed.
+ */
+ static void saveNodeResult(String futPath, ZookeeperClient client, long nodeOrder, byte[] data) throws Exception {
+ client.createIfNeeded(futPath + "/" + nodeOrder, data, CreateMode.PERSISTENT);
+ }
+
+ /**
+ * @param futPath
+ * @param client
+ * @param nodeOrder
+ * @return Node result data.
+ * @throws Exception If fai.ed
+ */
+ static byte[] readNodeResult(String futPath, ZookeeperClient client, long nodeOrder) throws Exception {
+ return client.getData(futPath + "/" + nodeOrder);
+ }
+
+ /**
+ * @param futResPath Result path.
+ * @param client Client.
+ * @param data Result data.
+ * @throws Exception If failed.
+ */
+ static void saveResult(String futResPath, ZookeeperClient client, byte[] data) throws Exception {
+ client.createIfNeeded(futResPath, data, CreateMode.PERSISTENT);
+ }
+
+ static byte[] readResult(ZookeeperClient client, ZkIgnitePaths paths, UUID futId) throws Exception {
+ return client.getData(paths.distributedFutureResultPath(futId));
+ }
+
+ /**
+ * @param client Client.
+ * @param paths Paths utils.
+ * @param futId Future ID.
+ * @param log Ignite Logger.
+ * @throws Exception If failed.
+ */
+ static void deleteFutureData(ZookeeperClient client,
+ ZkIgnitePaths paths,
+ UUID futId,
+ IgniteLogger log
+ ) throws Exception {
+ // TODO ZK: https://issues.apache.org/jira/browse/IGNITE-8189
+ String evtDir = paths.distributedFutureBasePath(futId);
+
+ try {
+ client.deleteAll(evtDir,
+ client.getChildren(evtDir),
+ -1);
+ }
+ catch (KeeperException.NoNodeException e) {
+ U.log(log, "Node for deletion was not found: " + e.getPath());
+
+ // TODO ZK: https://issues.apache.org/jira/browse/IGNITE-8189
+ }
+
+ client.deleteIfExists(evtDir, -1);
+
+ client.deleteIfExists(paths.distributedFutureResultPath(futId), -1);
+ }
+
+ /**
+ * @param top Current topology.
+ * @throws Exception If listener call failed.
+ */
+ void onTopologyChange(ZkClusterNodes top) throws Exception {
+ if (remainingNodes.isEmpty())
+ return;
+
+ for (Iterator<Long> it = remainingNodes.iterator(); it.hasNext();) {
+ Long nodeOrder = it.next();
+
+ if (!top.nodesByOrder.containsKey(nodeOrder)) {
+ it.remove();
+
+ int remaining = remainingNodes.size();
+
+ if (log.isInfoEnabled()) {
+ log.info("ZkDistributedCollectDataFuture removed remaining failed node [node=" + nodeOrder +
+ ", remaining=" + remaining +
+ ", futPath=" + futPath + ']');
+ }
+
+ if (remaining == 0) {
+ completeAndNotifyListener();
+
+ break;
+ }
+ }
+ }
+ }
+
+ /**
+ *
+ */
+ class NodeResultsWatcher extends ZkAbstractWatcher implements AsyncCallback.Children2Callback {
+ /**
+ * @param rtState Runtime state.
+ * @param impl Discovery impl.
+ */
+ NodeResultsWatcher(ZkRuntimeState rtState, ZookeeperDiscoveryImpl impl) {
+ super(rtState, impl);
+ }
+
+ /** {@inheritDoc} */
+ @Override protected void process0(WatchedEvent evt) {
+ if (evt.getType() == Watcher.Event.EventType.NodeChildrenChanged)
+ rtState.zkClient.getChildrenAsync(evt.getPath(), this, this);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void processResult(int rc, String path, Object ctx, List<String> children, Stat stat) {
+ if (!onProcessStart())
+ return;
+
+ try {
+ if (!isDone()) {
+ assert rc == 0 : KeeperException.Code.get(rc);
+
+ for (int i = 0; i < children.size(); i++) {
+ Long nodeOrder = Long.parseLong(children.get(i));
+
+ if (remainingNodes.remove(nodeOrder)) {
+ int remaining = remainingNodes.size();
+
+ if (log.isInfoEnabled()) {
+ log.info("ZkDistributedCollectDataFuture added new result [node=" + nodeOrder +
+ ", remaining=" + remaining +
+ ", futPath=" + path + ']');
+ }
+
+ if (remaining == 0)
+ completeAndNotifyListener();
+ }
+ }
+ }
+
+ onProcessEnd();
+ }
+ catch (Throwable e) {
+ onProcessError(e);
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkForceNodeFailMessage.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkForceNodeFailMessage.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkForceNodeFailMessage.java
new file mode 100644
index 0000000..de7291c
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkForceNodeFailMessage.java
@@ -0,0 +1,65 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Zk Force Node Fail Message.
+ */
+public class ZkForceNodeFailMessage implements DiscoverySpiCustomMessage, ZkInternalMessage {
+ /** */
+ private static final long serialVersionUID = 0L;
+
+ /** */
+ final long nodeInternalId;
+
+ /** */
+ final String warning;
+
+ /**
+ * @param nodeInternalId Node ID.
+ * @param warning Warning to be displayed on all nodes.
+ */
+ ZkForceNodeFailMessage(long nodeInternalId, String warning) {
+ this.nodeInternalId = nodeInternalId;
+ this.warning = warning;
+ }
+
+ /** {@inheritDoc} */
+ @Nullable @Override public DiscoverySpiCustomMessage ackMessage() {
+ return null;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean isMutable() {
+ return false;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean stopProcess() {
+ return false;
+ }
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return S.toString(ZkForceNodeFailMessage.class, this);
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkIgnitePaths.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkIgnitePaths.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkIgnitePaths.java
new file mode 100644
index 0000000..9caf00f
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkIgnitePaths.java
@@ -0,0 +1,307 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.util.UUID;
+
+/**
+ *
+ */
+class ZkIgnitePaths {
+ /** */
+ static final String PATH_SEPARATOR = "/";
+
+ /** */
+ private static final byte CLIENT_NODE_FLAG_MASK = 0x01;
+
+ /** */
+ private static final int UUID_LEN = 36;
+
+ /** Directory to store joined node data. */
+ private static final String JOIN_DATA_DIR = "jd";
+
+ /** Directory to store new custom events. */
+ private static final String CUSTOM_EVTS_DIR = "ce";
+
+ /** Directory to store parts of multi-parts custom events. */
+ private static final String CUSTOM_EVTS_PARTS_DIR = "cp";
+
+ /** Directory to store acknowledge messages for custom events. */
+ private static final String CUSTOM_EVTS_ACKS_DIR = "ca";
+
+ /** Directory to store EPHEMERAL znodes for alive cluster nodes. */
+ static final String ALIVE_NODES_DIR = "n";
+
+ /** Path to store discovery events {@link ZkDiscoveryEventsData}. */
+ private static final String DISCO_EVENTS_PATH = "e";
+
+ /** */
+ final String clusterDir;
+
+ /** */
+ final String aliveNodesDir;
+
+ /** */
+ final String joinDataDir;
+
+ /** */
+ final String evtsPath;
+
+ /** */
+ final String customEvtsDir;
+
+ /** */
+ final String customEvtsPartsDir;
+
+ /** */
+ final String customEvtsAcksDir;
+
+ /**
+ * @param zkRootPath Base Zookeeper directory for all Ignite nodes.
+ */
+ ZkIgnitePaths(String zkRootPath) {
+ clusterDir = zkRootPath;
+
+ aliveNodesDir = zkPath(ALIVE_NODES_DIR);
+ joinDataDir = zkPath(JOIN_DATA_DIR);
+ evtsPath = zkPath(DISCO_EVENTS_PATH);
+ customEvtsDir = zkPath(CUSTOM_EVTS_DIR);
+ customEvtsPartsDir = zkPath(CUSTOM_EVTS_PARTS_DIR);
+ customEvtsAcksDir = zkPath(CUSTOM_EVTS_ACKS_DIR);
+ }
+
+ /**
+ * @param path Relative path.
+ * @return Full path.
+ */
+ private String zkPath(String path) {
+ return clusterDir + "/" + path;
+ }
+
+ /**
+ * @param nodeId Node ID.
+ * @param prefixId Unique prefix ID.
+ * @return Path.
+ */
+ String joiningNodeDataPath(UUID nodeId, UUID prefixId) {
+ return joinDataDir + '/' + prefixId + ":" + nodeId.toString();
+ }
+
+ /**
+ * @param path Alive node zk path.
+ * @return Node internal ID.
+ */
+ static long aliveInternalId(String path) {
+ int idx = path.lastIndexOf('|');
+
+ return Integer.parseInt(path.substring(idx + 1));
+ }
+
+ /**
+ * @param prefix Node unique path prefix.
+ * @param node Node.
+ * @return Path.
+ */
+ String aliveNodePathForCreate(String prefix, ZookeeperClusterNode node) {
+ byte flags = 0;
+
+ if (node.isClient())
+ flags |= CLIENT_NODE_FLAG_MASK;
+
+ return aliveNodesDir + "/" + prefix + ":" + node.id() + ":" + encodeFlags(flags) + "|";
+ }
+
+ /**
+ * @param path Alive node zk path.
+ * @return {@code True} if node is client.
+ */
+ static boolean aliveNodeClientFlag(String path) {
+ return (aliveFlags(path) & CLIENT_NODE_FLAG_MASK) != 0;
+ }
+
+ /**
+ * @param path Alive node zk path.
+ * @return Node ID.
+ */
+ static UUID aliveNodePrefixId(String path) {
+ return UUID.fromString(path.substring(0, ZkIgnitePaths.UUID_LEN));
+ }
+
+ /**
+ * @param path Alive node zk path.
+ * @return Node ID.
+ */
+ static UUID aliveNodeId(String path) {
+ // <uuid prefix>:<node id>:<flags>|<alive seq>
+ int startIdx = ZkIgnitePaths.UUID_LEN + 1;
+
+ String idStr = path.substring(startIdx, startIdx + ZkIgnitePaths.UUID_LEN);
+
+ return UUID.fromString(idStr);
+ }
+
+ /**
+ * @param path Event zk path.
+ * @return Event sequence number.
+ */
+ static int customEventSequence(String path) {
+ int idx = path.lastIndexOf('|');
+
+ return Integer.parseInt(path.substring(idx + 1));
+ }
+
+ /**
+ * @param path Custom event zl path.
+ * @return Event node ID.
+ */
+ static UUID customEventSendNodeId(String path) {
+ // <uuid prefix>:<node id>:<partCnt>|<seq>
+ int startIdx = ZkIgnitePaths.UUID_LEN + 1;
+
+ String idStr = path.substring(startIdx, startIdx + ZkIgnitePaths.UUID_LEN);
+
+ return UUID.fromString(idStr);
+ }
+
+ /**
+ * @param path Event path.
+ * @return Event unique prefix.
+ */
+ static String customEventPrefix(String path) {
+ // <uuid prefix>:<node id>:<partCnt>|<seq>
+
+ return path.substring(0, ZkIgnitePaths.UUID_LEN);
+ }
+
+ /**
+ * @param path Custom event zl path.
+ * @return Event node ID.
+ */
+ static int customEventPartsCount(String path) {
+ // <uuid prefix>:<node id>:<partCnt>|<seq>
+ int startIdx = 2 * ZkIgnitePaths.UUID_LEN + 2;
+
+ String cntStr = path.substring(startIdx, startIdx + 4);
+
+ int partCnt = Integer.parseInt(cntStr);
+
+ assert partCnt >= 1 : partCnt;
+
+ return partCnt;
+ }
+
+ /**
+ * @param prefix Prefix.
+ * @param nodeId Node ID.
+ * @param partCnt Parts count.
+ * @return Path.
+ */
+ String createCustomEventPath(String prefix, UUID nodeId, int partCnt) {
+ return customEvtsDir + "/" + prefix + ":" + nodeId + ":" + String.format("%04d", partCnt) + '|';
+ }
+
+ /**
+ * @param prefix Prefix.
+ * @param nodeId Node ID.
+ * @return Path.
+ */
+ String customEventPartsBasePath(String prefix, UUID nodeId) {
+ return customEvtsPartsDir + "/" + prefix + ":" + nodeId + ":";
+ }
+
+ /**
+ * @param prefix Prefix.
+ * @param nodeId Node ID.
+ * @param part Part number.
+ * @return Path.
+ */
+ String customEventPartPath(String prefix, UUID nodeId, int part) {
+ return customEventPartsBasePath(prefix, nodeId) + String.format("%04d", part);
+ }
+
+ /**
+ * @param evtId Event ID.
+ * @return Event zk path.
+ */
+ String joinEventDataPathForJoined(long evtId) {
+ return evtsPath + "/fj-" + evtId;
+ }
+
+ /**
+ * @param topVer Event topology version.
+ * @return Event zk path.
+ */
+ String joinEventSecuritySubjectPath(long topVer) {
+ return evtsPath + "/s-" + topVer;
+ }
+
+ /**
+ * @param origEvtId ID of original custom event.
+ * @return Path for custom event ack.
+ */
+ String ackEventDataPath(long origEvtId) {
+ assert origEvtId != 0;
+
+ return customEvtsAcksDir + "/" + String.valueOf(origEvtId);
+ }
+
+ /**
+ * @param id Future ID.
+ * @return Future path.
+ */
+ String distributedFutureBasePath(UUID id) {
+ return evtsPath + "/f-" + id;
+ }
+
+ /**
+ * @param id Future ID.
+ * @return Future path.
+ */
+ String distributedFutureResultPath(UUID id) {
+ return evtsPath + "/fr-" + id;
+ }
+
+ /**
+ * @param flags Flags.
+ * @return Flags string.
+ */
+ private static String encodeFlags(byte flags) {
+ int intVal = flags + 128;
+
+ String str = Integer.toString(intVal, 16);
+
+ if (str.length() == 1)
+ str = '0' + str;
+
+ assert str.length() == 2 : str;
+
+ return str;
+ }
+
+ /**
+ * @param path Alive node zk path.
+ * @return Flags.
+ */
+ private static byte aliveFlags(String path) {
+ int startIdx = path.lastIndexOf(':') + 1;
+
+ String flagsStr = path.substring(startIdx, startIdx + 2);
+
+ return (byte)(Integer.parseInt(flagsStr, 16) - 128);
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkInternalJoinErrorMessage.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkInternalJoinErrorMessage.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkInternalJoinErrorMessage.java
new file mode 100644
index 0000000..a73312c
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkInternalJoinErrorMessage.java
@@ -0,0 +1,44 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+/**
+ *
+ */
+class ZkInternalJoinErrorMessage implements ZkInternalMessage {
+ /** */
+ private static final long serialVersionUID = 0L;
+
+ /** */
+ transient boolean notifyNode = true;
+
+ /** */
+ final long nodeInternalId;
+
+ /** */
+ final String err;
+
+ /**
+ * @param nodeInternalId Joining node internal ID.
+ * @param err Error message.
+ */
+ ZkInternalJoinErrorMessage(long nodeInternalId, String err) {
+ this.nodeInternalId = nodeInternalId;
+ this.err = err;
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkInternalMessage.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkInternalMessage.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkInternalMessage.java
new file mode 100644
index 0000000..c1d56f0
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkInternalMessage.java
@@ -0,0 +1,27 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.io.Serializable;
+
+/**
+ *
+ */
+interface ZkInternalMessage extends Serializable {
+ // No-op.
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkJoinEventDataForJoined.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkJoinEventDataForJoined.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkJoinEventDataForJoined.java
new file mode 100644
index 0000000..e4ae4ba0
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkJoinEventDataForJoined.java
@@ -0,0 +1,83 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ *
+ */
+class ZkJoinEventDataForJoined implements Serializable {
+ /** */
+ private static final long serialVersionUID = 0L;
+
+ /** */
+ private final List<ZookeeperClusterNode> top;
+
+ /** */
+ private final Map<Long, byte[]> discoData;
+
+ /** */
+ private final Map<Long, Long> dupDiscoData;
+
+ /**
+ * @param top Topology.
+ * @param discoData Discovery data.
+ */
+ ZkJoinEventDataForJoined(List<ZookeeperClusterNode> top, Map<Long, byte[]> discoData, @Nullable Map<Long, Long> dupDiscoData) {
+ assert top != null;
+ assert discoData != null && !discoData.isEmpty();
+
+ this.top = top;
+ this.discoData = discoData;
+ this.dupDiscoData = dupDiscoData;
+ }
+
+ byte[] discoveryDataForNode(long nodeOrder) {
+ assert discoData != null;
+
+ byte[] dataBytes = discoData.get(nodeOrder);
+
+ if (dataBytes != null)
+ return dataBytes;
+
+ assert dupDiscoData != null;
+
+ Long dupDataNode = dupDiscoData.get(nodeOrder);
+
+ assert dupDataNode != null;
+
+ dataBytes = discoData.get(dupDataNode);
+
+ assert dataBytes != null;
+
+ return dataBytes;
+ }
+
+ /**
+ * @return Current topology.
+ */
+ List<ZookeeperClusterNode> topology() {
+ assert top != null;
+
+ return top;
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkJoinedNodeEvtData.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkJoinedNodeEvtData.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkJoinedNodeEvtData.java
new file mode 100644
index 0000000..3c367cf
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkJoinedNodeEvtData.java
@@ -0,0 +1,79 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.io.Serializable;
+import java.util.UUID;
+
+/**
+ * Zk Joined Node Evt Data.
+ */
+public class ZkJoinedNodeEvtData implements Serializable {
+ /** */
+ private static final long serialVersionUID = 0L;
+
+ /** */
+ final long topVer;
+
+ /** */
+ final long joinedInternalId;
+
+ /** */
+ final UUID nodeId;
+
+ /** */
+ final int joinDataPartCnt;
+
+ /** */
+ final int secSubjPartCnt;
+
+ /** */
+ final UUID joinDataPrefixId;
+
+ /** */
+ transient ZkJoiningNodeData joiningNodeData;
+
+ /**
+ * @param topVer Topology version for node join event.
+ * @param nodeId Joined node ID.
+ * @param joinedInternalId Joined node internal ID.
+ * @param joinDataPrefixId Join data unique prefix.
+ * @param joinDataPartCnt Join data part count.
+ * @param secSubjPartCnt Security subject part count.
+ */
+ ZkJoinedNodeEvtData(
+ long topVer,
+ UUID nodeId,
+ long joinedInternalId,
+ UUID joinDataPrefixId,
+ int joinDataPartCnt,
+ int secSubjPartCnt)
+ {
+ this.topVer = topVer;
+ this.nodeId = nodeId;
+ this.joinedInternalId = joinedInternalId;
+ this.joinDataPrefixId = joinDataPrefixId;
+ this.joinDataPartCnt = joinDataPartCnt;
+ this.secSubjPartCnt = secSubjPartCnt;
+ }
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return "ZkJoinedNodeData [id=" + nodeId + ", order=" + topVer + ']';
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkJoiningNodeData.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkJoiningNodeData.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkJoiningNodeData.java
new file mode 100644
index 0000000..ff8311d
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkJoiningNodeData.java
@@ -0,0 +1,87 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.io.Serializable;
+import java.util.Map;
+import org.apache.ignite.internal.util.tostring.GridToStringInclude;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ *
+ */
+class ZkJoiningNodeData implements Serializable {
+ /** */
+ private static final long serialVersionUID = 0L;
+
+ /** */
+ private int partCnt;
+
+ /** */
+ @GridToStringInclude
+ private ZookeeperClusterNode node;
+
+ /** */
+ @GridToStringInclude
+ private Map<Integer, Serializable> discoData;
+
+ /**
+ * @param partCnt Number of parts in multi-parts message.
+ */
+ ZkJoiningNodeData(int partCnt) {
+ this.partCnt = partCnt;
+ }
+
+ /**
+ * @param node Node.
+ * @param discoData Discovery data.
+ */
+ ZkJoiningNodeData(ZookeeperClusterNode node, Map<Integer, Serializable> discoData) {
+ assert node != null && node.id() != null : node;
+ assert discoData != null;
+
+ this.node = node;
+ this.discoData = discoData;
+ }
+
+ /**
+ * @return Number of parts in multi-parts message.
+ */
+ int partCount() {
+ return partCnt;
+ }
+
+ /**
+ * @return Node.
+ */
+ ZookeeperClusterNode node() {
+ return node;
+ }
+
+ /**
+ * @return Discovery data.
+ */
+ Map<Integer, Serializable> discoveryData() {
+ return discoData;
+ }
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return S.toString(ZkJoiningNodeData.class, this);
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkNoServersMessage.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkNoServersMessage.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkNoServersMessage.java
new file mode 100644
index 0000000..626fe74
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkNoServersMessage.java
@@ -0,0 +1,50 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ *
+ */
+class ZkNoServersMessage implements DiscoverySpiCustomMessage, ZkInternalMessage {
+ /** */
+ private static final long serialVersionUID = 0L;
+
+ /** {@inheritDoc} */
+ @Nullable @Override public DiscoverySpiCustomMessage ackMessage() {
+ return null;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean isMutable() {
+ return false;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean stopProcess() {
+ return false;
+ }
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return S.toString(ZkNoServersMessage.class, this);
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkNodeValidateResult.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkNodeValidateResult.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkNodeValidateResult.java
new file mode 100644
index 0000000..2abfee3
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkNodeValidateResult.java
@@ -0,0 +1,43 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+/**
+ *
+ */
+class ZkNodeValidateResult {
+ /** */
+ String err;
+
+ /** */
+ byte[] secSubjZipBytes;
+
+ /**
+ * @param err Error.
+ */
+ ZkNodeValidateResult(String err) {
+ this.err = err;
+ }
+
+ /**
+ * @param secSubjZipBytes Marshalled security subject.
+ */
+ ZkNodeValidateResult(byte[] secSubjZipBytes) {
+ this.secSubjZipBytes = secSubjZipBytes;
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkRunnable.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkRunnable.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkRunnable.java
new file mode 100644
index 0000000..965bdc0
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkRunnable.java
@@ -0,0 +1,51 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+/**
+ * Zk Runnable.
+ */
+public abstract class ZkRunnable extends ZkAbstractCallabck implements Runnable {
+ /**
+ * @param rtState Runtime state.
+ * @param impl Discovery impl.
+ */
+ ZkRunnable(ZkRuntimeState rtState, ZookeeperDiscoveryImpl impl) {
+ super(rtState, impl);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void run() {
+ if (!onProcessStart())
+ return;
+
+ try {
+ run0();
+
+ onProcessEnd();
+ }
+ catch (Throwable e) {
+ onProcessError(e);
+ }
+ }
+
+ /**
+ *
+ */
+ protected abstract void run0() throws Exception;
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkRuntimeState.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkRuntimeState.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkRuntimeState.java
new file mode 100644
index 0000000..cb04ac3
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkRuntimeState.java
@@ -0,0 +1,135 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.util.List;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.spi.IgniteSpiTimeoutObject;
+import org.apache.zookeeper.AsyncCallback;
+import org.apache.zookeeper.Watcher;
+
+/**
+ *
+ */
+class ZkRuntimeState {
+ /** */
+ ZkWatcher watcher;
+
+ /** */
+ ZkAliveNodeDataWatcher aliveNodeDataWatcher;
+
+ /** */
+ volatile Exception errForClose;
+
+ /** */
+ final boolean prevJoined;
+
+ /** */
+ ZookeeperClient zkClient;
+
+ /** */
+ long internalOrder;
+
+ /** */
+ int joinDataPartCnt;
+
+ /** */
+ long gridStartTime;
+
+ /** */
+ volatile boolean joined;
+
+ /** */
+ ZkDiscoveryEventsData evtsData;
+
+ /** */
+ boolean crd;
+
+ /** */
+ String locNodeZkPath;
+
+ /** */
+ final ZkAliveNodeData locNodeInfo = new ZkAliveNodeData();
+
+ /** */
+ int procEvtCnt;
+
+ /** */
+ final ZkClusterNodes top = new ZkClusterNodes();
+
+ /** */
+ List<ClusterNode> commErrProcNodes;
+
+ /** Timeout callback registering watcher for join error
+ * (set this watcher after timeout as a minor optimization).
+ */
+ ZkTimeoutObject joinErrTo;
+
+ /** Timeout callback set to wait for join timeout. */
+ ZkTimeoutObject joinTo;
+
+ /** Timeout callback to update processed events counter. */
+ ZkTimeoutObject procEvtsUpdateTo;
+
+ /** */
+ boolean updateAlives;
+
+ /**
+ * @param prevJoined {@code True} if joined topology before reconnect attempt.
+ */
+ ZkRuntimeState(boolean prevJoined) {
+ this.prevJoined = prevJoined;
+ }
+
+ /**
+ * @param watcher Watcher.
+ * @param aliveNodeDataWatcher Alive nodes data watcher.
+ */
+ void init(ZkWatcher watcher, ZkAliveNodeDataWatcher aliveNodeDataWatcher) {
+ this.watcher = watcher;
+ this.aliveNodeDataWatcher = aliveNodeDataWatcher;
+ }
+
+ /**
+ * @param err Error.
+ */
+ void onCloseStart(Exception err) {
+ assert err != null;
+
+ errForClose = err;
+
+ ZookeeperClient zkClient = this.zkClient;
+
+ if (zkClient != null)
+ zkClient.onCloseStart();
+ }
+
+ /**
+ *
+ */
+ interface ZkWatcher extends Watcher, AsyncCallback.Children2Callback, AsyncCallback.DataCallback {
+ // No-op.
+ }
+
+ /**
+ *
+ */
+ interface ZkAliveNodeDataWatcher extends Watcher, AsyncCallback.DataCallback {
+ // No-op.
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkTimeoutObject.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkTimeoutObject.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkTimeoutObject.java
new file mode 100644
index 0000000..4d3d5b4
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkTimeoutObject.java
@@ -0,0 +1,54 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.spi.IgniteSpiTimeoutObject;
+
+/**
+ *
+ */
+abstract class ZkTimeoutObject implements IgniteSpiTimeoutObject {
+ /** */
+ private final IgniteUuid id = IgniteUuid.randomUuid();
+
+ /** */
+ private final long endTime;
+
+ /** */
+ volatile boolean cancelled;
+
+ /**
+ * @param timeout Timeout.
+ */
+ ZkTimeoutObject(long timeout) {
+ long endTime = timeout >= 0 ? System.currentTimeMillis() + timeout : Long.MAX_VALUE;
+
+ this.endTime = endTime >= 0 ? endTime : Long.MAX_VALUE;
+ }
+
+ /** {@inheritDoc} */
+ @Override public final IgniteUuid id() {
+ return id;
+ }
+
+ /** {@inheritDoc} */
+ @Override public final long endTime() {
+ return endTime;
+ }
+}
[07/12] ignite git commit: IGNITE-7222 Added ZooKeeper discovery SPI
Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java
index 54b3a78..e89a4c8 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java
@@ -17,11 +17,21 @@
package org.apache.ignite.spi.communication.tcp;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.List;
+import java.util.Map;
import java.util.UUID;
+import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CyclicBarrier;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.IgniteInternalFuture;
import org.apache.ignite.internal.util.nio.GridCommunicationClient;
import org.apache.ignite.internal.util.typedef.F;
import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.plugin.extensions.communication.Message;
import org.apache.ignite.spi.IgniteSpiAdapter;
import org.apache.ignite.spi.communication.CommunicationSpi;
import org.apache.ignite.spi.communication.GridAbstractCommunicationSelfTest;
@@ -85,6 +95,67 @@ abstract class GridTcpCommunicationSpiAbstractTest extends GridAbstractCommunica
}
}
+ /**
+ *
+ */
+ public void testCheckConnection1() {
+ for (int i = 0; i < 100; i++) {
+ for (Map.Entry<UUID, CommunicationSpi<Message>> entry : spis.entrySet()) {
+ TcpCommunicationSpi spi = (TcpCommunicationSpi)entry.getValue();
+
+ List<ClusterNode> checkNodes = new ArrayList<>(nodes);
+
+ assert checkNodes.size() > 1;
+
+ IgniteFuture<BitSet> fut = spi.checkConnection(checkNodes);
+
+ BitSet res = fut.get();
+
+ for (int n = 0; n < checkNodes.size(); n++)
+ assertTrue(res.get(n));
+ }
+ }
+ }
+
+ /**
+ * @throws Exception If failed.
+ */
+ public void testCheckConnection2() throws Exception {
+ final int THREADS = spis.size();
+
+ final CyclicBarrier b = new CyclicBarrier(THREADS);
+
+ List<IgniteInternalFuture> futs = new ArrayList<>();
+
+ for (Map.Entry<UUID, CommunicationSpi<Message>> entry : spis.entrySet()) {
+ final TcpCommunicationSpi spi = (TcpCommunicationSpi)entry.getValue();
+
+ futs.add(GridTestUtils.runAsync(new Callable() {
+ @Override public Object call() throws Exception {
+ List<ClusterNode> checkNodes = new ArrayList<>(nodes);
+
+ assert checkNodes.size() > 1;
+
+ b.await();
+
+ for (int i = 0; i < 100; i++) {
+ IgniteFuture<BitSet> fut = spi.checkConnection(checkNodes);
+
+ BitSet res = fut.get();
+
+ for (int n = 0; n < checkNodes.size(); n++)
+ assertTrue(res.get(n));
+ }
+
+ return null;
+ }
+ }));
+ }
+
+ for (IgniteInternalFuture f : futs)
+ f.get();
+ }
+
/** {@inheritDoc} */
@Override protected void afterTest() throws Exception {
super.afterTest();
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/spi/discovery/FilterDataForClientNodeDiscoveryTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/FilterDataForClientNodeDiscoveryTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/FilterDataForClientNodeDiscoveryTest.java
index 54b48e5..9a45d2d 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/FilterDataForClientNodeDiscoveryTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/FilterDataForClientNodeDiscoveryTest.java
@@ -206,6 +206,11 @@ public class FilterDataForClientNodeDiscoveryTest extends GridCommonAbstractTest
}
/** {@inheritDoc} */
+ @Override public boolean stopProcess() {
+ return false;
+ }
+
+ /** {@inheritDoc} */
@Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr, AffinityTopologyVersion topVer,
DiscoCache discoCache) {
return null;
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java b/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java
index ca05288..51dcb23 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java
@@ -613,6 +613,16 @@ public class GridSpiTestContext implements IgniteSpiContext {
return Collections.emptyMap();
}
+ /** {@inheritDoc} */
+ @Override public boolean communicationFailureResolveSupported() {
+ return false;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void resolveCommunicationFailure(ClusterNode node, Exception err) {
+ throw new UnsupportedOperationException();
+ }
+
/**
* @param cacheName Cache name.
* @return Map representing cache.
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/testframework/config/GridTestProperties.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/config/GridTestProperties.java b/modules/core/src/test/java/org/apache/ignite/testframework/config/GridTestProperties.java
index 4507572..e2594ca 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/config/GridTestProperties.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/config/GridTestProperties.java
@@ -83,6 +83,15 @@ public final class GridTestProperties {
/** "True value" enables {@link BinaryBasicNameMapper} in {@link BinaryTypeConfiguration#getNameMapper()} */
public static final String BINARY_MARSHALLER_USE_SIMPLE_NAME_MAPPER = "binary.marshaller.use.simple.name.mapper";
+ /**
+ * Name of class which provides static method preprocessConfiguration(IgniteConfiguration cfg) to
+ * alter {@link org.apache.ignite.configuration.IgniteConfiguration} before node is started.
+ * <p>
+ * Note: this pre-preprocessor is started only if test starts node using one of GridAbstractTest's startGrid
+ * method.
+ */
+ public static final String IGNITE_CFG_PREPROCESSOR_CLS = "ignite.cfg.preprocessor.class";
+
/** */
static {
// Initialize IGNITE_HOME system property.
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
index c3b262c..f5784eb 100755
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
@@ -46,6 +46,7 @@ import junit.framework.TestCase;
import org.apache.ignite.Ignite;
import org.apache.ignite.IgniteCache;
import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteClientDisconnectedException;
import org.apache.ignite.IgniteException;
import org.apache.ignite.IgniteLogger;
import org.apache.ignite.IgniteSystemProperties;
@@ -84,6 +85,7 @@ import org.apache.ignite.internal.util.typedef.internal.U;
import org.apache.ignite.lang.IgniteBiTuple;
import org.apache.ignite.lang.IgniteCallable;
import org.apache.ignite.lang.IgniteClosure;
+import org.apache.ignite.lang.IgniteFuture;
import org.apache.ignite.logger.NullLogger;
import org.apache.ignite.marshaller.Marshaller;
import org.apache.ignite.marshaller.MarshallerContextTestImpl;
@@ -92,6 +94,7 @@ import org.apache.ignite.marshaller.jdk.JdkMarshaller;
import org.apache.ignite.resources.IgniteInstanceResource;
import org.apache.ignite.spi.checkpoint.sharedfs.SharedFsCheckpointSpi;
import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+import org.apache.ignite.spi.discovery.DiscoverySpi;
import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
import org.apache.ignite.spi.discovery.tcp.TestTcpDiscoverySpi;
@@ -124,6 +127,7 @@ import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
import static org.apache.ignite.internal.GridKernalState.DISCONNECTED;
import static org.apache.ignite.testframework.config.GridTestProperties.BINARY_MARSHALLER_USE_SIMPLE_NAME_MAPPER;
+import static org.apache.ignite.testframework.config.GridTestProperties.IGNITE_CFG_PREPROCESSOR_CLS;
/**
* Common abstract test for Ignite tests.
@@ -203,13 +207,15 @@ public abstract class GridAbstractTest extends TestCase {
if (BINARY_MARSHALLER)
GridTestProperties.setProperty(GridTestProperties.MARSH_CLASS_NAME, BinaryMarshaller.class.getName());
- Thread timer = new Thread(new GridTestClockTimer(), "ignite-clock-for-tests");
+ if (GridTestClockTimer.startTestTimer()) {
+ Thread timer = new Thread(new GridTestClockTimer(), "ignite-clock-for-tests");
- timer.setDaemon(true);
+ timer.setDaemon(true);
- timer.setPriority(10);
+ timer.setPriority(10);
- timer.start();
+ timer.start();
+ }
}
/** */
@@ -838,6 +844,7 @@ public abstract class GridAbstractTest extends TestCase {
protected Ignite startGrid(String igniteInstanceName, GridSpringResourceContext ctx) throws Exception {
return startGrid(igniteInstanceName, optimize(getConfiguration(igniteInstanceName)), ctx);
}
+
/**
* Starts new grid with given name.
*
@@ -852,12 +859,33 @@ public abstract class GridAbstractTest extends TestCase {
startingIgniteInstanceName.set(igniteInstanceName);
try {
+ String cfgProcClsName = System.getProperty(IGNITE_CFG_PREPROCESSOR_CLS);
+
+ if (cfgProcClsName != null) {
+ try {
+ Class<?> cfgProc = Class.forName(cfgProcClsName);
+
+ Method method = cfgProc.getMethod("preprocessConfiguration", IgniteConfiguration.class);
+
+ if (!Modifier.isStatic(method.getModifiers()))
+ throw new Exception("Non-static pre-processor method in pre-processor class: " + cfgProcClsName);
+
+ method.invoke(null, cfg);
+ }
+ catch (Exception e) {
+ log.error("Failed to pre-process IgniteConfiguration using pre-processor class: " + cfgProcClsName);
+
+ throw new IgniteException(e);
+ }
+ }
+
Ignite node = IgnitionEx.start(cfg, ctx);
IgniteConfiguration nodeCfg = node.configuration();
log.info("Node started with the following configuration [id=" + node.cluster().localNode().id()
+ ", marshaller=" + nodeCfg.getMarshaller()
+ + ", discovery=" + nodeCfg.getDiscoverySpi()
+ ", binaryCfg=" + nodeCfg.getBinaryConfiguration()
+ ", lateAff=" + nodeCfg.isLateAffinityAssignment() + "]");
@@ -967,6 +995,26 @@ public abstract class GridAbstractTest extends TestCase {
if (cfg == null)
cfg = optimize(getConfiguration(igniteInstanceName));
+ if (locNode != null) {
+ DiscoverySpi discoverySpi = locNode.configuration().getDiscoverySpi();
+
+ if (discoverySpi != null && !(discoverySpi instanceof TcpDiscoverySpi)) {
+ try {
+ // Clone added to support ZookeeperDiscoverySpi.
+ Method m = discoverySpi.getClass().getDeclaredMethod("cloneSpiConfiguration");
+
+ m.setAccessible(true);
+
+ cfg.setDiscoverySpi((DiscoverySpi) m.invoke(discoverySpi));
+
+ resetDiscovery = false;
+ }
+ catch (NoSuchMethodException e) {
+ // Ignore.
+ }
+ }
+ }
+
return new IgniteProcessProxy(cfg, log, locNode, resetDiscovery);
}
@@ -1075,7 +1123,9 @@ public abstract class GridAbstractTest extends TestCase {
for (Ignite g : srvs)
stopGrid(g.name(), cancel, false);
- assert G.allGrids().isEmpty();
+ List<Ignite> nodes = G.allGrids();
+
+ assert nodes.isEmpty() : nodes;
}
finally {
IgniteProcessProxy.killAll(); // In multi-JVM case.
@@ -1177,6 +1227,14 @@ public abstract class GridAbstractTest extends TestCase {
}
/**
+ * @param nodeIdx Node index.
+ * @return Node ID.
+ */
+ protected final UUID nodeId(int nodeIdx) {
+ return ignite(nodeIdx).cluster().localNode().id();
+ }
+
+ /**
* Gets grid for given test.
*
* @return Grid for given test.
@@ -1217,7 +1275,11 @@ public abstract class GridAbstractTest extends TestCase {
* @throws Exception If failed.
*/
protected Ignite startGrid(String igniteInstanceName, String springCfgPath) throws Exception {
- return startGrid(igniteInstanceName, loadConfiguration(springCfgPath));
+ IgniteConfiguration cfg = loadConfiguration(springCfgPath);
+
+ cfg.setGridLogger(getTestResources().getLogger());
+
+ return startGrid(igniteInstanceName, cfg);
}
/**
@@ -2142,6 +2204,50 @@ public abstract class GridAbstractTest extends TestCase {
}
}
}
+ /**
+ * @param expSize Expected nodes number.
+ * @throws Exception If failed.
+ */
+ protected void waitForTopology(final int expSize) throws Exception {
+ assertTrue(GridTestUtils.waitForCondition(new GridAbsPredicate() {
+ @Override public boolean apply() {
+ List<Ignite> nodes = G.allGrids();
+
+ if (nodes.size() != expSize) {
+ info("Wait all nodes [size=" + nodes.size() + ", exp=" + expSize + ']');
+
+ return false;
+ }
+
+ for (Ignite node: nodes) {
+ try {
+ IgniteFuture<?> reconnectFut = node.cluster().clientReconnectFuture();
+
+ if (reconnectFut != null && !reconnectFut.isDone()) {
+ info("Wait for size on node, reconnect is in progress [node=" + node.name() + ']');
+
+ return false;
+ }
+
+ int sizeOnNode = node.cluster().nodes().size();
+
+ if (sizeOnNode != expSize) {
+ info("Wait for size on node [node=" + node.name() + ", size=" + sizeOnNode + ", exp=" + expSize + ']');
+
+ return false;
+ }
+ }
+ catch (IgniteClientDisconnectedException e) {
+ info("Wait for size on node, node disconnected [node=" + node.name() + ']');
+
+ return false;
+ }
+ }
+
+ return true;
+ }
+ }, 30_000));
+ }
/**
* @param millis Time to sleep.
@@ -2172,6 +2278,17 @@ public abstract class GridAbstractTest extends TestCase {
}
/**
+ * @return {@code True} if nodes use {@link TcpDiscoverySpi}.
+ */
+ protected static boolean tcpDiscovery() {
+ List<Ignite> nodes = G.allGrids();
+
+ assertFalse("There are no nodes", nodes.isEmpty());
+
+ return nodes.get(0).configuration().getDiscoverySpi() instanceof TcpDiscoverySpi;
+ }
+
+ /**
*
*/
private static interface WriteReplaceOwner {
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteNodeRunner.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteNodeRunner.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteNodeRunner.java
index d7be576..2b3a19c 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteNodeRunner.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteNodeRunner.java
@@ -162,6 +162,8 @@ public class IgniteNodeRunner {
cfg.setDiscoverySpi(disco);
}
+ X.println("Configured discovery: " + cfg.getDiscoverySpi().getClass().getName());
+
return cfg;
}
finally {
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java
index 55fab8d..14eb296 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java
@@ -19,6 +19,7 @@ package org.apache.ignite.testsuites;
import junit.framework.TestSuite;
import org.apache.ignite.internal.ClusterNodeMetricsSelfTest;
+import org.apache.ignite.internal.ClusterNodeMetricsUpdateTest;
import org.apache.ignite.internal.GridAffinityNoCacheSelfTest;
import org.apache.ignite.internal.GridAffinitySelfTest;
import org.apache.ignite.internal.GridAlwaysFailoverSpiFailSelfTest;
@@ -122,6 +123,7 @@ public class IgniteComputeGridTestSuite {
suite.addTestSuite(GridAlwaysFailoverSpiFailSelfTest.class);
suite.addTestSuite(GridTaskInstanceExecutionSelfTest.class);
suite.addTestSuite(ClusterNodeMetricsSelfTest.class);
+ suite.addTestSuite(ClusterNodeMetricsUpdateTest.class);
suite.addTestSuite(GridNonHistoryMetricsSelfTest.class);
suite.addTestSuite(GridCancelledJobsMetricsSelfTest.class);
suite.addTestSuite(GridCollisionJobsContextSelfTest.class);
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryCancelSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryCancelSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryCancelSelfTest.java
index b9ef1e4..e26b211 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryCancelSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryCancelSelfTest.java
@@ -117,7 +117,7 @@ public class IgniteCacheDistributedQueryCancelSelfTest extends GridCommonAbstrac
}
for (Ignite g : G.allGrids())
- if (!g.configuration().getDiscoverySpi().isClientMode())
+ if (!g.configuration().isClientMode())
stopGrid(g.name(), true);
}
}, 1);
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractBasicSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractBasicSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractBasicSelfTest.java
index 97720d5..bd3b093 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractBasicSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractBasicSelfTest.java
@@ -89,11 +89,14 @@ public abstract class DynamicIndexAbstractBasicSelfTest extends DynamicIndexAbst
* @param mode Mode.
* @param atomicityMode Atomicity mode.
* @param near Near flag.
+ * @throws Exception If failed.
*/
private void initialize(CacheMode mode, CacheAtomicityMode atomicityMode, boolean near)
- throws IgniteCheckedException {
+ throws Exception {
createSqlCache(node(), cacheConfiguration(mode, atomicityMode, near));
+ awaitPartitionMapExchange();
+
grid(IDX_CLI_NEAR_ONLY).getOrCreateNearCache(CACHE_NAME, new NearCacheConfiguration<>());
assertNoIndex(CACHE_NAME, TBL_NAME, IDX_NAME_1);
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridJtaTransactionManagerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridJtaTransactionManagerSelfTest.java b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridJtaTransactionManagerSelfTest.java
index a181068..5cad167 100644
--- a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridJtaTransactionManagerSelfTest.java
+++ b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridJtaTransactionManagerSelfTest.java
@@ -44,13 +44,7 @@ public class GridJtaTransactionManagerSelfTest extends GridCommonAbstractTest {
IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName).
setCacheConfiguration(defaultCacheConfiguration().setCacheMode(PARTITIONED));
- cfg.getTransactionConfiguration().setTxManagerFactory(new Factory<TransactionManager>() {
- private static final long serialVersionUID = 0L;
-
- @Override public TransactionManager create() {
- return jotm.getTransactionManager();
- }
- });
+ cfg.getTransactionConfiguration().setTxManagerFactory(new TestTxManagerFactory());
return cfg;
}
@@ -205,4 +199,17 @@ public class GridJtaTransactionManagerSelfTest extends GridCommonAbstractTest {
cache.removeAll();
}
}
+
+ /**
+ *
+ */
+ static class TestTxManagerFactory implements Factory<TransactionManager> {
+ /** */
+ private static final long serialVersionUID = 0L;
+
+ /** {@inheritDoc} */
+ @Override public TransactionManager create() {
+ return jotm.getTransactionManager();
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/GridPartitionedCacheJtaFactorySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/GridPartitionedCacheJtaFactorySelfTest.java b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/GridPartitionedCacheJtaFactorySelfTest.java
index f6fd5c7..14b7fae 100644
--- a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/GridPartitionedCacheJtaFactorySelfTest.java
+++ b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/GridPartitionedCacheJtaFactorySelfTest.java
@@ -30,12 +30,19 @@ public class GridPartitionedCacheJtaFactorySelfTest extends AbstractCacheJtaSelf
@Override protected void configureJta(IgniteConfiguration cfg) {
TransactionConfiguration txCfg = cfg.getTransactionConfiguration();
- txCfg.setTxManagerFactory(new Factory<TransactionManager>() {
- private static final long serialVersionUID = 0L;
+ txCfg.setTxManagerFactory(new TestTxManagerFactory());
+ }
+
+ /**
+ *
+ */
+ static class TestTxManagerFactory implements Factory<TransactionManager> {
+ /** */
+ private static final long serialVersionUID = 0L;
- @Override public TransactionManager create() {
- return jotm.getTransactionManager();
- }
- });
+ /** {@inheritDoc} */
+ @Override public TransactionManager create() {
+ return jotm.getTransactionManager();
+ }
}
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
index fce47a6..d87ea0a 100644
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
+++ b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
@@ -25,6 +25,7 @@ import org.apache.ignite.internal.processors.cache.query.QueryCursorEx
import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata
import org.apache.ignite.lang.IgniteUuid
import org.apache.ignite.spark.impl._
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi
import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode
import org.apache.spark._
import org.apache.spark.rdd.RDD
@@ -91,8 +92,14 @@ class IgniteRDD[K, V] (
override protected[spark] def getPreferredLocations(split: Partition): Seq[String] = {
ensureCache()
- ic.ignite().affinity(cacheName).mapPartitionToPrimaryAndBackups(split.index)
+ if (ic.ignite().configuration().getDiscoverySpi().isInstanceOf[TcpDiscoverySpi]) {
+ ic.ignite().affinity(cacheName).mapPartitionToPrimaryAndBackups(split.index)
.map(_.asInstanceOf[TcpDiscoveryNode].socketAddresses()).flatten.map(_.getHostName).toList
+ }
+ else {
+ ic.ignite().affinity(cacheName).mapPartitionToPrimaryAndBackups(split.index)
+ .flatten(_.hostNames).toSeq
+ }
}
/**
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/spring/src/test/java/org/apache/ignite/internal/GridFactorySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/internal/GridFactorySelfTest.java b/modules/spring/src/test/java/org/apache/ignite/internal/GridFactorySelfTest.java
index 75128fc..b453858 100644
--- a/modules/spring/src/test/java/org/apache/ignite/internal/GridFactorySelfTest.java
+++ b/modules/spring/src/test/java/org/apache/ignite/internal/GridFactorySelfTest.java
@@ -998,7 +998,8 @@ public class GridFactorySelfTest extends GridCommonAbstractTest {
startGrid("1", c);
- assert ((TcpDiscoverySpi)c.getDiscoverySpi()).started();
+ if (tcpDiscovery())
+ assert ((TcpDiscoverySpi)c.getDiscoverySpi()).started();
try {
startGrid("2", c);
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/spring/src/test/java/org/apache/ignite/p2p/GridP2PUserVersionChangeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/p2p/GridP2PUserVersionChangeSelfTest.java b/modules/spring/src/test/java/org/apache/ignite/p2p/GridP2PUserVersionChangeSelfTest.java
index b861e19..46da3cc 100644
--- a/modules/spring/src/test/java/org/apache/ignite/p2p/GridP2PUserVersionChangeSelfTest.java
+++ b/modules/spring/src/test/java/org/apache/ignite/p2p/GridP2PUserVersionChangeSelfTest.java
@@ -44,6 +44,7 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
import org.apache.ignite.testsuites.IgniteIgnore;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
import static org.apache.ignite.events.EventType.EVT_TASK_UNDEPLOYED;
@@ -255,12 +256,12 @@ public class GridP2PUserVersionChangeSelfTest extends GridCommonAbstractTest {
ignite2.events().localListen(new IgnitePredicate<Event>() {
@Override public boolean apply(Event evt) {
- if (evt.type() == EVT_NODE_LEFT)
+ if (evt.type() == EVT_NODE_LEFT || evt.type() == EVT_NODE_FAILED)
discoLatch.countDown();
return true;
}
- }, EVT_NODE_LEFT);
+ }, EVT_NODE_LEFT, EVT_NODE_FAILED);
Integer res1 = (Integer)ignite1.compute().execute(task1, ignite2.cluster().localNode().id());
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/yardstick/pom-standalone.xml
----------------------------------------------------------------------
diff --git a/modules/yardstick/pom-standalone.xml b/modules/yardstick/pom-standalone.xml
index 577a95e..6905d94 100644
--- a/modules/yardstick/pom-standalone.xml
+++ b/modules/yardstick/pom-standalone.xml
@@ -54,6 +54,12 @@
<dependency>
<groupId>org.apache.ignite</groupId>
+ <artifactId>ignite-zookeeper</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+
+ <dependency>
+ <groupId>org.apache.ignite</groupId>
<artifactId>ignite-log4j</artifactId>
<version>${project.version}</version>
</dependency>
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/yardstick/pom.xml
----------------------------------------------------------------------
diff --git a/modules/yardstick/pom.xml b/modules/yardstick/pom.xml
index 8cad24b..9923bb7 100644
--- a/modules/yardstick/pom.xml
+++ b/modules/yardstick/pom.xml
@@ -55,6 +55,12 @@
<dependency>
<groupId>org.apache.ignite</groupId>
+ <artifactId>ignite-zookeeper</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+
+ <dependency>
+ <groupId>org.apache.ignite</groupId>
<artifactId>ignite-log4j</artifactId>
<version>${project.version}</version>
</dependency>
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/pom.xml
----------------------------------------------------------------------
diff --git a/modules/zookeeper/pom.xml b/modules/zookeeper/pom.xml
index c3c3679..2d47ece 100644
--- a/modules/zookeeper/pom.xml
+++ b/modules/zookeeper/pom.xml
@@ -49,6 +49,12 @@
<dependency>
<groupId>org.apache.curator</groupId>
+ <artifactId>curator-recipes</artifactId>
+ <version>${curator.version}</version>
+ </dependency>
+
+ <dependency>
+ <groupId>org.apache.curator</groupId>
<artifactId>curator-x-discovery</artifactId>
<version>${curator.version}</version>
</dependency>
@@ -109,6 +115,13 @@
<dependency>
<groupId>org.apache.ignite</groupId>
+ <artifactId>ignite-indexing</artifactId>
+ <version>${project.version}</version>
+ <scope>test</scope>
+ </dependency>
+
+ <dependency>
+ <groupId>org.apache.ignite</groupId>
<artifactId>ignite-log4j</artifactId>
<version>${project.version}</version>
<scope>test</scope>
@@ -122,16 +135,43 @@
</dependency>
<dependency>
+ <groupId>com.thoughtworks.xstream</groupId>
+ <artifactId>xstream</artifactId>
+ <version>1.4.8</version>
+ <scope>test</scope>
+ </dependency>
+
+ <dependency>
<groupId>org.apache.ignite</groupId>
<artifactId>ignite-core</artifactId>
<version>${project.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>
+
+ <dependency>
+ <groupId>org.apache.ignite</groupId>
+ <artifactId>ignite-indexing</artifactId>
+ <version>${project.version}</version>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
</dependencies>
<build>
<plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-jar-plugin</artifactId>
+ <executions>
+ <execution>
+ <goals>
+ <goal>test-jar</goal>
+ </goals>
+ </execution>
+ </executions>
+ </plugin>
+
<!-- Generate the OSGi MANIFEST.MF for this bundle. -->
<plugin>
<groupId>org.apache.felix</groupId>
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpi.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpi.java
new file mode 100644
index 0000000..860c71c
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpi.java
@@ -0,0 +1,557 @@
+/*
+ * 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.spi.discovery.zk;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.net.InetAddress;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.curator.utils.PathUtils;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi;
+import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpiInternalListener;
+import org.apache.ignite.internal.util.tostring.GridToStringExclude;
+import org.apache.ignite.internal.util.tostring.GridToStringInclude;
+import org.apache.ignite.internal.util.typedef.internal.A;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.lang.IgniteProductVersion;
+import org.apache.ignite.resources.LoggerResource;
+import org.apache.ignite.spi.IgniteSpiAdapter;
+import org.apache.ignite.spi.IgniteSpiConfiguration;
+import org.apache.ignite.spi.IgniteSpiContext;
+import org.apache.ignite.spi.IgniteSpiException;
+import org.apache.ignite.spi.IgniteSpiMultipleInstancesSupport;
+import org.apache.ignite.spi.communication.CommunicationSpi;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+import org.apache.ignite.spi.discovery.DiscoveryMetricsProvider;
+import org.apache.ignite.spi.discovery.DiscoverySpi;
+import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
+import org.apache.ignite.spi.discovery.DiscoverySpiDataExchange;
+import org.apache.ignite.spi.discovery.DiscoverySpiHistorySupport;
+import org.apache.ignite.spi.discovery.DiscoverySpiListener;
+import org.apache.ignite.spi.discovery.DiscoverySpiMutableCustomMessageSupport;
+import org.apache.ignite.spi.discovery.DiscoverySpiNodeAuthenticator;
+import org.apache.ignite.spi.discovery.DiscoverySpiOrderSupport;
+import org.apache.ignite.spi.discovery.zk.internal.ZookeeperClusterNode;
+import org.apache.ignite.spi.discovery.zk.internal.ZookeeperDiscoveryImpl;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_CONSISTENT_ID_BY_HOST_WITHOUT_PORT;
+import static org.apache.ignite.IgniteSystemProperties.getBoolean;
+
+/**
+ * Zookeeper Discovery Spi.
+ */
+@IgniteSpiMultipleInstancesSupport(true)
+@DiscoverySpiOrderSupport(true)
+@DiscoverySpiHistorySupport(true)
+@DiscoverySpiMutableCustomMessageSupport(false)
+public class ZookeeperDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, IgniteDiscoverySpi {
+ /** */
+ public static final String DFLT_ROOT_PATH = "/apacheIgnite";
+
+ /** */
+ public static final long DFLT_JOIN_TIMEOUT = 0;
+
+ /** */
+ @GridToStringInclude
+ private String zkRootPath = DFLT_ROOT_PATH;
+
+ /** */
+ @GridToStringInclude
+ private String zkConnectionString;
+
+ /** */
+ private long joinTimeout = DFLT_JOIN_TIMEOUT;
+
+ /** */
+ @GridToStringInclude
+ private long sesTimeout;
+
+ /** */
+ private boolean clientReconnectDisabled;
+
+ /** */
+ @GridToStringExclude
+ private DiscoverySpiListener lsnr;
+
+ /** */
+ @GridToStringExclude
+ private DiscoverySpiDataExchange exchange;
+
+ /** */
+ @GridToStringExclude
+ private DiscoverySpiNodeAuthenticator nodeAuth;
+
+ /** */
+ @GridToStringExclude
+ private DiscoveryMetricsProvider metricsProvider;
+
+ /** */
+ @GridToStringExclude
+ private ZookeeperDiscoveryImpl impl;
+
+ /** */
+ @GridToStringExclude
+ private Map<String, Object> locNodeAttrs;
+
+ /** */
+ @GridToStringExclude
+ private IgniteProductVersion locNodeVer;
+
+ /** */
+ @GridToStringExclude
+ private Serializable consistentId;
+
+ /** Local node addresses. */
+ private IgniteBiTuple<Collection<String>, Collection<String>> addrs;
+
+ /** */
+ @LoggerResource
+ @GridToStringExclude
+ private IgniteLogger log;
+
+ /** */
+ private IgniteDiscoverySpiInternalListener internalLsnr;
+
+ /**
+ * @return Base path in ZK for znodes created by SPI.
+ */
+ public String getZkRootPath() {
+ return zkRootPath;
+ }
+
+ /**
+ * @param zkRootPath Base path in ZooKeeper for znodes created by SPI.
+ * @return {@code this} for chaining.
+ */
+ @IgniteSpiConfiguration(optional = true)
+ public ZookeeperDiscoverySpi setZkRootPath(String zkRootPath) {
+ this.zkRootPath = zkRootPath;
+
+ return this;
+ }
+
+ /**
+ * @return ZooKeeper session timeout.
+ */
+ public long getSessionTimeout() {
+ return sesTimeout;
+ }
+
+ /**
+ * @param sesTimeout ZooKeeper session timeout.
+ * @return {@code this} for chaining.
+ */
+ @IgniteSpiConfiguration(optional = true)
+ public ZookeeperDiscoverySpi setSessionTimeout(long sesTimeout) {
+ this.sesTimeout = sesTimeout;
+
+ return this;
+ }
+
+ /**
+ * @return Cluster join timeout.
+ */
+ public long getJoinTimeout() {
+ return joinTimeout;
+ }
+
+ /**
+ * @param joinTimeout Cluster join timeout ({@code 0} means wait forever).
+ * @return {@code this} for chaining.
+ */
+ @IgniteSpiConfiguration(optional = true)
+ public ZookeeperDiscoverySpi setJoinTimeout(long joinTimeout) {
+ this.joinTimeout = joinTimeout;
+
+ return this;
+ }
+
+ /**
+ * @return ZooKeeper connection string
+ */
+ public String getZkConnectionString() {
+ return zkConnectionString;
+ }
+
+ /**
+ * @param zkConnectionString ZooKeeper connection string
+ * @return {@code this} for chaining.
+ */
+ @IgniteSpiConfiguration(optional = false)
+ public ZookeeperDiscoverySpi setZkConnectionString(String zkConnectionString) {
+ this.zkConnectionString = zkConnectionString;
+
+ return this;
+ }
+
+ /**
+ * If {@code true} client does not try to reconnect.
+ *
+ * @return Client reconnect disabled flag.
+ */
+ public boolean isClientReconnectDisabled() {
+ return clientReconnectDisabled;
+ }
+
+ /**
+ * Sets client reconnect disabled flag.
+ *
+ * @param clientReconnectDisabled Client reconnect disabled flag.
+ * @return {@code this} for chaining.
+ */
+ @IgniteSpiConfiguration(optional = true)
+ public ZookeeperDiscoverySpi setClientReconnectDisabled(boolean clientReconnectDisabled) {
+ this.clientReconnectDisabled = clientReconnectDisabled;
+
+ return this;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean clientReconnectSupported() {
+ return !clientReconnectDisabled;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void clientReconnect() {
+ impl.reconnect();
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean knownNode(UUID nodeId) {
+ return impl.knownNode(nodeId);
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean supportsCommunicationFailureResolve() {
+ return true;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void resolveCommunicationFailure(ClusterNode node, Exception err) {
+ impl.resolveCommunicationError(node, err);
+ }
+
+ /** {@inheritDoc} */
+ @Nullable @Override public Serializable consistentId() throws IgniteSpiException {
+ if (consistentId == null) {
+ consistentId = ignite.configuration().getConsistentId();
+
+ if (consistentId == null) {
+ initAddresses();
+
+ final List<String> sortedAddrs = new ArrayList<>(addrs.get1());
+
+ Collections.sort(sortedAddrs);
+
+ if (getBoolean(IGNITE_CONSISTENT_ID_BY_HOST_WITHOUT_PORT))
+ consistentId = U.consistentId(sortedAddrs);
+ else {
+ Integer commPort = null;
+
+ if (locNodeAttrs != null) {
+ commPort = (Integer)locNodeAttrs.get(
+ TcpCommunicationSpi.class.getSimpleName() + "." + TcpCommunicationSpi.ATTR_PORT);
+ }
+ else {
+ CommunicationSpi commSpi = ignite.configuration().getCommunicationSpi();
+
+ if (commSpi instanceof TcpCommunicationSpi) {
+ commPort = ((TcpCommunicationSpi)commSpi).boundPort();
+
+ if (commPort == -1)
+ commPort = null;
+ }
+ }
+
+ if (commPort == null) {
+ U.warn(log, "Can not initialize default consistentId, TcpCommunicationSpi port is not initialized.");
+
+ consistentId = ignite.configuration().getNodeId();
+ }
+ else
+ consistentId = U.consistentId(sortedAddrs, commPort);
+ }
+ }
+ }
+
+ return consistentId;
+ }
+
+ /**
+ *
+ */
+ private void initAddresses() {
+ if (addrs == null) {
+ String locHost = ignite != null ? ignite.configuration().getLocalHost() : null;
+
+ InetAddress locAddr;
+
+ try {
+ locAddr = U.resolveLocalHost(locHost);
+ }
+ catch (IOException e) {
+ throw new IgniteSpiException("Unknown local address: " + locHost, e);
+ }
+
+ try {
+ addrs = U.resolveLocalAddresses(locAddr);
+ }
+ catch (Exception e) {
+ throw new IgniteSpiException("Failed to resolve local host to set of external addresses: " + locHost,
+ e);
+ }
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override public Collection<ClusterNode> getRemoteNodes() {
+ return impl.remoteNodes();
+ }
+
+ /** {@inheritDoc} */
+ @Override public ClusterNode getLocalNode() {
+ return impl != null ? impl.localNode() : null;
+ }
+
+ /** {@inheritDoc} */
+ @Nullable @Override public ClusterNode getNode(UUID nodeId) {
+ return impl.node(nodeId);
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean pingNode(UUID nodeId) {
+ return impl.pingNode(nodeId);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void setNodeAttributes(Map<String, Object> attrs, IgniteProductVersion ver) {
+ assert locNodeAttrs == null;
+ assert locNodeVer == null;
+
+ if (log.isDebugEnabled()) {
+ log.debug("Node attributes to set: " + attrs);
+ log.debug("Node version to set: " + ver);
+ }
+
+ locNodeAttrs = attrs;
+ locNodeVer = ver;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void setListener(@Nullable DiscoverySpiListener lsnr) {
+ this.lsnr = lsnr;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void setDataExchange(DiscoverySpiDataExchange exchange) {
+ this.exchange = exchange;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void setMetricsProvider(DiscoveryMetricsProvider metricsProvider) {
+ this.metricsProvider = metricsProvider;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void disconnect() throws IgniteSpiException {
+ impl.stop();
+ }
+
+ /** {@inheritDoc} */
+ @Override public void setAuthenticator(DiscoverySpiNodeAuthenticator auth) {
+ this.nodeAuth = auth;
+ }
+
+ /**
+ * @return Authenticator.
+ */
+ public DiscoverySpiNodeAuthenticator getAuthenticator() {
+ return nodeAuth;
+ }
+
+ /** {@inheritDoc} */
+ @Override public long getGridStartTime() {
+ return impl.gridStartTime();
+ }
+
+ /** {@inheritDoc} */
+ @Override public void sendCustomEvent(DiscoverySpiCustomMessage msg) {
+ IgniteDiscoverySpiInternalListener internalLsnr = impl.internalLsnr;
+
+ if (internalLsnr != null) {
+ if (!internalLsnr.beforeSendCustomEvent(this, log, msg))
+ return;
+ }
+
+ impl.sendCustomMessage(msg);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void failNode(UUID nodeId, @Nullable String warning) {
+ impl.failNode(nodeId, warning);
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean isClientMode() throws IllegalStateException {
+ return impl.localNode().isClient();
+ }
+
+ /** {@inheritDoc} */
+ @Override protected void onContextInitialized0(IgniteSpiContext spiCtx) throws IgniteSpiException {
+ super.onContextInitialized0(spiCtx);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void spiStart(@Nullable String igniteInstanceName) throws IgniteSpiException {
+ if (sesTimeout == 0)
+ sesTimeout = ignite.configuration().getFailureDetectionTimeout().intValue();
+
+ assertParameter(sesTimeout > 0, "sessionTimeout > 0");
+
+ A.notNullOrEmpty(zkConnectionString, "zkConnectionString can not be empty");
+
+ A.notNullOrEmpty(zkRootPath, "zkRootPath can not be empty");
+
+ zkRootPath = zkRootPath.trim();
+
+ if (zkRootPath.endsWith("/"))
+ zkRootPath = zkRootPath.substring(0, zkRootPath.length() - 1);
+
+ try {
+ PathUtils.validatePath(zkRootPath);
+ }
+ catch (IllegalArgumentException e) {
+ throw new IgniteSpiException("zkRootPath is invalid: " + zkRootPath, e);
+ }
+
+ ZookeeperClusterNode locNode = initLocalNode();
+
+ if (log.isInfoEnabled()) {
+ log.info("Start Zookeeper discovery [zkConnectionString=" + zkConnectionString +
+ ", sessionTimeout=" + sesTimeout +
+ ", zkRootPath=" + zkRootPath + ']');
+ }
+
+ impl = new ZookeeperDiscoveryImpl(
+ this,
+ igniteInstanceName,
+ log,
+ zkRootPath,
+ locNode,
+ lsnr,
+ exchange,
+ internalLsnr);
+
+ try {
+ impl.startJoinAndWait();
+ }
+ catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+
+ throw new IgniteSpiException("Failed to join cluster, thread was interrupted", e);
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override public void setInternalListener(IgniteDiscoverySpiInternalListener lsnr) {
+ if (impl != null)
+ impl.internalLsnr = lsnr;
+ else
+ internalLsnr = lsnr;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void simulateNodeFailure() {
+ impl.simulateNodeFailure();
+ }
+
+ /** {@inheritDoc} */
+ @Override public void spiStop() throws IgniteSpiException {
+ if (impl != null)
+ impl.stop();
+ }
+
+ /**
+ * @return Local node instance.
+ */
+ private ZookeeperClusterNode initLocalNode() {
+ assert ignite != null;
+
+ initAddresses();
+
+ ZookeeperClusterNode locNode = new ZookeeperClusterNode(
+ ignite.configuration().getNodeId(),
+ addrs.get1(),
+ addrs.get2(),
+ locNodeVer,
+ locNodeAttrs,
+ consistentId(),
+ sesTimeout,
+ ignite.configuration().isClientMode(),
+ metricsProvider);
+
+ locNode.local(true);
+
+ DiscoverySpiListener lsnr = this.lsnr;
+
+ if (lsnr != null)
+ lsnr.onLocalNodeInitialized(locNode);
+
+ if (log.isDebugEnabled())
+ log.debug("Local node initialized: " + locNode);
+
+ if (metricsProvider != null) {
+ locNode.setMetrics(metricsProvider.metrics());
+ locNode.setCacheMetrics(metricsProvider.cacheMetrics());
+ }
+
+ return locNode;
+ }
+
+ /**
+ * Used in tests (called via reflection).
+ *
+ * @return Copy of SPI.
+ */
+ private ZookeeperDiscoverySpi cloneSpiConfiguration() {
+ ZookeeperDiscoverySpi spi = new ZookeeperDiscoverySpi();
+
+ spi.setZkRootPath(zkRootPath);
+ spi.setZkConnectionString(zkConnectionString);
+ spi.setSessionTimeout(sesTimeout);
+ spi.setJoinTimeout(joinTimeout);
+ spi.setClientReconnectDisabled(clientReconnectDisabled);
+
+ return spi;
+ }
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return S.toString(ZookeeperDiscoverySpi.class, this);
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAbstractCallabck.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAbstractCallabck.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAbstractCallabck.java
new file mode 100644
index 0000000..b80a9dd
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAbstractCallabck.java
@@ -0,0 +1,83 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+
+/**
+ *
+ */
+abstract class ZkAbstractCallabck {
+ /** */
+ final ZkRuntimeState rtState;
+
+ /** */
+ private final ZookeeperDiscoveryImpl impl;
+
+ /** */
+ private final GridSpinBusyLock busyLock;
+
+ /**
+ * @param rtState Runtime state.
+ * @param impl Discovery impl.
+ */
+ ZkAbstractCallabck(ZkRuntimeState rtState, ZookeeperDiscoveryImpl impl) {
+ this.rtState = rtState;
+ this.impl = impl;
+
+ busyLock = impl.busyLock;
+ }
+
+ /**
+ * @return {@code True} if is able to start processing.
+ */
+ final boolean onProcessStart() {
+ boolean start = rtState.errForClose == null && busyLock.enterBusy();
+
+ if (!start) {
+ assert rtState.errForClose != null;
+
+ onStartFailed();
+
+ return false;
+ }
+
+ return true;
+ }
+
+ /**
+ *
+ */
+ void onStartFailed() {
+ // No-op.
+ }
+
+ /**
+ *
+ */
+ final void onProcessEnd() {
+ busyLock.leaveBusy();
+ }
+
+ /**
+ * @param e Error.
+ */
+ final void onProcessError(Throwable e) {
+ impl.onFatalError(busyLock, e);
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAbstractChildrenCallback.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAbstractChildrenCallback.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAbstractChildrenCallback.java
new file mode 100644
index 0000000..2292e35
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAbstractChildrenCallback.java
@@ -0,0 +1,61 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.util.List;
+import org.apache.zookeeper.AsyncCallback;
+import org.apache.zookeeper.data.Stat;
+
+/**
+ *
+ */
+abstract class ZkAbstractChildrenCallback extends ZkAbstractCallabck implements AsyncCallback.Children2Callback {
+ /**
+ * @param rtState Runtime state.
+ * @param impl Discovery impl.
+ */
+ ZkAbstractChildrenCallback(ZkRuntimeState rtState, ZookeeperDiscoveryImpl impl) {
+ super(rtState, impl);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void processResult(int rc, String path, Object ctx, List<String> children, Stat stat) {
+ if (!onProcessStart())
+ return;
+
+ try {
+ processResult0(rc, path, ctx, children, stat);
+
+ onProcessEnd();
+ }
+ catch (Throwable e) {
+ onProcessError(e);
+ }
+ }
+
+ /**
+ * @param rc
+ * @param path
+ * @param ctx
+ * @param children
+ * @param stat
+ * @throws Exception If failed.
+ */
+ abstract void processResult0(int rc, String path, Object ctx, List<String> children, Stat stat)
+ throws Exception;
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAbstractWatcher.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAbstractWatcher.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAbstractWatcher.java
new file mode 100644
index 0000000..9098d05
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAbstractWatcher.java
@@ -0,0 +1,55 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+
+/**
+ *
+ */
+abstract class ZkAbstractWatcher extends ZkAbstractCallabck implements Watcher {
+ /**
+ * @param rtState Runtime state.
+ * @param impl Discovery impl.
+ */
+ ZkAbstractWatcher(ZkRuntimeState rtState, ZookeeperDiscoveryImpl impl) {
+ super(rtState, impl);
+ }
+
+ /** {@inheritDoc} */
+ @Override public final void process(WatchedEvent evt) {
+ if (!onProcessStart())
+ return;
+
+ try {
+ process0(evt);
+
+ onProcessEnd();
+ }
+ catch (Throwable e) {
+ onProcessError(e);
+ }
+ }
+
+ /**
+ * @param evt Event.
+ * @throws Exception If failed.
+ */
+ protected abstract void process0(WatchedEvent evt) throws Exception;
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAliveNodeData.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAliveNodeData.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAliveNodeData.java
new file mode 100644
index 0000000..d824377
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAliveNodeData.java
@@ -0,0 +1,40 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.io.Serializable;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * Zk Alive Node Data.
+ */
+public class ZkAliveNodeData implements Serializable {
+ /** */
+ private static final long serialVersionUID = 0L;
+
+ /** */
+ long lastProcEvt = -1;
+
+ /** */
+ transient boolean needUpdate;
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return S.toString(ZkAliveNodeData.class, this);
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkBulkJoinContext.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkBulkJoinContext.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkBulkJoinContext.java
new file mode 100644
index 0000000..a186aed
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkBulkJoinContext.java
@@ -0,0 +1,50 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import org.apache.ignite.internal.util.typedef.T2;
+
+/**
+ *
+ */
+class ZkBulkJoinContext {
+ /** */
+ List<T2<ZkJoinedNodeEvtData, Map<Integer, Serializable>>> nodes;
+
+ /**
+ * @param nodeEvtData Node event data.
+ * @param discoData Discovery data for node.
+ */
+ void addJoinedNode(ZkJoinedNodeEvtData nodeEvtData, Map<Integer, Serializable> discoData) {
+ if (nodes == null)
+ nodes = new ArrayList<>();
+
+ nodes.add(new T2<>(nodeEvtData, discoData));
+ }
+
+ /**
+ * @return Number of joined nodes.
+ */
+ int nodes() {
+ return nodes != null ? nodes.size() : 0;
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkClusterNodes.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkClusterNodes.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkClusterNodes.java
new file mode 100644
index 0000000..7e2ea7b
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkClusterNodes.java
@@ -0,0 +1,103 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import org.apache.ignite.cluster.ClusterNode;
+
+/**
+ * Zk Cluster Nodes.
+ */
+public class ZkClusterNodes {
+ /** */
+ final ConcurrentSkipListMap<Long, ZookeeperClusterNode> nodesByOrder = new ConcurrentSkipListMap<>();
+
+ /** */
+ final ConcurrentSkipListMap<Long, ZookeeperClusterNode> nodesByInternalId = new ConcurrentSkipListMap<>();
+
+ /** */
+ final ConcurrentHashMap<UUID, ZookeeperClusterNode> nodesById = new ConcurrentHashMap<>();
+
+ /**
+ * @return Remote nodes.
+ */
+ public Collection<ClusterNode> remoteNodes() {
+ List<ClusterNode> nodes = new ArrayList<>();
+
+ for (ClusterNode node : nodesById.values()) {
+ if (!node.isLocal())
+ nodes.add(node);
+ }
+
+ return nodes;
+ }
+
+ /**
+ * @return Current nodes in topology.
+ */
+ @SuppressWarnings("unchecked")
+ List<ClusterNode> topologySnapshot() {
+ return new ArrayList<>((Collection)nodesByOrder.values());
+ }
+
+ /**
+ * @param node New node.
+ */
+ void addNode(ZookeeperClusterNode node) {
+ assert node.id() != null : node;
+ assert node.order() > 0 : node;
+
+ ZookeeperClusterNode old = nodesById.put(node.id(), node);
+
+ assert old == null : old;
+
+ old = nodesByOrder.put(node.order(), node);
+
+ assert old == null : old;
+
+ old = nodesByInternalId.put(node.internalId(), node);
+
+ assert old == null : old;
+ }
+
+ /**
+ * @param internalId Node internal ID.
+ * @return Removed node.
+ */
+ ZookeeperClusterNode removeNode(long internalId) {
+ ZookeeperClusterNode node = nodesByInternalId.remove(internalId);
+
+ assert node != null : internalId;
+ assert node.order() > 0 : node;
+
+ Object rvmd = nodesByOrder.remove(node.order());
+
+ assert rvmd != null;
+
+ rvmd = nodesById.remove(node.id());
+
+ assert rvmd != null;
+
+ return node;
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorNodeState.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorNodeState.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorNodeState.java
new file mode 100644
index 0000000..9c21f13
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorNodeState.java
@@ -0,0 +1,46 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.io.Serializable;
+import java.util.BitSet;
+
+/**
+ *
+ */
+class ZkCommunicationErrorNodeState implements Serializable {
+ /** */
+ private static final long serialVersionUID = 0L;
+
+ /** */
+ final BitSet commState;
+
+ /** */
+ final Exception err;
+
+ /**
+ * @param commState Communication state.
+ * @param err Error if failed get communication state..
+ */
+ ZkCommunicationErrorNodeState(BitSet commState, Exception err) {
+ assert commState != null || err != null;
+
+ this.commState = commState;
+ this.err = err;
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorProcessFuture.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorProcessFuture.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorProcessFuture.java
new file mode 100644
index 0000000..accda6e
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorProcessFuture.java
@@ -0,0 +1,411 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.util.BitSet;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.spi.IgniteSpiTimeoutObject;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+import org.jboss.netty.util.internal.ConcurrentHashMap;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Future is created on each node when either connection error occurs or resolve communication error request
+ * received.
+ */
+class ZkCommunicationErrorProcessFuture extends GridFutureAdapter<Void> implements IgniteSpiTimeoutObject, Runnable {
+ /** */
+ private final ZookeeperDiscoveryImpl impl;
+
+ /** */
+ private final IgniteLogger log;
+
+ /** */
+ private final Map<Long, GridFutureAdapter<Boolean>> nodeFuts = new ConcurrentHashMap<>();
+
+ /** */
+ private final long endTime;
+
+ /** */
+ private final IgniteUuid id;
+
+ /** */
+ private State state;
+
+ /** */
+ private long resolveTopVer;
+
+ /** */
+ private Set<Long> resFailedNodes;
+
+ /** */
+ private Exception resErr;
+
+ /** */
+ private ZkDistributedCollectDataFuture collectResFut;
+
+ /**
+ * @param impl Discovery impl.
+ * @param timeout Timeout to wait before initiating resolve process.
+ * @return Future.
+ */
+ static ZkCommunicationErrorProcessFuture createOnCommunicationError(ZookeeperDiscoveryImpl impl, long timeout) {
+ return new ZkCommunicationErrorProcessFuture(impl, State.WAIT_TIMEOUT, timeout);
+ }
+
+ /**
+ * @param impl Discovery impl.
+ * @return Future.
+ */
+ static ZkCommunicationErrorProcessFuture createOnStartResolveRequest(ZookeeperDiscoveryImpl impl) {
+ return new ZkCommunicationErrorProcessFuture(impl, State.RESOLVE_STARTED, 0);
+ }
+
+ /**
+ * @param impl Discovery implementation.
+ * @param state Initial state.
+ * @param timeout Wait timeout before initiating communication errors resolve.
+ */
+ private ZkCommunicationErrorProcessFuture(ZookeeperDiscoveryImpl impl, State state, long timeout) {
+ assert state != State.DONE;
+
+ this.impl = impl;
+ this.log = impl.log();
+
+ if (state == State.WAIT_TIMEOUT) {
+ assert timeout > 0 : timeout;
+
+ id = IgniteUuid.fromUuid(impl.localNode().id());
+ endTime = System.currentTimeMillis() + timeout;
+ }
+ else {
+ id = null;
+ endTime = 0;
+ }
+
+ this.state = state;
+ }
+
+ /** {@inheritDoc} */
+ @Nullable @Override public IgniteLogger logger() {
+ return log;
+ }
+
+ /**
+ * @param collectResFut Collect nodes' communication status future.
+ */
+ void nodeResultCollectFuture(ZkDistributedCollectDataFuture collectResFut) {
+ assert this.collectResFut == null : collectResFut;
+
+ this.collectResFut = collectResFut;
+ }
+
+ /**
+ * @param top Topology.
+ * @throws Exception If failed.
+ */
+ void onTopologyChange(ZkClusterNodes top) throws Exception {
+ for (Map.Entry<Long, GridFutureAdapter<Boolean>> e : nodeFuts.entrySet()) {
+ if (!top.nodesByOrder.containsKey(e.getKey()))
+ e.getValue().onDone(false);
+ }
+
+ if (collectResFut != null)
+ collectResFut.onTopologyChange(top);
+ }
+
+ /**
+ * @param rtState Runtime state.
+ * @param futPath Future path.
+ * @param nodes Nodes to ping.
+ */
+ void checkConnection(final ZkRuntimeState rtState, final String futPath, List<ClusterNode> nodes) {
+ final TcpCommunicationSpi spi = (TcpCommunicationSpi)impl.spi.ignite().configuration().getCommunicationSpi();
+
+ IgniteFuture<BitSet> fut = spi.checkConnection(nodes);
+
+ fut.listen(new IgniteInClosure<IgniteFuture<BitSet>>() {
+ @Override public void apply(final IgniteFuture<BitSet> fut) {
+ // Future completed either from NIO thread or timeout worker, save result from another thread.
+ impl.runInWorkerThread(new ZkRunnable(rtState, impl) {
+ @Override public void run0() throws Exception {
+ BitSet commState = null;
+ Exception err = null;
+
+ try {
+ commState = fut.get();
+ }
+ catch (Exception e) {
+ err = e;
+ }
+
+ ZkCommunicationErrorNodeState state = new ZkCommunicationErrorNodeState(commState, err);
+
+ ZkDistributedCollectDataFuture.saveNodeResult(futPath,
+ rtState.zkClient,
+ impl.localNode().order(),
+ impl.marshalZip(state));
+ }
+
+ @Override void onStartFailed() {
+ onError(rtState.errForClose);
+ }
+ });
+
+ }
+ });
+ }
+
+ /**
+ *
+ */
+ void scheduleCheckOnTimeout() {
+ synchronized (this) {
+ if (state == State.WAIT_TIMEOUT)
+ impl.spi.getSpiContext().addTimeoutObject(this);
+ }
+ }
+
+ /**
+ * @param topVer Topology version.
+ * @return {@code False} if future was already completed and need create another future instance.
+ */
+ boolean onStartResolveRequest(long topVer) {
+ synchronized (this) {
+ if (state == State.DONE)
+ return false;
+
+ if (state == State.WAIT_TIMEOUT)
+ impl.spi.getSpiContext().removeTimeoutObject(this);
+
+ assert resolveTopVer == 0 : resolveTopVer;
+
+ resolveTopVer = topVer;
+
+ state = State.RESOLVE_STARTED;
+ }
+
+ return true;
+ }
+
+ /**
+ * @param err Error.
+ */
+ void onError(Exception err) {
+ assert err != null;
+
+ Map<Long, GridFutureAdapter<Boolean>> futs;
+
+ synchronized (this) {
+ if (state == State.DONE) {
+ assert resErr != null;
+
+ return;
+ }
+
+ state = State.DONE;
+
+ resErr = err;
+
+ futs = nodeFuts; // nodeFuts should not be modified after state changed to DONE.
+ }
+
+ for (Map.Entry<Long, GridFutureAdapter<Boolean>> e : futs.entrySet())
+ e.getValue().onDone(err);
+
+ onDone(err);
+ }
+
+ /**
+ * @param failedNodes Node failed as result of resolve process.
+ */
+ void onFinishResolve(Set<Long> failedNodes) {
+ Map<Long, GridFutureAdapter<Boolean>> futs;
+
+ synchronized (this) {
+ if (state == State.DONE) {
+ assert resErr != null;
+
+ return;
+ }
+
+ assert state == State.RESOLVE_STARTED : state;
+
+ state = State.DONE;
+
+ resFailedNodes = failedNodes;
+
+ futs = nodeFuts; // nodeFuts should not be modified after state changed to DONE.
+ }
+
+ for (Map.Entry<Long, GridFutureAdapter<Boolean>> e : futs.entrySet()) {
+ Boolean res = !F.contains(resFailedNodes, e.getKey());
+
+ e.getValue().onDone(res);
+ }
+
+ onDone();
+ }
+
+ /**
+ * @param node Node.
+ * @return Future finished when communication error resolve is done or {@code null} if another
+ * resolve process should be started.
+ */
+ @Nullable IgniteInternalFuture<Boolean> nodeStatusFuture(ClusterNode node) {
+ GridFutureAdapter<Boolean> fut;
+
+ synchronized (this) {
+ if (state == State.DONE) {
+ if (resolveTopVer != 0 && node.order() <= resolveTopVer) {
+ Boolean res = !F.contains(resFailedNodes, node.order());
+
+ return new GridFinishedFuture<>(res);
+ }
+ else
+ return null;
+ }
+
+ fut = nodeFuts.get(node.order());
+
+ if (fut == null)
+ nodeFuts.put(node.order(), fut = new GridFutureAdapter<>());
+ }
+
+ if (impl.node(node.order()) == null)
+ fut.onDone(false);
+
+ return fut;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void run() {
+ // Run from zk discovery worker pool after timeout.
+ if (needProcessTimeout()) {
+ try {
+ UUID reqId = UUID.randomUUID();
+
+ if (log.isInfoEnabled()) {
+ log.info("Initiate cluster-wide communication error resolve process [reqId=" + reqId +
+ ", errNodes=" + nodeFuts.size() + ']');
+ }
+
+ impl.sendCustomMessage(new ZkCommunicationErrorResolveStartMessage(reqId));
+ }
+ catch (Exception e) {
+ Collection<GridFutureAdapter<Boolean>> futs;
+
+ synchronized (this) {
+ if (state != State.WAIT_TIMEOUT)
+ return;
+
+ state = State.DONE;
+ resErr = e;
+
+ futs = nodeFuts.values(); // nodeFuts should not be modified after state changed to DONE.
+ }
+
+ for (GridFutureAdapter<Boolean> fut : futs)
+ fut.onDone(e);
+
+ onDone(e);
+ }
+ }
+ }
+
+ /**
+ * @return {@code True} if need initiate resolve process after timeout expired.
+ */
+ private boolean needProcessTimeout() {
+ synchronized (this) {
+ if (state != State.WAIT_TIMEOUT)
+ return false;
+
+ for (GridFutureAdapter<Boolean> fut : nodeFuts.values()) {
+ if (!fut.isDone())
+ return true;
+ }
+
+ state = State.DONE;
+ }
+
+ onDone(null, null);
+
+ return false;
+ }
+
+ /** {@inheritDoc} */
+ @Override public IgniteUuid id() {
+ return id;
+ }
+
+ /** {@inheritDoc} */
+ @Override public long endTime() {
+ return endTime;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void onTimeout() {
+ if (needProcessTimeout())
+ impl.runInWorkerThread(this);
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean onDone(@Nullable Void res, @Nullable Throwable err) {
+ if (super.onDone(res, err)) {
+ impl.clearCommunicationErrorProcessFuture(this);
+
+ return true;
+ }
+
+ return false;
+ }
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return S.toString(ZkCommunicationErrorProcessFuture.class, this);
+ }
+
+ /**
+ *
+ */
+ enum State {
+ /** */
+ DONE,
+
+ /** */
+ WAIT_TIMEOUT,
+
+ /** */
+ RESOLVE_STARTED
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorResolveFinishMessage.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorResolveFinishMessage.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorResolveFinishMessage.java
new file mode 100644
index 0000000..9b7476c
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorResolveFinishMessage.java
@@ -0,0 +1,69 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.util.UUID;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ *
+ */
+class ZkCommunicationErrorResolveFinishMessage implements DiscoverySpiCustomMessage, ZkInternalMessage {
+ /** */
+ private static final long serialVersionUID = 0L;
+
+ /** */
+ final UUID futId;
+
+ /** */
+ final long topVer;
+
+ /** */
+ transient ZkCommunicationErrorResolveResult res;
+
+ /**
+ * @param futId Future ID.
+ * @param topVer Topology version when resolve process finished.
+ */
+ ZkCommunicationErrorResolveFinishMessage(UUID futId, long topVer) {
+ this.futId = futId;
+ this.topVer = topVer;
+ }
+
+ /** {@inheritDoc} */
+ @Nullable @Override public DiscoverySpiCustomMessage ackMessage() {
+ return null;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean isMutable() {
+ return false;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean stopProcess() {
+ return false;
+ }
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return S.toString(ZkCommunicationErrorResolveFinishMessage.class, this);
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorResolveResult.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorResolveResult.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorResolveResult.java
new file mode 100644
index 0000000..23495aa
--- /dev/null
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorResolveResult.java
@@ -0,0 +1,45 @@
+/*
+ * 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.spi.discovery.zk.internal;
+
+import java.io.Serializable;
+import org.apache.ignite.internal.util.GridLongList;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ *
+ */
+class ZkCommunicationErrorResolveResult implements Serializable {
+ /** */
+ private static final long serialVersionUID = 0L;
+
+ /** */
+ final GridLongList killedNodes;
+
+ /** */
+ final Exception err;
+
+ /**
+ * @param killedNodes Killed nodes.
+ * @param err Error.
+ */
+ ZkCommunicationErrorResolveResult(@Nullable GridLongList killedNodes, Exception err) {
+ this.killedNodes = killedNodes;
+ this.err = err;
+ }
+}
[10/12] ignite git commit: IGNITE-7222 Added ZooKeeper discovery SPI
Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
index 01a5a71..cebe4b1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
@@ -56,6 +56,8 @@ import org.apache.ignite.internal.managers.deployment.GridDeployment;
import org.apache.ignite.internal.managers.deployment.GridDeploymentInfo;
import org.apache.ignite.internal.managers.deployment.GridDeploymentInfoBean;
import org.apache.ignite.internal.managers.discovery.CustomEventListener;
+import org.apache.ignite.internal.managers.discovery.DiscoCache;
+import org.apache.ignite.internal.managers.discovery.DiscoveryMessageResultsCollector;
import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
import org.apache.ignite.internal.managers.eventstorage.HighPriorityListener;
import org.apache.ignite.internal.processors.GridProcessorAdapter;
@@ -63,6 +65,7 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
import org.apache.ignite.internal.processors.cache.GridCacheContext;
import org.apache.ignite.internal.processors.cache.GridCacheProcessor;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.CachePartitionPartialCountersMap;
import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryHandler;
import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
import org.apache.ignite.internal.util.future.GridFinishedFuture;
@@ -147,6 +150,12 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
/** Query sequence number for message topic. */
private final AtomicLong seq = new AtomicLong();
+ /** */
+ private ContinuousRoutinesInfo routinesInfo;
+
+ /** */
+ private int discoProtoVer;
+
/**
* @param ctx Kernal context.
*/
@@ -156,6 +165,11 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
/** {@inheritDoc} */
@Override public void start() throws IgniteCheckedException {
+ discoProtoVer = ctx.discovery().mutableCustomMessages() ? 1 : 2;
+
+ if (discoProtoVer == 2)
+ routinesInfo = new ContinuousRoutinesInfo();
+
if (ctx.config().isDaemon())
return;
@@ -177,6 +191,8 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
@Override public void onCustomEvent(AffinityTopologyVersion topVer,
ClusterNode snd,
StartRoutineDiscoveryMessage msg) {
+ assert discoProtoVer == 1 : discoProtoVer;
+
if (ctx.isStopping())
return;
@@ -184,6 +200,20 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
}
});
+ ctx.discovery().setCustomEventListener(StartRoutineDiscoveryMessageV2.class,
+ new CustomEventListener<StartRoutineDiscoveryMessageV2>() {
+ @Override public void onCustomEvent(AffinityTopologyVersion topVer,
+ ClusterNode snd,
+ StartRoutineDiscoveryMessageV2 msg) {
+ assert discoProtoVer == 2 : discoProtoVer;
+
+ if (ctx.isStopping())
+ return;
+
+ processStartRequestV2(topVer, snd, msg);
+ }
+ });
+
ctx.discovery().setCustomEventListener(StartRoutineAckDiscoveryMessage.class,
new CustomEventListener<StartRoutineAckDiscoveryMessage>() {
@Override public void onCustomEvent(AffinityTopologyVersion topVer,
@@ -201,6 +231,9 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
@Override public void onCustomEvent(AffinityTopologyVersion topVer,
ClusterNode snd,
StopRoutineDiscoveryMessage msg) {
+ if (discoProtoVer == 2)
+ routinesInfo.removeRoutine(msg.routineId);
+
if (ctx.isStopping())
return;
@@ -222,32 +255,36 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
ctx.io().addMessageListener(TOPIC_CONTINUOUS, new GridMessageListener() {
@Override public void onMessage(UUID nodeId, Object obj, byte plc) {
- GridContinuousMessage msg = (GridContinuousMessage)obj;
+ if (obj instanceof ContinuousRoutineStartResultMessage)
+ processRoutineStartResultMessage(nodeId, (ContinuousRoutineStartResultMessage)obj);
+ else {
+ GridContinuousMessage msg = (GridContinuousMessage)obj;
- if (msg.data() == null && msg.dataBytes() != null) {
- try {
- msg.data(U.unmarshal(marsh, msg.dataBytes(), U.resolveClassLoader(ctx.config())));
- }
- catch (IgniteCheckedException e) {
- U.error(log, "Failed to process message (ignoring): " + msg, e);
+ if (msg.data() == null && msg.dataBytes() != null) {
+ try {
+ msg.data(U.unmarshal(marsh, msg.dataBytes(), U.resolveClassLoader(ctx.config())));
+ }
+ catch (IgniteCheckedException e) {
+ U.error(log, "Failed to process message (ignoring): " + msg, e);
- return;
+ return;
+ }
}
- }
- switch (msg.type()) {
- case MSG_EVT_NOTIFICATION:
- processNotification(nodeId, msg);
+ switch (msg.type()) {
+ case MSG_EVT_NOTIFICATION:
+ processNotification(nodeId, msg);
- break;
+ break;
- case MSG_EVT_ACK:
- processMessageAck(msg);
+ case MSG_EVT_ACK:
+ processMessageAck(msg);
- break;
+ break;
- default:
- assert false : "Unexpected message received: " + msg.type();
+ default:
+ assert false : "Unexpected message received: " + msg.type();
+ }
}
}
});
@@ -341,6 +378,15 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
/** {@inheritDoc} */
@Override public void collectJoiningNodeData(DiscoveryDataBag dataBag) {
+ if (ctx.isDaemon())
+ return;
+
+ if (discoProtoVer == 2) {
+ routinesInfo.collectJoiningNodeData(dataBag);
+
+ return;
+ }
+
Serializable data = getDiscoveryData(dataBag.joiningNodeId());
if (data != null)
@@ -349,6 +395,15 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
/** {@inheritDoc} */
@Override public void collectGridNodeData(DiscoveryDataBag dataBag) {
+ if (ctx.isDaemon())
+ return;
+
+ if (discoProtoVer == 2) {
+ routinesInfo.collectGridNodeData(dataBag);
+
+ return;
+ }
+
Serializable data = getDiscoveryData(dataBag.joiningNodeId());
if (data != null)
@@ -393,6 +448,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
return data;
}
+
return null;
}
@@ -430,22 +486,118 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
@Override public void onJoiningNodeDataReceived(JoiningNodeDiscoveryData data) {
if (log.isDebugEnabled()) {
log.debug("onJoiningNodeDataReceived [joining=" + data.joiningNodeId() +
- ", loc=" + ctx.localNodeId() +
- ", data=" + data.joiningNodeData() +
- ']');
+ ", loc=" + ctx.localNodeId() +
+ ", data=" + data.joiningNodeData() +
+ ']');
}
- if (data.hasJoiningNodeData())
- onDiscoDataReceived((DiscoveryData) data.joiningNodeData());
+ if (discoProtoVer == 2) {
+ if (data.hasJoiningNodeData()) {
+ ContinuousRoutinesJoiningNodeDiscoveryData nodeData = (ContinuousRoutinesJoiningNodeDiscoveryData)
+ data.joiningNodeData();
+
+ for (ContinuousRoutineInfo routineInfo : nodeData.startedRoutines) {
+ routinesInfo.addRoutineInfo(routineInfo);
+
+ startDiscoveryDataRoutine(routineInfo);
+ }
+ }
+ }
+ else {
+ if (data.hasJoiningNodeData())
+ onDiscoDataReceived((DiscoveryData) data.joiningNodeData());
+ }
}
/** {@inheritDoc} */
@Override public void onGridDataReceived(GridDiscoveryData data) {
- Map<UUID, Serializable> nodeSpecData = data.nodeSpecificData();
+ if (discoProtoVer == 2) {
+ if (ctx.isDaemon())
+ return;
+
+ if (data.commonData() != null) {
+ ContinuousRoutinesCommonDiscoveryData commonData =
+ (ContinuousRoutinesCommonDiscoveryData)data.commonData();
+
+ for (ContinuousRoutineInfo routineInfo : commonData.startedRoutines) {
+ if (routinesInfo.routineExists(routineInfo.routineId))
+ continue;
+
+ routinesInfo.addRoutineInfo(routineInfo);
+
+ startDiscoveryDataRoutine(routineInfo);
+ }
+ }
+ }
+ else {
+ Map<UUID, Serializable> nodeSpecData = data.nodeSpecificData();
+
+ if (nodeSpecData != null) {
+ for (Map.Entry<UUID, Serializable> e : nodeSpecData.entrySet())
+ onDiscoDataReceived((DiscoveryData) e.getValue());
+ }
+ }
+ }
+
+ /**
+ * @param routineInfo Routine info.
+ */
+ private void startDiscoveryDataRoutine(ContinuousRoutineInfo routineInfo) {
+ IgnitePredicate<ClusterNode> nodeFilter = null;
+
+ try {
+ if (routineInfo.nodeFilter != null) {
+ nodeFilter = U.unmarshal(marsh, routineInfo.nodeFilter, U.resolveClassLoader(ctx.config()));
+
+ ctx.resource().injectGeneric(nodeFilter);
+ }
+ }
+ catch (IgniteCheckedException e) {
+ U.error(log, "Failed to unmarshal continuous routine filter, ignore routine [" +
+ "routineId=" + routineInfo.routineId +
+ ", srcNodeId=" + routineInfo.srcNodeId + ']', e);
+
+ return;
+ }
+
+ if (nodeFilter == null || nodeFilter.apply(ctx.discovery().localNode())) {
+ GridContinuousHandler hnd;
- if (nodeSpecData != null) {
- for (Map.Entry<UUID, Serializable> e : nodeSpecData.entrySet())
- onDiscoDataReceived((DiscoveryData) e.getValue());
+ try {
+ hnd = U.unmarshal(marsh, routineInfo.hnd, U.resolveClassLoader(ctx.config()));
+
+ if (ctx.config().isPeerClassLoadingEnabled())
+ hnd.p2pUnmarshal(routineInfo.srcNodeId, ctx);
+ }
+ catch (IgniteCheckedException e) {
+ U.error(log, "Failed to unmarshal continuous routine handler, ignore routine [" +
+ "routineId=" + routineInfo.routineId +
+ ", srcNodeId=" + routineInfo.srcNodeId + ']', e);
+
+ return;
+ }
+
+ try {
+ registerHandler(routineInfo.srcNodeId,
+ routineInfo.routineId,
+ hnd,
+ routineInfo.bufSize,
+ routineInfo.interval,
+ routineInfo.autoUnsubscribe,
+ false);
+ }
+ catch (IgniteCheckedException e) {
+ U.error(log, "Failed to register continuous routine handler, ignore routine [" +
+ "routineId=" + routineInfo.routineId +
+ ", srcNodeId=" + routineInfo.srcNodeId + ']', e);
+ }
+ }
+ else {
+ if (log.isDebugEnabled()) {
+ log.debug("Do not register continuous routine, rejected by node filter [" +
+ "routineId=" + routineInfo.routineId +
+ ", srcNodeId=" + routineInfo.srcNodeId + ']');
+ }
}
}
@@ -564,13 +716,14 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
* @param rmtFilter Remote filter.
* @param prjPred Projection predicate.
* @return Routine ID.
+ * @throws IgniteCheckedException If failed.
*/
@SuppressWarnings("unchecked")
public UUID registerStaticRoutine(
String cacheName,
CacheEntryUpdatedListener<?, ?> locLsnr,
CacheEntryEventSerializableFilter rmtFilter,
- @Nullable IgnitePredicate<ClusterNode> prjPred) {
+ @Nullable IgnitePredicate<ClusterNode> prjPred) throws IgniteCheckedException {
String topicPrefix = "CONTINUOUS_QUERY_STATIC" + "_" + cacheName;
CacheContinuousQueryHandler hnd = new CacheContinuousQueryHandler(
@@ -589,6 +742,17 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
LocalRoutineInfo routineInfo = new LocalRoutineInfo(prjPred, hnd, 1, 0, true);
+ if (discoProtoVer == 2) {
+ routinesInfo.addRoutineInfo(createRoutineInfo(
+ ctx.localNodeId(),
+ routineId,
+ hnd,
+ prjPred,
+ routineInfo.bufSize,
+ routineInfo.interval,
+ routineInfo.autoUnsubscribe));
+ }
+
locInfos.put(routineId, routineInfo);
registerMessageListener(hnd);
@@ -597,6 +761,40 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
}
/**
+ * @param srcNodeId Source node ID.
+ * @param routineId Routine ID.
+ * @param hnd Handler.
+ * @param nodeFilter Node filter.
+ * @param bufSize Handler buffer size.
+ * @param interval Time interval.
+ * @param autoUnsubscribe Auto unsubscribe flag.
+ * @return Routine info instance.
+ * @throws IgniteCheckedException If failed.
+ */
+ private ContinuousRoutineInfo createRoutineInfo(
+ UUID srcNodeId,
+ UUID routineId,
+ GridContinuousHandler hnd,
+ @Nullable IgnitePredicate<ClusterNode> nodeFilter,
+ int bufSize,
+ long interval,
+ boolean autoUnsubscribe)
+ throws IgniteCheckedException {
+ byte[] hndBytes = marsh.marshal(hnd);
+
+ byte[] filterBytes = nodeFilter != null ? marsh.marshal(nodeFilter) : null;
+
+ return new ContinuousRoutineInfo(
+ srcNodeId,
+ routineId,
+ hndBytes,
+ filterBytes,
+ bufSize,
+ interval,
+ autoUnsubscribe);
+ }
+
+ /**
* @param hnd Handler.
* @param bufSize Buffer size.
* @param interval Time interval.
@@ -638,30 +836,10 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
// Whether local node is included in routine.
boolean locIncluded = prjPred == null || prjPred.apply(ctx.discovery().localNode());
- StartRequestData reqData = new StartRequestData(prjPred, hnd.clone(), bufSize, interval, autoUnsubscribe);
+ AbstractContinuousMessage msg;
try {
- if (ctx.config().isPeerClassLoadingEnabled()) {
- // Handle peer deployment for projection predicate.
- if (prjPred != null && !U.isGrid(prjPred.getClass())) {
- Class cls = U.detectClass(prjPred);
-
- String clsName = cls.getName();
-
- GridDeployment dep = ctx.deploy().deploy(cls, U.detectClassLoader(cls));
-
- if (dep == null)
- throw new IgniteDeploymentCheckedException("Failed to deploy projection predicate: " + prjPred);
-
- reqData.className(clsName);
- reqData.deploymentInfo(new GridDeploymentInfoBean(dep));
-
- reqData.p2pMarshal(marsh);
- }
-
- // Handle peer deployment for other handler-specific objects.
- reqData.handler().p2pMarshal(ctx);
- }
+ msg = createStartMessage(routineId, hnd, bufSize, interval, autoUnsubscribe, prjPred);
}
catch (IgniteCheckedException e) {
return new GridFinishedFuture<>(e);
@@ -674,20 +852,26 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
return new GridFinishedFuture<>(new NodeStoppingException("Failed to start continuous query (node is stopping)"));
try {
- StartFuture fut = new StartFuture(ctx, routineId);
+ StartFuture fut = new StartFuture(routineId);
startFuts.put(routineId, fut);
try {
- if (locIncluded || hnd.isQuery())
- registerHandler(ctx.localNodeId(), routineId, hnd, bufSize, interval, autoUnsubscribe, true);
+ if (locIncluded || hnd.isQuery()) {
+ registerHandler(ctx.localNodeId(),
+ routineId,
+ hnd,
+ bufSize,
+ interval,
+ autoUnsubscribe,
+ true);
+ }
- ctx.discovery().sendCustomEvent(new StartRoutineDiscoveryMessage(routineId, reqData,
- reqData.handler().keepBinary()));
- }
- catch (IgniteCheckedException e) {
- startFuts.remove(routineId);
- locInfos.remove(routineId);
+ ctx.discovery().sendCustomEvent(msg);
+ }
+ catch (IgniteCheckedException e) {
+ startFuts.remove(routineId);
+ locInfos.remove(routineId);
unregisterHandler(routineId, hnd, true);
@@ -707,6 +891,92 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
}
/**
+ * @param routineId Routine ID.
+ * @param hnd Handler.
+ * @param bufSize Buffer size.
+ * @param interval Interval.
+ * @param autoUnsubscribe Auto unsubscribe flag.
+ * @param nodeFilter Node filter.
+ * @return Routine start message.
+ * @throws IgniteCheckedException If failed.
+ */
+ private AbstractContinuousMessage createStartMessage(UUID routineId,
+ GridContinuousHandler hnd,
+ int bufSize,
+ long interval,
+ boolean autoUnsubscribe,
+ @Nullable IgnitePredicate<ClusterNode> nodeFilter)
+ throws IgniteCheckedException
+ {
+ hnd = hnd.clone();
+
+ String clsName = null;
+ GridDeploymentInfoBean dep = null;
+
+ if (ctx.config().isPeerClassLoadingEnabled()) {
+ // Handle peer deployment for projection predicate.
+ if (nodeFilter != null && !U.isGrid(nodeFilter.getClass())) {
+ Class cls = U.detectClass(nodeFilter);
+
+ clsName = cls.getName();
+
+ GridDeployment dep0 = ctx.deploy().deploy(cls, U.detectClassLoader(cls));
+
+ if (dep0 == null)
+ throw new IgniteDeploymentCheckedException("Failed to deploy projection predicate: " + nodeFilter);
+
+ dep = new GridDeploymentInfoBean(dep0);
+ }
+
+ // Handle peer deployment for other handler-specific objects.
+ hnd.p2pMarshal(ctx);
+ }
+
+ if (discoProtoVer == 1) {
+ StartRequestData reqData = new StartRequestData(
+ nodeFilter,
+ hnd,
+ bufSize,
+ interval,
+ autoUnsubscribe);
+
+ if (clsName != null) {
+ reqData.className(clsName);
+ reqData.deploymentInfo(dep);
+
+ reqData.p2pMarshal(marsh);
+ }
+
+ return new StartRoutineDiscoveryMessage(
+ routineId,
+ reqData,
+ reqData.handler().keepBinary());
+ }
+ else {
+ assert discoProtoVer == 2 : discoProtoVer;
+
+ byte[] nodeFilterBytes = nodeFilter != null ? U.marshal(marsh, nodeFilter) : null;
+ byte[] hndBytes = U.marshal(marsh, hnd);
+
+ StartRequestDataV2 reqData = new StartRequestDataV2(nodeFilterBytes,
+ hndBytes,
+ bufSize,
+ interval,
+ autoUnsubscribe);
+
+ if (clsName != null) {
+ reqData.className(clsName);
+ reqData.deploymentInfo(dep);
+ }
+
+ return new StartRoutineDiscoveryMessageV2(
+ routineId,
+ reqData,
+ hnd.keepBinary());
+ }
+ }
+
+ /**
* @param hnd Handler.
*/
private void registerMessageListener(GridContinuousHandler hnd) {
@@ -760,29 +1030,38 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
doStop = true;
}
- if (doStop) {
- // Unregister routine locally.
- LocalRoutineInfo routine = locInfos.remove(routineId);
-
- // Finish if routine is not found (wrong ID is provided).
- if (routine == null) {
- stopFuts.remove(routineId);
+ if (doStop) {
+ boolean stop = false;
- fut.onDone();
+ // Unregister routine locally.
+ LocalRoutineInfo routine = locInfos.remove(routineId);
- return fut;
- }
+ if (routine != null) {
+ stop = true;
// Unregister handler locally.
unregisterHandler(routineId, routine.hnd, true);
+ }
- try {
- ctx.discovery().sendCustomEvent(new StopRoutineDiscoveryMessage(routineId));
- }
- catch (IgniteCheckedException e) {
- fut.onDone(e);
+ if (!stop && discoProtoVer == 2)
+ stop = routinesInfo.routineExists(routineId);
+
+ // Finish if routine is not found (wrong ID is provided).
+ if (!stop) {
+ stopFuts.remove(routineId);
+
+ fut.onDone();
+
+ return fut;
}
+ try {
+ ctx.discovery().sendCustomEvent(new StopRoutineDiscoveryMessage(routineId));
+ }
+ catch (IgniteCheckedException e) {
+ fut.onDone(e);
+ }
+
if (ctx.isStopping())
fut.onDone();
}
@@ -924,6 +1203,9 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
clientInfos.clear();
+ if (discoProtoVer == 2)
+ routinesInfo.onClientDisconnected(locInfos.keySet());
+
if (log.isDebugEnabled()) {
log.debug("after onDisconnected [rmtInfos=" + rmtInfos +
", locInfos=" + locInfos +
@@ -996,35 +1278,11 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
StartFuture fut = startFuts.remove(msg.routineId());
if (fut != null) {
- if (msg.errs().isEmpty()) {
- LocalRoutineInfo routine = locInfos.get(msg.routineId());
-
- // Update partition counters.
- if (routine != null && routine.handler().isQuery()) {
- Map<UUID, Map<Integer, T2<Long, Long>>> cntrsPerNode = msg.updateCountersPerNode();
- Map<Integer, T2<Long, Long>> cntrs = msg.updateCounters();
-
- GridCacheAdapter<Object, Object> interCache =
- ctx.cache().internalCache(routine.handler().cacheName());
-
- GridCacheContext cctx = interCache != null ? interCache.context() : null;
-
- if (cctx != null && cntrsPerNode != null && !cctx.isLocal() && cctx.affinityNode())
- cntrsPerNode.put(ctx.localNodeId(),
- toCountersMap(cctx.topology().localUpdateCounters(false)));
-
- routine.handler().updateCounters(topVer, cntrsPerNode, cntrs);
- }
-
- fut.onRemoteRegistered();
- }
- else {
- IgniteCheckedException firstEx = F.first(msg.errs().values());
-
- fut.onDone(firstEx);
-
- stopRoutine(msg.routineId());
- }
+ fut.onAllRemoteRegistered(
+ topVer,
+ msg.errs(),
+ msg.updateCountersPerNode(),
+ msg.updateCounters());
}
}
@@ -1138,6 +1396,199 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
}
/**
+ * @param sndId Sender node ID.
+ * @param msg Message.
+ */
+ private void processRoutineStartResultMessage(UUID sndId, ContinuousRoutineStartResultMessage msg) {
+ StartFuture fut = startFuts.get(msg.routineId());
+
+ if (fut != null)
+ fut.onResult(sndId, msg);
+ }
+
+ /**
+ * @param topVer Current topology version.
+ * @param snd Sender.
+ * @param msg Start request.
+ */
+ private void processStartRequestV2(final AffinityTopologyVersion topVer,
+ final ClusterNode snd,
+ final StartRoutineDiscoveryMessageV2 msg) {
+ StartRequestDataV2 reqData = msg.startRequestData();
+
+ ContinuousRoutineInfo routineInfo = new ContinuousRoutineInfo(snd.id(),
+ msg.routineId(),
+ reqData.handlerBytes(),
+ reqData.nodeFilterBytes(),
+ reqData.bufferSize(),
+ reqData.interval(),
+ reqData.autoUnsubscribe());
+
+ routinesInfo.addRoutineInfo(routineInfo);
+
+ final DiscoCache discoCache = ctx.discovery().discoCache(topVer);
+
+ // Should not use marshaller and send messages from discovery thread.
+ ctx.getSystemExecutorService().execute(new Runnable() {
+ @Override public void run() {
+ if (snd.id().equals(ctx.localNodeId())) {
+ StartFuture fut = startFuts.get(msg.routineId());
+
+ if (fut != null)
+ fut.initRemoteNodes(discoCache);
+
+ return;
+ }
+
+ StartRequestDataV2 reqData = msg.startRequestData();
+
+ Exception err = null;
+
+ IgnitePredicate<ClusterNode> nodeFilter = null;
+
+ byte[] cntrs = null;
+
+ if (reqData.nodeFilterBytes() != null) {
+ try {
+ if (ctx.config().isPeerClassLoadingEnabled() && reqData.className() != null) {
+ String clsName = reqData.className();
+ GridDeploymentInfo depInfo = reqData.deploymentInfo();
+
+ GridDeployment dep = ctx.deploy().getGlobalDeployment(depInfo.deployMode(),
+ clsName,
+ clsName,
+ depInfo.userVersion(),
+ snd.id(),
+ depInfo.classLoaderId(),
+ depInfo.participants(),
+ null);
+
+ if (dep == null) {
+ throw new IgniteDeploymentCheckedException("Failed to obtain deployment " +
+ "for class: " + clsName);
+ }
+
+ nodeFilter = U.unmarshal(marsh,
+ reqData.nodeFilterBytes(),
+ U.resolveClassLoader(dep.classLoader(), ctx.config()));
+ }
+ else {
+ nodeFilter = U.unmarshal(marsh,
+ reqData.nodeFilterBytes(),
+ U.resolveClassLoader(ctx.config()));
+ }
+
+ if (nodeFilter != null)
+ ctx.resource().injectGeneric(nodeFilter);
+ }
+ catch (Exception e) {
+ err = e;
+
+ U.error(log, "Failed to unmarshal continuous routine filter [" +
+ "routineId=" + msg.routineId +
+ ", srcNodeId=" + snd.id() + ']', e);
+ }
+ }
+
+ boolean register = err == null &&
+ (nodeFilter == null || nodeFilter.apply(ctx.discovery().localNode()));
+
+ if (register) {
+ try {
+ GridContinuousHandler hnd = U.unmarshal(marsh,
+ reqData.handlerBytes(),
+ U.resolveClassLoader(ctx.config()));
+
+ if (ctx.config().isPeerClassLoadingEnabled())
+ hnd.p2pUnmarshal(snd.id(), ctx);
+
+ if (msg.keepBinary()) {
+ assert hnd instanceof CacheContinuousQueryHandler : hnd;
+
+ ((CacheContinuousQueryHandler)hnd).keepBinary(true);
+ }
+
+ GridContinuousHandler hnd0 = hnd instanceof GridMessageListenHandler ?
+ new GridMessageListenHandler((GridMessageListenHandler)hnd) :
+ hnd;
+
+ registerHandler(snd.id(),
+ msg.routineId,
+ hnd0,
+ reqData.bufferSize(),
+ reqData.interval(),
+ reqData.autoUnsubscribe(),
+ false);
+
+ if (hnd0.isQuery()) {
+ GridCacheProcessor proc = ctx.cache();
+
+ if (proc != null) {
+ GridCacheAdapter cache = ctx.cache().internalCache(hnd0.cacheName());
+
+ if (cache != null && !cache.isLocal() && cache.context().userCache()) {
+ CachePartitionPartialCountersMap cntrsMap =
+ cache.context().topology().localUpdateCounters(false);
+
+ cntrs = U.marshal(marsh, cntrsMap);
+ }
+ }
+ }
+ }
+ catch (Exception e) {
+ err = e;
+
+ U.error(log, "Failed to register continuous routine handler [" +
+ "routineId=" + msg.routineId +
+ ", srcNodeId=" + snd.id() + ']', e);
+ }
+ }
+
+ sendMessageStartResult(snd, msg.routineId(), cntrs, err);
+ }
+ });
+ }
+
+ /**
+ * @param node Target node.
+ * @param routineId Routine ID.
+ * @param cntrsMapBytes Marshalled {@link CachePartitionPartialCountersMap}.
+ * @param err Start error if any.
+ */
+ private void sendMessageStartResult(final ClusterNode node,
+ final UUID routineId,
+ byte[] cntrsMapBytes,
+ final @Nullable Exception err)
+ {
+ byte[] errBytes = null;
+
+ if (err != null) {
+ try {
+ errBytes = U.marshal(marsh, err);
+ }
+ catch (Exception e) {
+ U.error(log, "Failed to marshal routine start error: " + e, e);
+ }
+ }
+
+ ContinuousRoutineStartResultMessage msg = new ContinuousRoutineStartResultMessage(routineId,
+ cntrsMapBytes,
+ errBytes,
+ err != null);
+
+ try {
+ ctx.io().sendToGridTopic(node, TOPIC_CONTINUOUS, msg, SYSTEM_POOL, null);
+ }
+ catch (ClusterTopologyCheckedException e) {
+ if (log.isDebugEnabled())
+ log.debug("Failed to send routine start result, node failed: " + e);
+ }
+ catch (IgniteCheckedException e) {
+ U.error(log, "Failed to send routine start result: " + e, e);
+ }
+ }
+
+ /**
* @param msg Message.
*/
private void processMessageAck(GridContinuousMessage msg) {
@@ -1455,6 +1906,13 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
UUID nodeId = ((DiscoveryEvent)evt).eventNode().id();
+ if (discoProtoVer == 2) {
+ routinesInfo.onNodeFail(nodeId);
+
+ for (StartFuture fut : startFuts.values())
+ fut.onNodeFail(nodeId);
+ }
+
clientInfos.remove(nodeId);
// Unregister handlers created by left node.
@@ -1894,10 +2352,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
/**
* Future for start routine.
*/
- private static class StartFuture extends GridFutureAdapter<UUID> {
- /** */
- private GridKernalContext ctx;
-
+ private class StartFuture extends GridFutureAdapter<UUID> {
/** Consume ID. */
private UUID routineId;
@@ -1907,56 +2362,170 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
/** All remote listeners are registered. */
private volatile boolean rmt;
- /** Timeout object. */
- private volatile GridTimeoutObject timeoutObj;
+ /** */
+ private final DiscoveryMessageResultsCollector<ContinuousRoutineStartResultMessage, RoutineRegisterResults>
+ resCollect;
/**
- * @param ctx Kernal context.
* @param routineId Consume ID.
*/
- StartFuture(GridKernalContext ctx, UUID routineId) {
- this.ctx = ctx;
-
+ StartFuture(UUID routineId) {
this.routineId = routineId;
+
+ resCollect = new DiscoveryMessageResultsCollector<ContinuousRoutineStartResultMessage, RoutineRegisterResults>(ctx) {
+ @Override protected RoutineRegisterResults createResult(Map<UUID, NodeMessage<ContinuousRoutineStartResultMessage>> rcvd) {
+ Map<UUID, Exception> errs = null;
+ Map<UUID, Map<Integer, T2<Long, Long>>> cntrsPerNode = null;
+
+ for (Map.Entry<UUID, NodeMessage<ContinuousRoutineStartResultMessage>> entry : rcvd.entrySet()) {
+ ContinuousRoutineStartResultMessage msg = entry.getValue().message();
+
+ if (msg == null)
+ continue;
+
+ if (msg.error()) {
+ byte[] errBytes = msg.errorBytes();
+
+ Exception err = null;
+
+ if (errBytes != null) {
+ try {
+ err = U.unmarshal(marsh, errBytes, U.resolveClassLoader(ctx.config()));
+ }
+ catch (Exception e) {
+ U.warn(log, "Failed to unmarhal continuous routine start error: " + e);
+ }
+ }
+
+ if (err == null) {
+ err = new IgniteCheckedException("Failed to start continuous " +
+ "routine on node: " + entry.getKey());
+ }
+
+ if (errs == null)
+ errs = new HashMap<>();
+
+ errs.put(entry.getKey(), err);
+ }
+ else {
+ byte[] cntrsMapBytes = msg.countersMapBytes();
+
+ if (cntrsMapBytes != null) {
+ try {
+ CachePartitionPartialCountersMap cntrsMap = U.unmarshal(
+ marsh,
+ cntrsMapBytes,
+ U.resolveClassLoader(ctx.config()));
+
+ if (cntrsPerNode == null)
+ cntrsPerNode = new HashMap<>();
+
+ cntrsPerNode.put(entry.getKey(), CachePartitionPartialCountersMap.toCountersMap(cntrsMap));
+ }
+ catch (Exception e) {
+ U.warn(log, "Failed to unmarhal continuous query update counters: " + e);
+ }
+ }
+ }
+ }
+
+ return new RoutineRegisterResults(discoCache.version(), errs, cntrsPerNode);
+ }
+
+ @Override protected void onResultsCollected(RoutineRegisterResults res0) {
+ onAllRemoteRegistered(res0.topVer, res0.errs, res0.cntrsPerNode, null);
+ }
+
+ @Override protected boolean waitForNode(DiscoCache discoCache, ClusterNode node) {
+ return !ctx.localNodeId().equals(node.id());
+ }
+ };
}
/**
- * Called when local listener is registered.
+ * @param topVer Topology version.
+ * @param errs Errors.
+ * @param cntrsPerNode Update counters.
+ * @param cntrs Update counters.
*/
- public void onLocalRegistered() {
- loc = true;
+ private void onAllRemoteRegistered(
+ AffinityTopologyVersion topVer,
+ @Nullable Map<UUID, ? extends Exception> errs,
+ Map<UUID, Map<Integer, T2<Long, Long>>> cntrsPerNode,
+ Map<Integer, T2<Long, Long>> cntrs) {
+ try {
+ if (errs == null || errs.isEmpty()) {
+ LocalRoutineInfo routine = locInfos.get(routineId);
- if (rmt && !isDone())
- onDone(routineId);
+ // Update partition counters.
+ if (routine != null && routine.handler().isQuery()) {
+ GridCacheAdapter<Object, Object> interCache =
+ ctx.cache().internalCache(routine.handler().cacheName());
+
+ GridCacheContext cctx = interCache != null ? interCache.context() : null;
+
+ if (cctx != null && cntrsPerNode != null && !cctx.isLocal() && cctx.affinityNode())
+ cntrsPerNode.put(ctx.localNodeId(),
+ toCountersMap(cctx.topology().localUpdateCounters(false)));
+
+ routine.handler().updateCounters(topVer, cntrsPerNode, cntrs);
+ }
+
+ onRemoteRegistered();
+ }
+ else {
+ Exception firstEx = F.first(errs.values());
+
+ onDone(firstEx);
+
+ stopRoutine(routineId);
+ }
+ }
+ finally {
+ startFuts.remove(routineId, this);
+ }
}
/**
- * Called when all remote listeners are registered.
+ * @param discoCache Discovery state.
*/
- public void onRemoteRegistered() {
- rmt = true;
+ void initRemoteNodes(DiscoCache discoCache) {
+ resCollect.init(discoCache);
+ }
- if (loc && !isDone())
- onDone(routineId);
+ /**
+ * @param nodeId Node ID.
+ * @param msg Message.
+ */
+ void onResult(UUID nodeId, ContinuousRoutineStartResultMessage msg) {
+ resCollect.onMessage(nodeId, msg);
}
/**
- * @param timeoutObj Timeout object.
+ * @param nodeId Failed node ID.
*/
- public void addTimeoutObject(GridTimeoutObject timeoutObj) {
- assert timeoutObj != null;
+ void onNodeFail(UUID nodeId) {
+ resCollect.onNodeFail(nodeId);
+ }
- this.timeoutObj = timeoutObj;
+ /**
+ * Called when local listener is registered.
+ */
+ void onLocalRegistered() {
+ loc = true;
- ctx.timeout().addTimeoutObject(timeoutObj);
+ if (rmt && !isDone())
+ onDone(routineId);
}
- /** {@inheritDoc} */
- @Override public boolean onDone(@Nullable UUID res, @Nullable Throwable err) {
- if (timeoutObj != null)
- ctx.timeout().removeTimeoutObject(timeoutObj);
+ /**
+ * Called when all remote listeners are registered.
+ */
+ void onRemoteRegistered() {
+ rmt = true;
- return super.onDone(res, err);
+ if (loc && !isDone())
+ onDone(routineId);
}
/** {@inheritDoc} */
@@ -1966,6 +2535,33 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
}
/**
+ *
+ */
+ private static class RoutineRegisterResults {
+ /** */
+ private final AffinityTopologyVersion topVer;
+
+ /** */
+ private final Map<UUID, ? extends Exception> errs;
+
+ /** */
+ private final Map<UUID, Map<Integer, T2<Long, Long>>> cntrsPerNode;
+
+ /**
+ * @param topVer Topology version.
+ * @param errs Errors.
+ * @param cntrsPerNode Update counters.
+ */
+ RoutineRegisterResults(AffinityTopologyVersion topVer,
+ Map<UUID, ? extends Exception> errs,
+ Map<UUID, Map<Integer, T2<Long, Long>>> cntrsPerNode) {
+ this.topVer = topVer;
+ this.errs = errs;
+ this.cntrsPerNode = cntrsPerNode;
+ }
+ }
+
+ /**
* Future for stop routine.
*/
private static class StopFuture extends GridFutureAdapter<Object> {
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRequestDataV2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRequestDataV2.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRequestDataV2.java
new file mode 100644
index 0000000..c001616
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRequestDataV2.java
@@ -0,0 +1,164 @@
+/*
+ * 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.processors.continuous;
+
+import java.io.Serializable;
+import org.apache.ignite.internal.managers.deployment.GridDeploymentInfo;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * Start request data.
+ */
+class StartRequestDataV2 implements Serializable {
+ /** */
+ private static final long serialVersionUID = 0L;
+
+ /** Serialized node filter. */
+ private byte[] nodeFilterBytes;
+
+ /** Deployment class name. */
+ private String clsName;
+
+ /** Deployment info. */
+ private GridDeploymentInfo depInfo;
+
+ /** Serialized handler. */
+ private byte[] hndBytes;
+
+ /** Buffer size. */
+ private int bufSize;
+
+ /** Time interval. */
+ private long interval;
+
+ /** Automatic unsubscribe flag. */
+ private boolean autoUnsubscribe;
+
+ /**
+ * @param nodeFilterBytes Serialized node filter.
+ * @param hndBytes Serialized handler.
+ * @param bufSize Buffer size.
+ * @param interval Time interval.
+ * @param autoUnsubscribe Automatic unsubscribe flag.
+ */
+ StartRequestDataV2(
+ byte[] nodeFilterBytes,
+ byte[] hndBytes,
+ int bufSize,
+ long interval,
+ boolean autoUnsubscribe) {
+ assert hndBytes != null;
+ assert bufSize > 0;
+ assert interval >= 0;
+
+ this.nodeFilterBytes = nodeFilterBytes;
+ this.hndBytes = hndBytes;
+ this.bufSize = bufSize;
+ this.interval = interval;
+ this.autoUnsubscribe = autoUnsubscribe;
+ }
+
+ /**
+ * @return Serialized node filter.
+ */
+ public byte[] nodeFilterBytes() {
+ return nodeFilterBytes;
+ }
+
+ /**
+ * @return Deployment class name.
+ */
+ public String className() {
+ return clsName;
+ }
+
+ /**
+ * @param clsName New deployment class name.
+ */
+ public void className(String clsName) {
+ this.clsName = clsName;
+ }
+
+ /**
+ * @return Deployment info.
+ */
+ public GridDeploymentInfo deploymentInfo() {
+ return depInfo;
+ }
+
+ /**
+ * @param depInfo New deployment info.
+ */
+ public void deploymentInfo(GridDeploymentInfo depInfo) {
+ this.depInfo = depInfo;
+ }
+
+ /**
+ * @return Handler.
+ */
+ public byte[] handlerBytes() {
+ return hndBytes;
+ }
+
+ /**
+ * @return Buffer size.
+ */
+ public int bufferSize() {
+ return bufSize;
+ }
+
+ /**
+ * @param bufSize New buffer size.
+ */
+ public void bufferSize(int bufSize) {
+ this.bufSize = bufSize;
+ }
+
+ /**
+ * @return Time interval.
+ */
+ public long interval() {
+ return interval;
+ }
+
+ /**
+ * @param interval New time interval.
+ */
+ public void interval(long interval) {
+ this.interval = interval;
+ }
+
+ /**
+ * @return Automatic unsubscribe flag.
+ */
+ public boolean autoUnsubscribe() {
+ return autoUnsubscribe;
+ }
+
+ /**
+ * @param autoUnsubscribe New automatic unsubscribe flag.
+ */
+ public void autoUnsubscribe(boolean autoUnsubscribe) {
+ this.autoUnsubscribe = autoUnsubscribe;
+ }
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return S.toString(StartRequestDataV2.class, this);
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRoutineDiscoveryMessageV2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRoutineDiscoveryMessageV2.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRoutineDiscoveryMessageV2.java
new file mode 100644
index 0000000..275765d
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRoutineDiscoveryMessageV2.java
@@ -0,0 +1,77 @@
+/*
+ * 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.processors.continuous;
+
+import java.util.UUID;
+import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ *
+ */
+public class StartRoutineDiscoveryMessageV2 extends AbstractContinuousMessage {
+ /** */
+ private static final long serialVersionUID = 0L;
+
+ /** */
+ private static final int KEEP_BINARY_FLAG = 0x01;
+
+ /** */
+ private final StartRequestDataV2 startReqData;
+
+ /** Flags. */
+ private int flags;
+
+ /**
+ * @param routineId Routine id.
+ * @param startReqData Start request data.
+ * @param keepBinary Keep binary flag.
+ */
+ StartRoutineDiscoveryMessageV2(UUID routineId, StartRequestDataV2 startReqData, boolean keepBinary) {
+ super(routineId);
+
+ this.startReqData = startReqData;
+
+ if (keepBinary)
+ flags |= KEEP_BINARY_FLAG;
+ }
+
+ /**
+ * @return Start request data.
+ */
+ public StartRequestDataV2 startRequestData() {
+ return startReqData;
+ }
+
+ /**
+ * @return {@code True} if keep binary flag was set on continuous handler.
+ */
+ public boolean keepBinary() {
+ return (flags & KEEP_BINARY_FLAG) != 0;
+ }
+
+ /** {@inheritDoc} */
+ @Override public DiscoveryCustomMessage ackMessage() {
+ return null;
+ }
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return S.toString(StartRoutineDiscoveryMessageV2.class, this, "routineId", routineId());
+ }
+}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StopRoutineAckDiscoveryMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StopRoutineAckDiscoveryMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StopRoutineAckDiscoveryMessage.java
index 79d8b29..dfba0e7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StopRoutineAckDiscoveryMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StopRoutineAckDiscoveryMessage.java
@@ -42,6 +42,11 @@ public class StopRoutineAckDiscoveryMessage extends AbstractContinuousMessage {
}
/** {@inheritDoc} */
+ @Override public boolean stopProcess() {
+ return false;
+ }
+
+ /** {@inheritDoc} */
@Override public String toString() {
return S.toString(StopRoutineAckDiscoveryMessage.class, this, "routineId", routineId());
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
index 4a893f4..8cad342 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
@@ -47,6 +47,7 @@ import javax.cache.expiry.ExpiryPolicy;
import org.apache.ignite.Ignite;
import org.apache.ignite.IgniteCache;
import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteClientDisconnectedException;
import org.apache.ignite.IgniteDataStreamer;
import org.apache.ignite.IgniteDataStreamerTimeoutException;
import org.apache.ignite.IgniteException;
@@ -100,6 +101,7 @@ import org.apache.ignite.internal.util.tostring.GridToStringExclude;
import org.apache.ignite.internal.util.tostring.GridToStringInclude;
import org.apache.ignite.internal.util.typedef.CI1;
import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.X;
import org.apache.ignite.internal.util.typedef.internal.A;
import org.apache.ignite.internal.util.typedef.internal.CU;
import org.apache.ignite.internal.util.typedef.internal.GPC;
@@ -1059,6 +1061,9 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
return;
while (true) {
+ if (disconnectErr != null)
+ throw disconnectErr;
+
Queue<IgniteInternalFuture<?>> q = null;
for (Buffer buf : bufMappings.values()) {
@@ -1826,15 +1831,19 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
catch (IgniteCheckedException e) {
GridFutureAdapter<Object> fut0 = ((GridFutureAdapter<Object>)fut);
- try {
- if (ctx.discovery().alive(node) && ctx.discovery().pingNode(node.id()))
- fut0.onDone(e);
- else
- fut0.onDone(new ClusterTopologyCheckedException("Failed to send request (node has left): "
- + node.id()));
- }
- catch (IgniteClientDisconnectedCheckedException e0) {
- fut0.onDone(e0);
+ if (X.hasCause(e, IgniteClientDisconnectedCheckedException.class, IgniteClientDisconnectedException.class))
+ fut0.onDone(e);
+ else {
+ try {
+ if (ctx.discovery().alive(node) && ctx.discovery().pingNode(node.id()))
+ fut0.onDone(e);
+ else
+ fut0.onDone(new ClusterTopologyCheckedException("Failed to send request (node has left): "
+ + node.id()));
+ }
+ catch (IgniteClientDisconnectedCheckedException e0) {
+ fut0.onDone(e0);
+ }
}
}
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappingAcceptedMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappingAcceptedMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappingAcceptedMessage.java
index 7af0559..80e3f7d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappingAcceptedMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappingAcceptedMessage.java
@@ -63,6 +63,11 @@ public class MappingAcceptedMessage implements DiscoveryCustomMessage {
}
/** {@inheritDoc} */
+ @Override public boolean stopProcess() {
+ return false;
+ }
+
+ /** {@inheritDoc} */
@Nullable @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr,
AffinityTopologyVersion topVer, DiscoCache discoCache) {
throw new UnsupportedOperationException();
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappingProposedMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappingProposedMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappingProposedMessage.java
index b4e13fb..9358585 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappingProposedMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappingProposedMessage.java
@@ -98,6 +98,11 @@ public class MappingProposedMessage implements DiscoveryCustomMessage {
}
/** {@inheritDoc} */
+ @Override public boolean stopProcess() {
+ return false;
+ }
+
+ /** {@inheritDoc} */
@Nullable @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr,
AffinityTopologyVersion topVer, DiscoCache discoCache) {
throw new UnsupportedOperationException();
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/message/SchemaFinishDiscoveryMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/message/SchemaFinishDiscoveryMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/message/SchemaFinishDiscoveryMessage.java
index 2245b24..f802e09 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/message/SchemaFinishDiscoveryMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/message/SchemaFinishDiscoveryMessage.java
@@ -59,6 +59,11 @@ public class SchemaFinishDiscoveryMessage extends SchemaAbstractDiscoveryMessage
}
/** {@inheritDoc} */
+ @Override public boolean stopProcess() {
+ return false;
+ }
+
+ /** {@inheritDoc} */
@Override public boolean exchange() {
return false;
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/message/SchemaProposeDiscoveryMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/message/SchemaProposeDiscoveryMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/message/SchemaProposeDiscoveryMessage.java
index 0e1270b..62b6d6a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/message/SchemaProposeDiscoveryMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/message/SchemaProposeDiscoveryMessage.java
@@ -60,6 +60,11 @@ public class SchemaProposeDiscoveryMessage extends SchemaAbstractDiscoveryMessag
}
/** {@inheritDoc} */
+ @Override public boolean stopProcess() {
+ return false;
+ }
+
+ /** {@inheritDoc} */
@Override public boolean exchange() {
return exchange;
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
index e0ec8d1..0fcde0e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
@@ -491,6 +491,17 @@ public class GridNioServer<T> {
/**
* @param ses Session.
+ */
+ public void closeFromWorkerThread(GridNioSession ses) {
+ assert ses instanceof GridSelectorNioSessionImpl : ses;
+
+ GridSelectorNioSessionImpl ses0 = (GridSelectorNioSessionImpl)ses;
+
+ ((AbstractNioClientWorker)ses0.worker()).close((GridSelectorNioSessionImpl)ses, null);
+ }
+
+ /**
+ * @param ses Session.
* @param msg Message.
* @param createFut {@code True} if future should be created.
* @param ackC Closure invoked when message ACK is received.
@@ -2170,7 +2181,12 @@ public class GridNioServer<T> {
dumpSelectorInfo(sb, keys);
for (SelectionKey key : keys) {
- GridSelectorNioSessionImpl ses = (GridSelectorNioSessionImpl)key.attachment();
+ GridNioKeyAttachment attach = (GridNioKeyAttachment)key.attachment();
+
+ if (!attach.hasSession())
+ continue;
+
+ GridSelectorNioSessionImpl ses = attach.session();
boolean sesInfo = p == null || p.apply(ses);
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
index 1754cc8..e8c27d2 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
@@ -963,5 +963,15 @@ public abstract class IgniteSpiAdapter implements IgniteSpi {
@Override public Map<String, Object> nodeAttributes() {
return Collections.emptyMap();
}
+
+ /** {@inheritDoc} */
+ @Override public boolean communicationFailureResolveSupported() {
+ return false;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void resolveCommunicationFailure(ClusterNode node, Exception err) {
+ throw new UnsupportedOperationException();
+ }
}
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java
index 108c4d4..d4402f4 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java
@@ -365,4 +365,15 @@ public interface IgniteSpiContext {
* @return Current node attributes.
*/
public Map<String, Object> nodeAttributes();
+
+ /**
+ * @return {@code True} if cluster supports communication error resolving.
+ */
+ public boolean communicationFailureResolveSupported();
+
+ /**
+ * @param node Problem node.
+ * @param err Error.
+ */
+ public void resolveCommunicationFailure(ClusterNode node, Exception err);
}
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
index e1addd8..4a0710e 100755
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
@@ -32,6 +32,7 @@ import java.nio.channels.SocketChannel;
import java.nio.channels.spi.AbstractInterruptibleChannel;
import java.util.ArrayList;
import java.util.Arrays;
+import java.util.BitSet;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
@@ -66,12 +67,14 @@ import org.apache.ignite.internal.IgniteInternalFuture;
import org.apache.ignite.internal.IgniteInterruptedCheckedException;
import org.apache.ignite.internal.IgniteKernal;
import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi;
import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
import org.apache.ignite.internal.managers.eventstorage.HighPriorityListener;
import org.apache.ignite.internal.util.GridConcurrentFactory;
import org.apache.ignite.internal.util.GridSpinReadWriteLock;
import org.apache.ignite.internal.util.future.GridFinishedFuture;
import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.future.IgniteFutureImpl;
import org.apache.ignite.internal.util.ipc.IpcEndpoint;
import org.apache.ignite.internal.util.ipc.IpcToNioAdapter;
import org.apache.ignite.internal.util.ipc.shmem.IpcOutOfSystemResourcesException;
@@ -134,6 +137,9 @@ import org.apache.ignite.spi.IgniteSpiThread;
import org.apache.ignite.spi.IgniteSpiTimeoutObject;
import org.apache.ignite.spi.communication.CommunicationListener;
import org.apache.ignite.spi.communication.CommunicationSpi;
+import org.apache.ignite.spi.communication.tcp.internal.ConnectionKey;
+import org.apache.ignite.spi.communication.tcp.internal.TcpCommunicationConnectionCheckFuture;
+import org.apache.ignite.spi.communication.tcp.internal.TcpCommunicationNodeConnectionCheckFuture;
import org.apache.ignite.spi.communication.tcp.messages.HandshakeMessage;
import org.apache.ignite.spi.communication.tcp.messages.HandshakeMessage2;
import org.apache.ignite.spi.communication.tcp.messages.NodeIdMessage;
@@ -146,6 +152,7 @@ import org.jetbrains.annotations.Nullable;
import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
import static org.apache.ignite.internal.util.nio.GridNioSessionMetaKey.SSL_META;
+import static org.apache.ignite.spi.communication.tcp.internal.TcpCommunicationConnectionCheckFuture.SES_FUT_META;
import static org.apache.ignite.spi.communication.tcp.messages.RecoveryLastReceivedMessage.ALREADY_CONNECTED;
import static org.apache.ignite.spi.communication.tcp.messages.RecoveryLastReceivedMessage.NEED_WAIT;
import static org.apache.ignite.spi.communication.tcp.messages.RecoveryLastReceivedMessage.NODE_STOPPING;
@@ -310,7 +317,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati
private static final IgniteProductVersion VERSION_SINCE_CLIENT_COULD_WAIT_TO_CONNECT = IgniteProductVersion.fromString("2.1.4");
/** Connection index meta for session. */
- private static final int CONN_IDX_META = GridNioSessionMetaKey.nextUniqueKey();
+ public static final int CONN_IDX_META = GridNioSessionMetaKey.nextUniqueKey();
/** Message tracker meta for session. */
private static final int TRACKER_META = GridNioSessionMetaKey.nextUniqueKey();
@@ -408,6 +415,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati
ConnectionKey connId = ses.meta(CONN_IDX_META);
if (connId != null) {
+ if (connId.dummy())
+ return;
+
UUID id = connId.nodeId();
GridCommunicationClient[] nodeClients = clients.get(id);
@@ -481,20 +491,22 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati
if (rmtNode == null) {
DiscoverySpi discoverySpi = ignite().configuration().getDiscoverySpi();
- assert discoverySpi instanceof TcpDiscoverySpi;
-
- TcpDiscoverySpi tcpDiscoverySpi = (TcpDiscoverySpi) discoverySpi;
+ boolean unknownNode = true;
- ClusterNode node0 = tcpDiscoverySpi.getNode0(sndId);
+ if (discoverySpi instanceof TcpDiscoverySpi) {
+ TcpDiscoverySpi tcpDiscoverySpi = (TcpDiscoverySpi) discoverySpi;
- boolean unknownNode = true;
+ ClusterNode node0 = tcpDiscoverySpi.getNode0(sndId);
- if (node0 != null) {
- assert node0.isClient() : node0;
+ if (node0 != null) {
+ assert node0.isClient() : node0;
- if (node0.version().compareTo(VERSION_SINCE_CLIENT_COULD_WAIT_TO_CONNECT) >= 0)
- unknownNode = false;
+ if (node0.version().compareTo(VERSION_SINCE_CLIENT_COULD_WAIT_TO_CONNECT) >= 0)
+ unknownNode = false;
+ }
}
+ else if (discoverySpi instanceof IgniteDiscoverySpi)
+ unknownNode = !((IgniteDiscoverySpi) discoverySpi).knownNode(sndId);
if (unknownNode) {
U.warn(log, "Close incoming connection, unknown node [nodeId=" + sndId + ", ses=" + ses + ']');
@@ -709,9 +721,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati
}
}
else {
- metricsLsnr.onMessageReceived(msg, connKey.nodeId());
-
if (msg instanceof RecoveryLastReceivedMessage) {
+ metricsLsnr.onMessageReceived(msg, connKey.nodeId());
+
GridNioRecoveryDescriptor recovery = ses.outRecoveryDescriptor();
if (recovery != null) {
@@ -724,9 +736,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati
}
recovery.ackReceived(msg0.received());
-
- return;
}
+
+ return;
}
else {
GridNioRecoveryDescriptor recovery = ses.inRecoveryDescriptor();
@@ -746,8 +758,23 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati
recovery.lastAcknowledged(rcvCnt);
}
}
+ else if (connKey.dummy()) {
+ assert msg instanceof NodeIdMessage : msg;
+
+ TcpCommunicationNodeConnectionCheckFuture fut = ses.meta(SES_FUT_META);
+
+ assert fut != null : msg;
+
+ fut.onConnected(U.bytesToUuid(((NodeIdMessage)msg).nodeIdBytes(), 0));
+
+ nioSrvr.closeFromWorkerThread(ses);
+
+ return;
+ }
}
+ metricsLsnr.onMessageReceived(msg, connKey.nodeId());
+
IgniteRunnable c;
if (msgQueueLimit > 0) {
@@ -2112,6 +2139,13 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati
}
}
+ /**
+ * @return Bound TCP server port.
+ */
+ public int boundPort() {
+ return boundTcpPort;
+ }
+
/** {@inheritDoc} */
@Override public void spiStart(String igniteInstanceName) throws IgniteSpiException {
assert locHost != null;
@@ -2570,6 +2604,27 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati
}
/**
+ * @param nodes Nodes to check connection with.
+ * @return Result future (each bit in result BitSet contains connection status to corresponding node).
+ */
+ public IgniteFuture<BitSet> checkConnection(List<ClusterNode> nodes) {
+ TcpCommunicationConnectionCheckFuture fut = new TcpCommunicationConnectionCheckFuture(
+ this,
+ log.getLogger(TcpCommunicationConnectionCheckFuture.class),
+ nioSrvr,
+ nodes);
+
+ long timeout = failureDetectionTimeoutEnabled() ? failureDetectionTimeout() : connTimeout;
+
+ if (log.isInfoEnabled())
+ log.info("Start check connection process [nodeCnt=" + nodes.size() + ", timeout=" + timeout + ']');
+
+ fut.init(timeout);
+
+ return new IgniteFutureImpl<>(fut);
+ }
+
+ /**
* Sends given message to destination node. Note that characteristics of the
* exchange such as durability, guaranteed delivery or error notification is
* dependant on SPI implementation.
@@ -3010,7 +3065,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati
ConnectionKey id = ses.meta(CONN_IDX_META);
if (id != null) {
- ClusterNode node = getSpiContext().node(id.nodeId);
+ ClusterNode node = getSpiContext().node(id.nodeId());
if (node != null && node.isClient()) {
String msg = "Client node outbound message queue size exceeded slowClientQueueLimit, " +
@@ -3031,9 +3086,20 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati
/**
* @param node Node.
* @return Node addresses.
+ * @throws IgniteCheckedException If failed.
+ */
+ private Collection<InetSocketAddress> nodeAddresses(ClusterNode node) throws IgniteCheckedException {
+ return nodeAddresses(node, filterReachableAddresses);
+ }
+
+ /**
+ * @param node Node.
+ * @param filterReachableAddresses Filter addresses flag.
+ * @return Node addresses.
* @throws IgniteCheckedException If node does not have addresses.
*/
- private LinkedHashSet<InetSocketAddress> nodeAddresses(ClusterNode node) throws IgniteCheckedException {
+ public Collection<InetSocketAddress> nodeAddresses(ClusterNode node, boolean filterReachableAddresses)
+ throws IgniteCheckedException {
Collection<String> rmtAddrs0 = node.attribute(createSpiAttributeName(ATTR_ADDRS));
Collection<String> rmtHostNames0 = node.attribute(createSpiAttributeName(ATTR_HOST_NAMES));
Integer boundPort = node.attribute(createSpiAttributeName(ATTR_PORT));
@@ -3114,7 +3180,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati
* @throws IgniteCheckedException If failed.
*/
protected GridCommunicationClient createTcpClient(ClusterNode node, int connIdx) throws IgniteCheckedException {
- LinkedHashSet<InetSocketAddress> addrs = nodeAddresses(node);
+ Collection<InetSocketAddress> addrs = nodeAddresses(node);
GridCommunicationClient client = null;
IgniteCheckedException errs = null;
@@ -3132,6 +3198,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati
int lastWaitingTimeout = 1;
while (client == null) { // Reconnection on handshake timeout.
+ if (stopping)
+ throw new IgniteSpiException("Node is stopping.");
+
if (addr.getAddress().isLoopbackAddress() && addr.getPort() == boundTcpPort) {
if (log.isDebugEnabled())
log.debug("Skipping local address [addr=" + addr +
@@ -3372,8 +3441,18 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati
"operating system firewall is disabled on local and remote hosts) " +
"[addrs=" + addrs + ']');
- if (enableForcibleNodeKill) {
- if (getSpiContext().node(node.id()) != null
+ boolean commErrResolve = false;
+
+ IgniteSpiContext ctx = getSpiContext();
+
+ if (connectionError(errs) && ctx.communicationFailureResolveSupported()) {
+ commErrResolve = true;
+
+ ctx.resolveCommunicationFailure(node, errs);
+ }
+
+ if (!commErrResolve && enableForcibleNodeKill) {
+ if (ctx.node(node.id()) != null
&& (CU.clientNode(node) || !CU.clientNode(getLocalNode())) &&
connectionError(errs)) {
String msg = "TcpCommunicationSpi failed to establish connection to node, node will be dropped from " +
@@ -3384,7 +3463,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati
else
U.warn(log, msg);
- getSpiContext().failNode(node.id(), "TcpCommunicationSpi failed to establish connection to node [" +
+ ctx.failNode(node.id(), "TcpCommunicationSpi failed to establish connection to node [" +
"rmtNode=" + node +
", errs=" + errs +
", connectErrs=" + Arrays.toString(errs.getSuppressed()) + ']');
@@ -4590,77 +4669,6 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati
/**
*
*/
- private static class ConnectionKey {
- /** */
- private final UUID nodeId;
-
- /** */
- private final int idx;
-
- /** */
- private final long connCnt;
-
- /**
- * @param nodeId Node ID.
- * @param idx Connection index.
- * @param connCnt Connection counter (set only for incoming connections).
- */
- ConnectionKey(UUID nodeId, int idx, long connCnt) {
- this.nodeId = nodeId;
- this.idx = idx;
- this.connCnt = connCnt;
- }
-
- /**
- * @return Connection counter.
- */
- long connectCount() {
- return connCnt;
- }
-
- /**
- * @return Node ID.
- */
- UUID nodeId() {
- return nodeId;
- }
-
- /**
- * @return Connection index.
- */
- int connectionIndex() {
- return idx;
- }
-
- /** {@inheritDoc} */
- @Override public boolean equals(Object o) {
- if (this == o)
- return true;
-
- if (o == null || getClass() != o.getClass())
- return false;
-
- ConnectionKey key = (ConnectionKey) o;
-
- return idx == key.idx && nodeId.equals(key.nodeId);
- }
-
- /** {@inheritDoc} */
- @Override public int hashCode() {
- int res = nodeId.hashCode();
- res = 31 * res + idx;
- return res;
- }
-
- /** {@inheritDoc} */
- @Override public String toString() {
- return S.toString(ConnectionKey.class, this);
- }
- }
-
- /**
- *
- */
interface ConnectionPolicy {
/**
* @return Thread connection index.
http://git-wip-us.apache.org/repos/asf/ignite/blob/a64b941d/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/ConnectionKey.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/ConnectionKey.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/ConnectionKey.java
new file mode 100644
index 0000000..0559df7
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/ConnectionKey.java
@@ -0,0 +1,117 @@
+/*
+ * 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.spi.communication.tcp.internal;
+
+import java.util.UUID;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Connection Key.
+ */
+public class ConnectionKey {
+ /** */
+ private final UUID nodeId;
+
+ /** */
+ private final int idx;
+
+ /** */
+ private final long connCnt;
+
+ /** */
+ private final boolean dummy;
+
+ /**
+ * Creates ConnectionKey with false value of dummy flag.
+ *
+ * @param nodeId Node ID. Should be not null.
+ * @param idx Connection index.
+ * @param connCnt Connection counter (set only for incoming connections).
+ */
+ public ConnectionKey(@NotNull UUID nodeId, int idx, long connCnt) {
+ this(nodeId, idx, connCnt, false);
+ }
+
+ /**
+ * @param nodeId Node ID. Should be not null.
+ * @param idx Connection index.
+ * @param connCnt Connection counter (set only for incoming connections).
+ * @param dummy Indicates that session with this ConnectionKey is temporary
+ * (for now dummy sessions are used only for Communication Failure Resolving process).
+ */
+ public ConnectionKey(@NotNull UUID nodeId, int idx, long connCnt, boolean dummy) {
+ this.nodeId = nodeId;
+ this.idx = idx;
+ this.connCnt = connCnt;
+ this.dummy = dummy;
+ }
+
+ /**
+ * @return Connection counter.
+ */
+ public long connectCount() {
+ return connCnt;
+ }
+
+ /**
+ * @return Node ID.
+ */
+ public UUID nodeId() {
+ return nodeId;
+ }
+
+ /**
+ * @return Connection index.
+ */
+ public int connectionIndex() {
+ return idx;
+ }
+
+ /**
+ * @return {@code True} if this ConnectionKey is dummy and serves temporary session.
+ */
+ public boolean dummy() {
+ return dummy;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean equals(Object o) {
+ if (this == o)
+ return true;
+
+ if (o == null || getClass() != o.getClass())
+ return false;
+
+ ConnectionKey key = (ConnectionKey) o;
+
+ return idx == key.idx && nodeId.equals(key.nodeId);
+ }
+
+ /** {@inheritDoc} */
+ @Override public int hashCode() {
+ int res = nodeId.hashCode();
+ res = 31 * res + idx;
+ return res;
+ }
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return S.toString(ConnectionKey.class, this);
+ }
+}