# ignite-sprint-3 reverted fix for ignite-537 (cache indexing with p2p did not work properly)
Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/0b248129 Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/0b248129 Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/0b248129 Branch: refs/heads/ignite-sprint-4 Commit: 0b24812941206112ae62cb2b86db0095f3d87bff Parents: df1fbef Author: sboikov <semen.boi...@inria.fr> Authored: Sat Apr 11 10:40:43 2015 +0300 Committer: sboikov <semen.boi...@inria.fr> Committed: Sat Apr 11 10:45:39 2015 +0300 ---------------------------------------------------------------------- .../spi/discovery/tcp/TcpDiscoverySpi.java | 385 +------------------ .../tcp/messages/TcpDiscoveryClassRequest.java | 55 --- .../tcp/messages/TcpDiscoveryClassResponse.java | 76 ---- .../CacheConfigurationP2PTestClient.java | 14 +- .../cache/CacheConfigurationP2PTest.java | 2 + 5 files changed, 32 insertions(+), 500 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0b248129/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 81c608a..ed4be4e 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 @@ -51,7 +51,6 @@ import java.net.*; import java.text.*; import java.util.*; import java.util.concurrent.*; -import java.util.concurrent.locks.*; import static org.apache.ignite.events.EventType.*; import static org.apache.ignite.internal.IgniteNodeAttributes.*; @@ -239,9 +238,9 @@ public class TcpDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpDiscov @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized") private CheckStatusSender chkStatusSnd; - /** IP finder and p2p loaders cleaner. */ + /** IP finder cleaner. */ @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized") - private DiscoveryCleaner cleaner; + private IpFinderCleaner ipFinderCleaner; /** Statistics printer thread. */ @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized") @@ -289,10 +288,6 @@ public class TcpDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpDiscov @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized") private ConcurrentLinkedDeque<String> debugLog; - /** Class loaders for event data unmarshalling. */ - @GridToStringExclude - private transient ConcurrentMap<UUID, DiscoveryDeploymentClassLoader> p2pLdrs = new ConcurrentHashMap<>(); - /** {@inheritDoc} */ @IgniteInstanceResource @Override public void injectResources(Ignite ignite) { @@ -765,9 +760,9 @@ public class TcpDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpDiscov chkStatusSnd = new CheckStatusSender(); chkStatusSnd.start(); - if (ipFinder.isShared() || ignite.configuration().isPeerClassLoadingEnabled()) { - cleaner = new DiscoveryCleaner(); - cleaner.start(); + if (ipFinder.isShared()) { + ipFinderCleaner = new IpFinderCleaner(); + ipFinderCleaner.start(); } if (log.isDebugEnabled() && !restart) @@ -982,8 +977,8 @@ public class TcpDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpDiscov U.interrupt(chkStatusSnd); U.join(chkStatusSnd, log); - U.interrupt(cleaner); - U.join(cleaner, log); + U.interrupt(ipFinderCleaner); + U.join(ipFinderCleaner, log); U.interrupt(msgWorker); U.join(msgWorker, log); @@ -2011,8 +2006,8 @@ public class TcpDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpDiscov U.interrupt(chkStatusSnd); U.join(chkStatusSnd, log); - U.interrupt(cleaner); - U.join(cleaner, log); + U.interrupt(ipFinderCleaner); + U.join(ipFinderCleaner, log); Collection<SocketReader> tmp; @@ -2110,7 +2105,7 @@ public class TcpDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpDiscov b.append(" Check status sender: ").append(threadStatus(chkStatusSnd)).append(U.nl()); b.append(" HB sender: ").append(threadStatus(hbsSnd)).append(U.nl()); b.append(" Socket timeout worker: ").append(threadStatus(sockTimeoutWorker)).append(U.nl()); - b.append(" Cleaner: ").append(threadStatus(cleaner)).append(U.nl()); + b.append(" IP finder cleaner: ").append(threadStatus(ipFinderCleaner)).append(U.nl()); b.append(" Stats printer: ").append(threadStatus(statsPrinter)).append(U.nl()); b.append(U.nl()); @@ -2208,120 +2203,6 @@ public class TcpDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpDiscov } /** - * @param req Get class request. - * @return Get class response. - */ - private TcpDiscoveryClassResponse processClassRequest(TcpDiscoveryClassRequest req) { - assert !F.isEmpty(req.className()) : req; - - String rsrc = U.classNameToResourceName(req.className()); - - InputStream in = U.gridClassLoader().getResourceAsStream(rsrc); - - byte[] clsBytes = null; - String err = null; - - if (in != null) { - try { - GridByteArrayList bytes = new GridByteArrayList(1024); - - bytes.readAll(in); - - clsBytes = bytes.entireArray(); - } - catch (IOException e) { - err = "Failed to load class due IO error [cls=" + req.className() + ", err=" + e + ']'; - - U.error(log, err, e); - } - finally { - U.close(in, log); - } - } - else { - if (log.isDebugEnabled()) - log.debug("Failed to find requested class: " + req.className()); - - err = "Failed to find requested class: " + req.className(); - } - - TcpDiscoveryClassResponse res; - - if (clsBytes != null) - res = new TcpDiscoveryClassResponse(getLocalNodeId(), clsBytes); - else { - assert err != null; - - res = new TcpDiscoveryClassResponse(getLocalNodeId(), err); - } - - return res; - } - - /** - * @param node Node created event. - * @return Class loader for custom event unmarshalling. - */ - @Nullable protected ClassLoader customMessageClassLoader(TcpDiscoveryNode node) { - assert ignite != null; - - if (!ignite.configuration().isPeerClassLoadingEnabled()) - return U.gridClassLoader(); - - if (node.id().equals(getLocalNodeId()) || node.isClient()) - return U.gridClassLoader(); - - DiscoveryDeploymentClassLoader ldr = p2pLdrs.get(node.id()); - - if (ldr == null) - ldr = F.addIfAbsent(p2pLdrs, node.id(), new DiscoveryDeploymentClassLoader(node)); - - return ldr; - } - - /** - * @param joiningNode Joining node. - * @param nodeId Remote node provided data. - * @return Class loader for exchange data unmarshalling. - */ - @Nullable protected ClassLoader exchangeClassLoader(TcpDiscoveryNode joiningNode, UUID nodeId) { - assert joiningNode != null; - assert ignite != null; - - if (!ignite.configuration().isPeerClassLoadingEnabled()) - return U.gridClassLoader(); - - if (nodeId.equals(getLocalNodeId())) - return U.gridClassLoader(); - - TcpDiscoveryNode node; - - if (joiningNode.id().equals(nodeId)) - node = joiningNode; - else { - node = ring.node(nodeId); - - if (node == null) { - if (log.isDebugEnabled()) - log.debug("Node provided exchange data left, will use local class loader " + - "for exchange data [nodeId=" + nodeId + ']'); - - return U.gridClassLoader(); - } - } - - if (node.isClient()) // Do not support loading from client nodes. - return U.gridClassLoader(); - - DiscoveryDeploymentClassLoader ldr = p2pLdrs.get(nodeId); - - if (ldr == null) - ldr = F.addIfAbsent(p2pLdrs, nodeId, new DiscoveryDeploymentClassLoader(node)); - - return ldr; - } - - /** * @param nodeId Node ID. * @return Marshalled exchange data. * @throws IgniteSpiException If failed. @@ -2471,19 +2352,18 @@ public class TcpDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpDiscov } /** - * Thread that periodically tries to release p2p class loaders connections, cleans - * IP finder and keeps it in the correct state, unregistering addresses of the nodes - * that has left the topology. + * Thread that cleans IP finder and keeps it in the correct state, unregistering + * addresses of the nodes that has left the topology. * <p> - * IP finder cleaner should run only on coordinator node and will clean IP finder + * This thread should run only on coordinator node and will clean IP finder * if and only if {@link TcpDiscoveryIpFinder#isShared()} is {@code true}. */ - private class DiscoveryCleaner extends IgniteSpiThread { + private class IpFinderCleaner extends IgniteSpiThread { /** * Constructor. */ - private DiscoveryCleaner() { - super(gridName, "tcp-disco-cleaner", log); + private IpFinderCleaner() { + super(gridName, "tcp-disco-ip-finder-cleaner", log); setPriority(threadPri); } @@ -2492,21 +2372,11 @@ public class TcpDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpDiscov @SuppressWarnings("BusyWait") @Override protected void body() throws InterruptedException { if (log.isDebugEnabled()) - log.debug("Tcp discovery cleaner has been started."); + log.debug("IP finder cleaner has been started."); while (!isInterrupted()) { Thread.sleep(ipFinderCleanFreq); - for (DiscoveryDeploymentClassLoader ldr : p2pLdrs.values()) { - if (ring.node(ldr.nodeId()) == null) { - ldr.onNodeLeft(); - - p2pLdrs.remove(ldr.nodeId(), ldr); - } - else - ldr.closeConnectionIfNotUsed(); - } - if (!isLocalNodeCoordinator()) continue; @@ -3723,7 +3593,7 @@ public class TcpDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpDiscov Map<Integer, byte[]> data = msg.newNodeDiscoveryData(); if (data != null) - onExchange(node.id(), node.id(), data, exchangeClassLoader(node, node.id())); + onExchange(node.id(), node.id(), data, U.gridClassLoader()); msg.addDiscoveryData(locNodeId, collectExchangeData(node.id())); } @@ -3794,12 +3664,8 @@ public class TcpDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpDiscov // Notify outside of synchronized block. if (dataMap != null) { - for (Map.Entry<UUID, Map<Integer, byte[]>> entry : dataMap.entrySet()) { - onExchange(node.id(), - entry.getKey(), - entry.getValue(), - exchangeClassLoader(node, entry.getKey())); - } + for (Map.Entry<UUID, Map<Integer, byte[]>> entry : dataMap.entrySet()) + onExchange(node.id(), entry.getKey(), entry.getValue(), U.gridClassLoader()); } } @@ -4603,7 +4469,7 @@ public class TcpDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpDiscov try { if (msgObj == null) - msgObj = marsh.unmarshal(msg.messageBytes(), customMessageClassLoader(node)); + msgObj = marsh.unmarshal(msg.messageBytes(), U.gridClassLoader()); lsnr.onDiscovery(DiscoveryCustomEvent.EVT_DISCOVERY_CUSTOM_EVT, msg.topologyVersion(), @@ -5090,13 +4956,6 @@ public class TcpDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpDiscov continue; } - else if (msg instanceof TcpDiscoveryClassRequest) { - TcpDiscoveryClassResponse res = processClassRequest((TcpDiscoveryClassRequest) msg); - - writeToSocket(sock, res); - - continue; - } msgWorker.addMessage(msg); @@ -5388,206 +5247,4 @@ public class TcpDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpDiscov U.closeQuiet(sock); } } - - /** - * - */ - private class DiscoveryDeploymentClassLoader extends ClassLoader { - /** */ - private final TcpDiscoveryNode node; - - /** */ - private Socket sock; - - /** */ - private final ReadWriteLock lock = new ReentrantReadWriteLock(); - - /** - * @param node Node. - */ - public DiscoveryDeploymentClassLoader(TcpDiscoveryNode node) { - super(U.gridClassLoader()); - - assert !node.isClient() : node; - assert !node.id().equals(getLocalNodeId()) : node; - - this.node = node; - } - - /** - * @return Target node ID. - */ - UUID nodeId() { - return node.id(); - } - - /** - * Node left callback. - */ - void onNodeLeft() { - lock.writeLock().lock(); - - try { - if (sock != null) { - if (log.isDebugEnabled()) - log.debug("Closing deployment class loader connection on node left [node=" + node.id() + ']'); - - U.closeQuiet(sock); - - sock = null; - } - } - finally { - lock.writeLock().unlock(); - } - } - - /** - * Closes connection if there is no class loading in progress. - */ - void closeConnectionIfNotUsed() { - if (lock.writeLock().tryLock()) { - try { - if (sock != null) { - if (log.isDebugEnabled()) - log.debug("Closing idle deployment class loader connection [node=" + node.id() + ']'); - - U.closeQuiet(sock); - - sock = null; - } - } - finally { - lock.writeLock().unlock(); - } - } - } - - /** {@inheritDoc} */ - @Override protected Class<?> findClass(String name) throws ClassNotFoundException { - lock.readLock().lock(); - - try { - TcpDiscoveryClassResponse res = requestClass(name); - - if (res == null) - throw new ClassNotFoundException("Failed to load class, can not connect to peer node " + - "[cls=" + name + ", node=" + node.id() + ']'); - - if (res.error() != null) - throw new ClassNotFoundException(res.error()); - - assert res.classBytes() != null; - - return defineClass(name, res.classBytes(), 0, res.classBytes().length); - } - finally { - lock.readLock().unlock(); - } - } - - /** - * @param name Class name. - * @return Class response or {@code null} if failed to connect. - */ - @Nullable private synchronized TcpDiscoveryClassResponse requestClass(String name) { - TcpDiscoveryClassRequest msg = new TcpDiscoveryClassRequest(getLocalNodeId(), name); - - for (int i = 0; i < reconCnt; i++) { - if (sock == null) { - sock = connect(node); - - if (sock == null) - break; - } - - try { - return request(sock, msg); - } - catch (IOException | IgniteCheckedException e) { - U.closeQuiet(sock); - - sock = null; - } - } - - p2pLdrs.remove(node.id(), this); - - return null; - } - - /** - * @param sock Socket. - * @param msg Message. - * @return Response. - * @throws IOException If request failed. - * @throws IgniteCheckedException If request failed. - */ - private TcpDiscoveryClassResponse request(Socket sock, TcpDiscoveryClassRequest msg) - throws IOException, IgniteCheckedException - { - long tstamp = U.currentTimeMillis(); - - writeToSocket(sock, msg); - - stats.onMessageSent(msg, U.currentTimeMillis() - tstamp); - - TcpDiscoveryClassResponse res = readMessage(sock, null, netTimeout); - - stats.onMessageReceived(res); - - return res; - } - - /** - * @param node Node. - * @return Socket or {@code null} if failed to connect. - */ - private Socket connect(TcpDiscoveryNode node) { - Socket sock = null; - - for (InetSocketAddress addr : getNodeAddresses(node, U.sameMacs(locNode, node))) { - sock = connect(addr); - - if (sock != null) - break; - } - - return sock; - } - - /** - * @param addr Address. - * @return Socket or {@code null} if failed to connect. - */ - private Socket connect(InetSocketAddress addr) { - TcpDiscoveryHandshakeRequest req = new TcpDiscoveryHandshakeRequest(getLocalNodeId()); - - for (int i = 0; i < reconCnt; i++) { - Socket sock = null; - - long tstamp = U.currentTimeMillis(); - - try { - sock = openSocket(addr); - - writeToSocket(sock, req); - - TcpDiscoveryHandshakeResponse res = readMessage(sock, null, netTimeout); - - if (!res.creatorNodeId().equals(node.id())) - return null; - - stats.onClientSocketInitialized(U.currentTimeMillis() - tstamp); - - return sock; - } - catch (IOException | IgniteCheckedException e) { - U.closeQuiet(sock); - } - } - - return null; - } - } } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0b248129/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClassRequest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClassRequest.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClassRequest.java deleted file mode 100644 index befe483..0000000 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClassRequest.java +++ /dev/null @@ -1,55 +0,0 @@ -/* - * 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.tcp.messages; - -import org.apache.ignite.internal.util.typedef.internal.*; - -import java.util.*; - -/** - * - */ -public class TcpDiscoveryClassRequest extends TcpDiscoveryAbstractMessage { - /** */ - private static final long serialVersionUID = 0L; - - /** */ - private String clsName; - - /** - * @param creatorNodeId Creator node ID. - * @param clsName Class name. - */ - public TcpDiscoveryClassRequest(UUID creatorNodeId, String clsName) { - super(creatorNodeId); - - this.clsName = clsName; - } - - /** - * @return Class name. - */ - public String className() { - return clsName; - } - - /** {@inheritDoc} */ - @Override public String toString() { - return S.toString(TcpDiscoveryClassRequest.class, this, "super", super.toString()); - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0b248129/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClassResponse.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClassResponse.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClassResponse.java deleted file mode 100644 index 3bfe61d..0000000 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClassResponse.java +++ /dev/null @@ -1,76 +0,0 @@ -/* - * 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.tcp.messages; - -import org.apache.ignite.internal.util.typedef.internal.*; -import org.jetbrains.annotations.*; - -import java.util.*; - -/** - * - */ -public class TcpDiscoveryClassResponse extends TcpDiscoveryAbstractMessage { - /** */ - private static final long serialVersionUID = 0L; - - /** */ - private String errMsg; - - /** */ - private byte[] clsBytes; - - /** - * @param creatorNodeId Creator node ID. - * @param clsBytes Class bytes. - */ - public TcpDiscoveryClassResponse(UUID creatorNodeId, byte[] clsBytes) { - super(creatorNodeId); - - this.clsBytes = clsBytes; - } - - /** - * @param creatorNodeId Creator node ID. - * @param errMsg Error message. - */ - public TcpDiscoveryClassResponse(UUID creatorNodeId, String errMsg) { - super(creatorNodeId); - - this.errMsg = errMsg; - } - - /** - * @return Error if class loading failed. - */ - @Nullable public String error() { - return errMsg; - } - - /** - * @return Loaded class bytes. - */ - public byte[] classBytes() { - return clsBytes; - } - - /** {@inheritDoc} */ - @Override public String toString() { - return S.toString(TcpDiscoveryClassResponse.class, this, "super", super.toString()); - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0b248129/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/startcache/CacheConfigurationP2PTestClient.java ---------------------------------------------------------------------- diff --git a/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/startcache/CacheConfigurationP2PTestClient.java b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/startcache/CacheConfigurationP2PTestClient.java index be3f3bc..4550e21 100644 --- a/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/startcache/CacheConfigurationP2PTestClient.java +++ b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/startcache/CacheConfigurationP2PTestClient.java @@ -80,8 +80,10 @@ public class CacheConfigurationP2PTestClient { SqlQuery<Integer, Organization1> qry1 = new SqlQuery<>(Organization1.class, "_key >= 0"); - if (cache1.query(qry1).getAll().isEmpty()) - throw new Exception("Query failed."); + int cnt = cache1.query(qry1).getAll().size(); + + if (cnt != 500) + throw new Exception("Unexpected query result: " + cnt); System.out.println("Sleep some time."); @@ -97,13 +99,15 @@ public class CacheConfigurationP2PTestClient { IgniteCache<Integer, Organization2> cache2 = ignite.createCache(ccfg2); - for (int i = 0; i < 500; i++) + for (int i = 0; i < 600; i++) cache2.put(i, new Organization2("org-" + i)); SqlQuery<Integer, Organization2> qry2 = new SqlQuery<>(Organization2.class, "_key >= 0"); - if (cache2.query(qry2).getAll().isEmpty()) - throw new Exception("Query failed."); + cnt = cache2.query(qry2).getAll().size(); + + if (cnt != 600) + throw new Exception("Unexpected query result: " + cnt); cache1.close(); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0b248129/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheConfigurationP2PTest.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheConfigurationP2PTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheConfigurationP2PTest.java index 8f0f4ca..051b94f 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheConfigurationP2PTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheConfigurationP2PTest.java @@ -74,6 +74,8 @@ public class CacheConfigurationP2PTest extends GridCommonAbstractTest { * @throws Exception If failed. */ public void testCacheConfigurationP2P() throws Exception { + fail("Enable when IGNITE-537 is fixed."); + try (Ignite ignite = Ignition.start(createConfiguration())) { final CountDownLatch srvsReadyLatch = new CountDownLatch(2);