From a2b4751f5eefd70a5a1aa26652c9671240125f78 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Fri, 17 Mar 2017 14:57:48 +0300 Subject: [PATCH 01/90] IGNITE-4473 - Client should re-try connection attempt in case of concurrent network failure. (cherry picked from commit d124004) --- .../internal/GridKernalGatewayImpl.java | 8 +- .../apache/ignite/internal/IgniteKernal.java | 120 +++++- .../IgniteNeedReconnectException.java | 40 ++ .../discovery/GridDiscoveryManager.java | 24 ++ .../GridCachePartitionExchangeManager.java | 25 +- .../dht/GridDhtAssignmentFetchFuture.java | 14 +- .../GridDhtPartitionsExchangeFuture.java | 48 ++- .../service/GridServiceProcessor.java | 86 ++-- .../ignite/spi/discovery/tcp/ClientImpl.java | 201 ++++++++-- .../ignite/spi/discovery/tcp/ServerImpl.java | 5 + .../spi/discovery/tcp/TcpDiscoveryImpl.java | 8 + .../spi/discovery/tcp/TcpDiscoverySpi.java | 9 + .../IgniteClientReconnectCacheTest.java | 7 +- .../internal/IgniteClientRejoinTest.java | 378 ++++++++++++++++++ .../tcp/TcpClientDiscoverySpiSelfTest.java | 48 ++- .../IgniteClientReconnectTestSuite.java | 2 + 16 files changed, 929 insertions(+), 94 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/IgniteNeedReconnectException.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/IgniteClientRejoinTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalGatewayImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalGatewayImpl.java index fe8c580ca63ea..036954a3a280d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalGatewayImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalGatewayImpl.java @@ -44,7 +44,7 @@ public class GridKernalGatewayImpl implements GridKernalGateway, Serializable { /** */ @GridToStringExclude - private IgniteFutureImpl reconnectFut; + private volatile IgniteFutureImpl reconnectFut; /** */ private final AtomicReference state = new AtomicReference<>(GridKernalState.STOPPED); @@ -149,6 +149,12 @@ public GridKernalGatewayImpl(String gridName) { /** {@inheritDoc} */ @Override public GridFutureAdapter onDisconnected() { + if (state.get() == GridKernalState.DISCONNECTED) { + assert reconnectFut != null; + + return (GridFutureAdapter)reconnectFut.internalFuture(); + } + GridFutureAdapter fut = new GridFutureAdapter<>(); reconnectFut = new IgniteFutureImpl<>(fut); 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 8fda72fc18419..25f7884c889b9 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 @@ -250,6 +250,9 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable { /** Periodic starvation check interval. */ private static final long PERIODIC_STARVATION_CHECK_FREQ = 1000 * 30; + /** Force complete reconnect future. */ + private static final Object STOP_RECONNECT = new Object(); + /** */ @GridToStringExclude private GridKernalContextImpl ctx; @@ -327,6 +330,9 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable { @GridToStringExclude private final AtomicBoolean stopGuard = new AtomicBoolean(); + /** */ + private final ReconnectState reconnectState = new ReconnectState(); + /** * No-arg constructor is required by externalization. */ @@ -930,6 +936,8 @@ public void start(final IgniteConfiguration cfg, // Notify IO manager the second so further components can send and receive messages. ctx.io().onKernalStart(); + boolean recon = false; + // Callbacks. for (GridComponent comp : ctx) { // Skip discovery manager. @@ -940,10 +948,24 @@ public void start(final IgniteConfiguration cfg, if (comp instanceof GridIoManager) continue; - if (!skipDaemon(comp)) - comp.onKernalStart(); + if (!skipDaemon(comp)) { + try { + comp.onKernalStart(); + } + catch (IgniteNeedReconnectException e) { + assert ctx.discovery().reconnectSupported(); + + if (log.isDebugEnabled()) + log.debug("Failed to start node components on node start, will wait for reconnect: " + e); + + recon = true; + } + } } + if (recon) + reconnectState.waitFirstReconnect(); + // Register MBeans. registerKernalMBean(); registerLocalNodeMBean(); @@ -3274,6 +3296,8 @@ private void unguard() { public void onDisconnected() { Throwable err = null; + reconnectState.waitPreviousReconnect(); + GridFutureAdapter reconnectFut = ctx.gateway().onDisconnected(); if (reconnectFut == null) { @@ -3282,9 +3306,18 @@ public void onDisconnected() { return; } - IgniteFuture userFut = new IgniteFutureImpl<>(reconnectFut); + IgniteFutureImpl curFut = (IgniteFutureImpl)ctx.cluster().get().clientReconnectFuture(); + + IgniteFuture userFut; - ctx.cluster().get().clientReconnectFuture(userFut); + // In case of previous reconnect did not finish keep reconnect future. + if (curFut != null && curFut.internalFuture() == reconnectFut) + userFut = curFut; + else { + userFut = new IgniteFutureImpl<>(reconnectFut); + + ctx.cluster().get().clientReconnectFuture(userFut); + } ctx.disconnected(true); @@ -3337,30 +3370,53 @@ public void onReconnected(final boolean clusterRestarted) { try { ctx.disconnected(false); - GridCompoundFuture reconnectFut = new GridCompoundFuture<>(); + GridCompoundFuture curReconnectFut = reconnectState.curReconnectFut = new GridCompoundFuture<>(); + + reconnectState.reconnectDone = new GridFutureAdapter<>(); for (GridComponent comp : ctx.components()) { IgniteInternalFuture fut = comp.onReconnected(clusterRestarted); if (fut != null) - reconnectFut.add((IgniteInternalFuture)fut); + curReconnectFut.add(fut); } - reconnectFut.add((IgniteInternalFuture)ctx.cache().context().exchange().reconnectExchangeFuture()); + curReconnectFut.add(ctx.cache().context().exchange().reconnectExchangeFuture()); + + curReconnectFut.markInitialized(); - reconnectFut.markInitialized(); + final GridFutureAdapter reconnectDone = reconnectState.reconnectDone; - reconnectFut.listen(new CI1>() { + curReconnectFut.listen(new CI1>() { @Override public void apply(IgniteInternalFuture fut) { try { - fut.get(); + Object res = fut.get(); + + if (res == STOP_RECONNECT) + return; ctx.gateway().onReconnected(); + + reconnectState.firstReconnectFut.onDone(); } catch (IgniteCheckedException e) { - U.error(log, "Failed to reconnect, will stop node", e); + if (!X.hasCause(e, IgniteNeedReconnectException.class, + IgniteClientDisconnectedCheckedException.class)) { + U.error(log, "Failed to reconnect, will stop node.", e); + + reconnectState.firstReconnectFut.onDone(e); - close(); + close(); + } + else { + assert ctx.discovery().reconnectSupported(); + + U.error(log, "Failed to finish reconnect, will retry [locNodeId=" + ctx.localNodeId() + + ", err=" + e.getMessage() + ']'); + } + } + finally { + reconnectDone.onDone(); } } }); @@ -3519,6 +3575,46 @@ public void dumpDebugInfo() { } } + /** + * + */ + private class ReconnectState { + /** */ + private final GridFutureAdapter firstReconnectFut = new GridFutureAdapter(); + + /** */ + private GridCompoundFuture curReconnectFut; + + /** */ + private GridFutureAdapter reconnectDone; + + /** + * @throws IgniteCheckedException If failed. + */ + void waitFirstReconnect() throws IgniteCheckedException { + firstReconnectFut.get(); + } + + /** + * + */ + void waitPreviousReconnect() { + if (curReconnectFut != null && !curReconnectFut.isDone()) { + assert reconnectDone != null; + + curReconnectFut.onDone(STOP_RECONNECT); + + try { + reconnectDone.get(); + } + catch (IgniteCheckedException ignote) { + // No-op. + } + } + + } + } + /** {@inheritDoc} */ @Override public String toString() { return S.toString(IgniteKernal.class, this); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteNeedReconnectException.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteNeedReconnectException.java new file mode 100644 index 0000000000000..61ab5762b95ac --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteNeedReconnectException.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.internal; + +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.cluster.ClusterNode; +import org.jetbrains.annotations.Nullable; + +/** + * Indicates that node should try reconnect to cluster. + */ +public class IgniteNeedReconnectException extends IgniteCheckedException { + /** */ + private static final long serialVersionUID = 0L; + + /** + * @param locNode Local node. + * @param cause Cause. + */ + public IgniteNeedReconnectException(ClusterNode locNode, @Nullable Throwable cause) { + super("Local node need try to reconnect [locNodeId=" + locNode.id() + ']', cause); + + assert locNode.isClient(); + } +} 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 9aa4db1e0042f..2ec10705bedbb 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 @@ -112,6 +112,7 @@ import org.apache.ignite.spi.discovery.DiscoverySpiListener; 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.thread.IgniteThread; import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentHashMap8; @@ -1890,6 +1891,29 @@ public void failNode(UUID nodeId, @Nullable String warning) { } } + /** + * @return {@code True} if local node client and discovery SPI supports reconnect. + */ + public boolean reconnectSupported() { + DiscoverySpi spi = getSpi(); + + return ctx.clientNode() && (spi instanceof TcpDiscoverySpi) && + !(((TcpDiscoverySpi) spi).isClientReconnectDisabled()); + } + + /** + * Leave cluster and try to join again. + * + * @throws IgniteSpiException If failed. + */ + public void reconnect() { + assert reconnectSupported(); + + DiscoverySpi discoverySpi = getSpi(); + + ((TcpDiscoverySpi)discoverySpi).reconnect(); + } + /** * Updates topology version if current version is smaller than updated. * 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 7f11dc45aacb7..92142c0731240 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 @@ -43,6 +43,7 @@ import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.IgniteNeedReconnectException; import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.cache.affinity.AffinityFunction; import org.apache.ignite.cluster.ClusterNode; @@ -447,6 +448,15 @@ else if (m instanceof GridDhtPartitionDemandMessage) else U.warn(log, "Still waiting for initial partition map exchange [fut=" + fut + ']'); } + catch (IgniteNeedReconnectException e) { + throw e; + } + catch (Exception e) { + if (fut.reconnectOnError(e)) + throw new IgniteNeedReconnectException(cctx.localNode(), e); + + throw e; + } } for (GridCacheContext cacheCtx : cctx.cacheContexts()) { @@ -1690,6 +1700,12 @@ void addFuture(GridDhtPartitionsExchangeFuture exchFut) { dumpedObjects++; } } + catch (Exception e) { + if (exchFut.reconnectOnError(e)) + throw new IgniteNeedReconnectException(cctx.localNode(), e); + + throw e; + } } @@ -1829,7 +1845,14 @@ else if (r != null) { catch (IgniteInterruptedCheckedException e) { throw e; } - catch (IgniteClientDisconnectedCheckedException e) { + catch (IgniteClientDisconnectedCheckedException | IgniteNeedReconnectException e) { + assert cctx.discovery().reconnectSupported(); + + U.warn(log,"Local node failed to complete partition map exchange due to " + + "network issues, will try to reconnect to cluster", e); + + cctx.discovery().reconnect(); + return; } catch (IgniteCheckedException e) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAssignmentFetchFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAssignmentFetchFuture.java index ab8e863e90e7d..6425bc141245d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAssignmentFetchFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAssignmentFetchFuture.java @@ -17,15 +17,16 @@ package org.apache.ignite.internal.processors.cache.distributed.dht; +import java.io.IOException; import java.util.Collection; import java.util.Collections; import java.util.LinkedList; import java.util.Queue; import java.util.UUID; -import java.util.concurrent.Executor; import java.util.concurrent.atomic.AtomicReference; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.IgniteNeedReconnectException; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.internal.GridNodeOrderComparator; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; @@ -34,6 +35,7 @@ import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.T2; +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; @@ -202,8 +204,14 @@ private void requestFromNextNode() { "continue to another node): " + node); } catch (IgniteCheckedException e) { - U.error(log0, "Failed to request affinity assignment from remote node (will " + - "continue to another node): " + node, e); + if (ctx.discovery().reconnectSupported() && X.hasCause(e, IOException.class)) { + onDone(new IgniteNeedReconnectException(ctx.localNode(), e)); + + return; + } + + U.warn(log0, "Failed to request affinity assignment from remote node (will " + + "continue to another node): " + node); } } 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 e945de958a339..d4f95e5b55d05 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 @@ -17,6 +17,7 @@ package org.apache.ignite.internal.processors.cache.distributed.dht.preloader; +import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -32,6 +33,7 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.ReadWriteLock; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.IgniteNeedReconnectException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.cluster.ClusterNode; @@ -39,6 +41,7 @@ import org.apache.ignite.events.DiscoveryEvent; import org.apache.ignite.events.Event; import org.apache.ignite.events.EventType; +import org.apache.ignite.internal.IgniteClientDisconnectedCheckedException; import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; @@ -54,7 +57,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.distributed.dht.GridClientPartitionTopology; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFuture; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey; @@ -65,7 +67,7 @@ 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.T2; +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.LT; import org.apache.ignite.internal.util.typedef.internal.S; @@ -506,10 +508,17 @@ public void init() throws IgniteInterruptedCheckedException { throw e; } + catch (IgniteNeedReconnectException e) { + onDone(e); + } catch (Throwable e) { - U.error(log, "Failed to reinitialize local partitions (preloading will be stopped): " + exchId, e); + if (reconnectOnError(e)) + onDone(new IgniteNeedReconnectException(cctx.localNode(), e)); + else { + U.error(log, "Failed to reinitialize local partitions (preloading will be stopped): " + exchId, e); - onDone(e); + onDone(e); + } if (e instanceof Error) throw (Error)e; @@ -1297,7 +1306,10 @@ private void onAllReceived(boolean discoThread) { } } catch (IgniteCheckedException e) { - onDone(e); + if (reconnectOnError(e)) + onDone(new IgniteNeedReconnectException(cctx.localNode(), e)); + else + onDone(e); } } @@ -1314,8 +1326,15 @@ private void sendAllPartitions(final UUID nodeId, final int retryCnt) { } catch (IgniteCheckedException e) { if (e instanceof ClusterTopologyCheckedException || !cctx.discovery().alive(n)) { - log.debug("Failed to send full partition map to node, node left grid " + - "[rmtNode=" + nodeId + ", exchangeId=" + exchId + ']'); + if (log.isDebugEnabled()) + log.debug("Failed to send full partition map to node, node left grid " + + "[rmtNode=" + nodeId + ", exchangeId=" + exchId + ']'); + + return; + } + + if (reconnectOnError(e)) { + onDone(new IgniteNeedReconnectException(cctx.localNode(), e)); return; } @@ -1641,6 +1660,12 @@ public void onNodeLeft(final ClusterNode node) { } } } + catch (Exception e) { + if (reconnectOnError(e)) + onDone(new IgniteNeedReconnectException(cctx.localNode(), e)); + else + throw e; + } finally { leaveBusy(); } @@ -1652,6 +1677,15 @@ public void onNodeLeft(final ClusterNode node) { } } + /** + * @param e Exception. + * @return {@code True} if local node should try reconnect in case of error. + */ + public boolean reconnectOnError(Throwable e) { + return X.hasCause(e, IOException.class, IgniteClientDisconnectedCheckedException.class) && + cctx.discovery().reconnectSupported(); + } + /** {@inheritDoc} */ @Override public int compareTo(GridDhtPartitionsExchangeFuture fut) { return exchId.compareTo(fut.exchId); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java index 6bcfd65cb59e0..bd815189fb5ba 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java @@ -1498,60 +1498,60 @@ private void processDeployment(CacheEntryEvent 0; + if (!newTopVer.equals(topVer)) { + assert newTopVer.compareTo(topVer) > 0; - // Reassignment will happen from topology event. - return; - } + // Reassignment will happen from topology event. + return; + } - ctx.timeout().addTimeoutObject(new GridTimeoutObject() { - private IgniteUuid id = IgniteUuid.randomUuid(); + ctx.timeout().addTimeoutObject(new GridTimeoutObject() { + private IgniteUuid id = IgniteUuid.randomUuid(); - private long start = System.currentTimeMillis(); + private long start = System.currentTimeMillis(); - @Override public IgniteUuid timeoutId() { - return id; - } + @Override public IgniteUuid timeoutId() { + return id; + } - @Override public long endTime() { - return start + RETRY_TIMEOUT; - } + @Override public long endTime() { + return start + RETRY_TIMEOUT; + } - @Override public void onTimeout() { - if (!busyLock.enterBusy()) - return; + @Override public void onTimeout() { + if (!busyLock.enterBusy()) + return; - try { - // Try again. - onDeployment(dep, topVer); - } - finally { - busyLock.leaveBusy(); - } + try { + // Try again. + onDeployment(dep, topVer); } - }); + finally { + busyLock.leaveBusy(); + } + } + }); } } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java index 95e2cda4f6907..02ba56a884b33 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java @@ -129,6 +129,9 @@ class ClientImpl extends TcpDiscoveryImpl { /** */ private static final Object SPI_RECONNECT_FAILED = "SPI_RECONNECT_FAILED"; + /** */ + private static final Object SPI_RECONNECT = "SPI_RECONNECT"; + /** Remote nodes. */ private final ConcurrentMap rmtNodes = new ConcurrentHashMap8<>(); @@ -808,6 +811,11 @@ private NavigableSet allVisibleNodes() { log); } + /** {@inheritDoc} */ + @Override public void reconnect() throws IgniteSpiException { + msgWorker.addMessage(SPI_RECONNECT); + } + /** {@inheritDoc} */ @Override public void brakeConnection() { SocketStream sockStream = msgWorker.currSock; @@ -879,9 +887,12 @@ private class SocketReader extends IgniteSpiThread { /** */ private UUID rmtNodeId; + /** */ + private CountDownLatch stopReadLatch; + /** */ - protected SocketReader() { + SocketReader() { super(spi.ignite().name(), "tcp-client-disco-sock-reader", log); } @@ -889,7 +900,7 @@ protected SocketReader() { * @param sockStream Socket. * @param rmtNodeId Rmt node id. */ - public void setSocket(SocketStream sockStream, UUID rmtNodeId) { + void setSocket(SocketStream sockStream, UUID rmtNodeId) { synchronized (mux) { this.sockStream = sockStream; @@ -899,6 +910,31 @@ public void setSocket(SocketStream sockStream, UUID rmtNodeId) { } } + /** + * @throws InterruptedException If interrupted. + */ + private void forceStopRead() throws InterruptedException { + CountDownLatch stopReadLatch; + + synchronized (mux) { + SocketStream stream = sockStream; + + if (stream == null) + return; + + this.stopReadLatch = stopReadLatch = new CountDownLatch(1); + + U.closeQuiet(stream.socket()); + + this.sockStream = null; + this.rmtNodeId = null; + + mux.notifyAll(); + } + + stopReadLatch.await(); + } + /** {@inheritDoc} */ @Override protected void body() throws InterruptedException { while (!isInterrupted()) { @@ -906,6 +942,12 @@ public void setSocket(SocketStream sockStream, UUID rmtNodeId) { UUID rmtNodeId; synchronized (mux) { + if (stopReadLatch != null) { + stopReadLatch.countDown(); + + stopReadLatch = null; + } + if (this.sockStream == null) { mux.wait(); @@ -1007,18 +1049,21 @@ private class SocketWriter extends IgniteSpiThread { private final Queue queue = new ArrayDeque<>(); /** */ - private final long socketTimeout; + private final long sockTimeout; /** */ private TcpDiscoveryAbstractMessage unackedMsg; + /** */ + private CountDownLatch forceLeaveLatch; + /** * */ - protected SocketWriter() { + SocketWriter() { super(spi.ignite().name(), "tcp-client-disco-sock-writer", log); - socketTimeout = spi.failureDetectionTimeoutEnabled() ? spi.failureDetectionTimeout() : + sockTimeout = spi.failureDetectionTimeoutEnabled() ? spi.failureDetectionTimeout() : spi.getSocketTimeout(); } @@ -1033,6 +1078,29 @@ private void sendMessage(TcpDiscoveryAbstractMessage msg) { } } + /** + * Sends {@link TcpDiscoveryNodeLeftMessage} and closes socket. + * + * @throws InterruptedException If interrupted. + */ + private void forceLeave() throws InterruptedException { + CountDownLatch forceLeaveLatch; + + synchronized (mux) { + // If writer was stopped. + if (sock == null) + return; + + this.forceLeaveLatch = forceLeaveLatch = new CountDownLatch(1); + + unackedMsg = null; + + mux.notifyAll(); + } + + forceLeaveLatch.await(); + } + /** * @param sock Socket. * @param clientAck {@code True} is server supports client message acknowlede. @@ -1089,13 +1157,41 @@ void ackReceived(TcpDiscoveryClientAckResponse res) { continue; } - msg = queue.poll(); + if (forceLeaveLatch != null) { + msg = new TcpDiscoveryNodeLeftMessage(getLocalNodeId()); - if (msg == null) { - mux.wait(); + msg.client(true); + + try { + spi.writeToSocket( + sock, + msg, + sockTimeout); + } + catch (IOException | IgniteCheckedException e) { + if (log.isDebugEnabled()) { + log.debug("Failed to send TcpDiscoveryNodeLeftMessage on force leave [msg=" + msg + + ", err=" + e.getMessage() + ']'); + } + } + + U.closeQuiet(sock); + + this.sock = null; + + clear(); continue; } + else { + msg = queue.poll(); + + if (msg == null) { + mux.wait(); + + continue; + } + } } for (IgniteInClosure msgLsnr : spi.sndMsgLsnrs) @@ -1115,7 +1211,7 @@ void ackReceived(TcpDiscoveryClientAckResponse res) { spi.writeToSocket( sock, msg, - socketTimeout); + sockTimeout); msg = null; @@ -1165,10 +1261,30 @@ void ackReceived(TcpDiscoveryClientAckResponse res) { synchronized (mux) { if (sock == this.sock) this.sock = null; // Connection has dead. + + clear(); } } } } + + /** + * + */ + private void clear() { + assert Thread.holdsLock(mux); + + queue.clear(); + unackedMsg = null; + + CountDownLatch forceLeaveLatch = this.forceLeaveLatch; + + if (forceLeaveLatch != null) { + this.forceLeaveLatch = null; + + forceLeaveLatch.countDown(); + } + } } /** @@ -1413,6 +1529,38 @@ else if (msg == SPI_STOP) { else leaveLatch.countDown(); } + else if (msg == SPI_RECONNECT) { + if (state == CONNECTED) { + if (reconnector != null) { + reconnector.cancel(); + reconnector.join(); + + reconnector = null; + } + + sockWriter.forceLeave(); + sockReader.forceStopRead(); + + currSock = null; + + queue.clear(); + + onDisconnected(); + + notifyDiscovery(EVT_CLIENT_NODE_DISCONNECTED, topVer, locNode, allVisibleNodes()); + + 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+ ']'); + + locNode.onClientDisconnected(newId); + + tryJoin(); + } + } else if (msg instanceof TcpDiscoveryNodeFailedMessage && ((TcpDiscoveryNodeFailedMessage)msg).failedNodeId().equals(locNode.id())) { TcpDiscoveryNodeFailedMessage msg0 = (TcpDiscoveryNodeFailedMessage)msg; @@ -1495,20 +1643,7 @@ else if (msg == SPI_RECONNECT_FAILED) { ", failMsg=" + forceFailMsg + ']'); } - state = DISCONNECTED; - - nodeAdded = false; - - IgniteClientDisconnectedCheckedException err = - new IgniteClientDisconnectedCheckedException(null, "Failed to ping node, " + - "client node disconnected."); - - for (Map.Entry> e : pingFuts.entrySet()) { - GridFutureAdapter fut = e.getValue(); - - if (pingFuts.remove(e.getKey(), fut)) - fut.onDone(err); - } + onDisconnected(); notifyDiscovery(EVT_CLIENT_NODE_DISCONNECTED, topVer, locNode, allVisibleNodes()); } @@ -1603,6 +1738,26 @@ else if (discoMsg instanceof TcpDiscoveryCheckFailedMessage) } } + /** + * + */ + private void onDisconnected() { + state = DISCONNECTED; + + nodeAdded = false; + + IgniteClientDisconnectedCheckedException err = + new IgniteClientDisconnectedCheckedException(null, "Failed to ping node, " + + "client node disconnected."); + + for (Map.Entry> e : pingFuts.entrySet()) { + GridFutureAdapter fut = e.getValue(); + + if (pingFuts.remove(e.getKey(), fut)) + fut.onDone(err); + } + } + /** * @throws InterruptedException If interrupted. */ diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java index 4600be094c840..afd1c2ba7fc66 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java @@ -1609,6 +1609,11 @@ private void clearNodeAddedMessage(TcpDiscoveryAbstractMessage msg) { throw new UnsupportedOperationException(); } + /** {@inheritDoc} */ + @Override public void reconnect() throws IgniteSpiException { + throw new UnsupportedOperationException("Reconnect is not supported for server."); + } + /** {@inheritDoc} */ @Override protected IgniteSpiThread workerThread() { return msgWorker; diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java index f199c20e17edb..84c2ff28afef0 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java @@ -29,6 +29,7 @@ import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.util.typedef.internal.LT; import org.apache.ignite.internal.util.typedef.internal.U; @@ -258,6 +259,13 @@ protected static String threadStatus(Thread t) { return t.isAlive() ? "alive" : "dead"; } + /** + * Leave cluster and try to join again. + * + * @throws IgniteSpiException If failed. + */ + public abstract void reconnect() throws IgniteSpiException; + /** * FOR TEST ONLY!!! *

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 00ae97d1e302c..a2a47feb31486 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 @@ -1926,6 +1926,15 @@ boolean isSslEnabled() { return ignite().configuration().getSslContextFactory() != null; } + /** + * Force reconnect to cluster. + * + * @throws IgniteSpiException If failed. + */ + public void reconnect() throws IgniteSpiException { + impl.reconnect(); + } + /** * FOR TEST ONLY!!! */ 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 0f0165b0f04f9..6cdf465978907 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 @@ -700,9 +700,12 @@ public void testReconnectInitialExchangeInProgress() throws Exception { try { Ignition.start(optimize(getConfiguration(getTestGridName(SRV_CNT)))); - fail(); + // Commented due to IGNITE-4473, because + // IgniteClientDisconnectedException won't + // be thrown, but client will reconnect. +// fail(); - return false; + return true; } catch (IgniteClientDisconnectedException e) { log.info("Expected start error: " + e); 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 new file mode 100644 index 0000000000000..a5d42e9a12cba --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientRejoinTest.java @@ -0,0 +1,378 @@ +/* + * 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.io.IOException; +import java.io.OutputStream; +import java.net.InetSocketAddress; +import java.net.Socket; +import java.net.SocketException; +import java.util.ArrayList; +import java.util.List; +import java.util.Random; +import java.util.concurrent.Callable; +import java.util.concurrent.CountDownLatch; +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.cluster.ClusterNode; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteInClosure; +import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.spi.IgniteSpiException; +import org.apache.ignite.spi.IgniteSpiOperationTimeoutException; +import org.apache.ignite.spi.IgniteSpiOperationTimeoutHelper; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryAbstractMessage; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Tests client to be able restore connection to cluster if coordination is not available. + */ +public class IgniteClientRejoinTest extends GridCommonAbstractTest { + /** Block. */ + private volatile boolean block; + + /** Block all. */ + private volatile boolean blockAll; + + /** Coordinator. */ + private volatile ClusterNode crd; + + /** Client reconnect disabled. */ + private boolean clientReconnectDisabled; + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + System.setProperty("IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK", "true"); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + System.clearProperty("IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK"); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + clientReconnectDisabled = false; + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + if (gridName.contains("client")) { + cfg.setCommunicationSpi(new TcpCommunicationSpi()); + + TcpDiscoverySpi spi = (TcpDiscoverySpi)cfg.getDiscoverySpi(); + DiscoverySpi dspi = new DiscoverySpi(); + + dspi.setIpFinder(spi.getIpFinder()); + + cfg.setDiscoverySpi(dspi); + + dspi.setJoinTimeout(60_000); + dspi.setClientReconnectDisabled(clientReconnectDisabled); + + cfg.setClientMode(true); + } + + // TODO: IGNITE-4833 + cfg.setPeerClassLoadingEnabled(false); + + return cfg; + } + + /** + * @throws Exception If failed. + */ + public void testClientsReconnectAfterStart() throws Exception { + Ignite srv1 = startGrid("server1"); + + crd = ((IgniteKernal)srv1).localNode(); + + Ignite srv2 = startGrid("server2"); + + final CountDownLatch latch = new CountDownLatch(1); + + List clientNodes = new ArrayList<>(); + + final int CLIENTS_NUM = 5; + + for (int i = 0; i < CLIENTS_NUM; i++) + clientNodes.add(startGrid("client" + i)); + + blockAll = true; + + GridTestUtils.runAsync(new Callable() { + @Override public Object call() throws Exception { + U.sleep(5_000); + + block = true; + blockAll = false; + + System.out.println(">>> Allow with blocked coordinator."); + + latch.countDown(); + + return null; + } + }); + + IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { + @Override public Object call() throws Exception { + latch.await(); + + U.sleep((new Random().nextInt(15) + 30) * 1000); + + block = false; + + System.out.println(">>> Allow coordinator."); + + return null; + } + }); + + fut.get(); + + for (Ignite client : clientNodes) { + while (true) { + try { + IgniteCache cache = client.getOrCreateCache("some"); + + for (int i = 0; i < 100; i++) + cache.put(i, i); + + for (int i = 0; i < 100; i++) + assertEquals((Integer)i, cache.get(i)); + + cache.clear(); + + break; + } + catch (IgniteClientDisconnectedException e) { + e.reconnectFuture().get(); + } + } + } + + assertEquals(CLIENTS_NUM, srv1.cluster().forClients().nodes().size()); + assertEquals(CLIENTS_NUM, srv2.cluster().forClients().nodes().size()); + } + + /** + * @throws Exception If failed. + */ + public void testClientsReconnect() throws Exception { + Ignite srv1 = startGrid("server1"); + + crd = ((IgniteKernal)srv1).localNode(); + + Ignite srv2 = startGrid("server2"); + + block = true; + + List> futs = new ArrayList<>(); + + final CountDownLatch latch = new CountDownLatch(1); + + final int CLIENTS_NUM = 5; + + for (int i = 0; i < CLIENTS_NUM; i++) { + final int idx = i; + + IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { + @Override public Ignite call() throws Exception { + latch.await(); + + return startGrid("client" + idx); + } + }); + + futs.add(fut); + } + + GridTestUtils.runAsync(new Callable() { + @Override public Boolean call() throws Exception { + latch.countDown(); + + Random rnd = new Random(); + + U.sleep((rnd.nextInt(15) + 15) * 1000); + + block = false; + + System.out.println(">>> ALLOW connection to coordinator."); + + return true; + } + }); + + for (IgniteInternalFuture clientFut : futs) { + Ignite client = clientFut.get(); + + IgniteCache cache = client.getOrCreateCache(client.name()); + + for (int i = 0; i < 100; i++) + cache.put(i, i); + + for (int i = 0; i < 100; i++) + assert i == cache.get(i); + } + + assertEquals(CLIENTS_NUM, srv1.cluster().forClients().nodes().size()); + assertEquals(CLIENTS_NUM, srv2.cluster().forClients().nodes().size()); + } + + /** + * @throws Exception If failed. + */ + public void testClientsReconnectDisabled() throws Exception { + clientReconnectDisabled = true; + + Ignite srv1 = startGrid("server1"); + + crd = ((IgniteKernal)srv1).localNode(); + + Ignite srv2 = startGrid("server2"); + + block = true; + + List> futs = new ArrayList<>(); + + final CountDownLatch latch = new CountDownLatch(1); + + final int CLIENTS_NUM = 5; + + for (int i = 0; i < CLIENTS_NUM; i++) { + final int idx = i; + + IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { + @Override public Ignite call() throws Exception { + latch.await(); + + return startGrid("client" + idx); + } + }); + + futs.add(fut); + } + + latch.countDown(); + + for (final IgniteInternalFuture clientFut : futs) { + //noinspection ThrowableNotThrown + GridTestUtils.assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + clientFut.get(); + + return null; + } + }, IgniteCheckedException.class, null); + } + + assertEquals(0, srv1.cluster().forClients().nodes().size()); + assertEquals(0, srv2.cluster().forClients().nodes().size()); + } + + /** {@inheritDoc} */ + @Override protected long getTestTimeout() { + return 3 * 60_000; + } + + /** + * + */ + private class TcpCommunicationSpi extends org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi { + /** {@inheritDoc} */ + @Override public void sendMessage(ClusterNode node, Message msg) throws IgniteSpiException { + if (blockAll || block && node.id().equals(crd.id())) + throw new IgniteSpiException(new SocketException("Test communication exception")); + + super.sendMessage(node, msg); + } + + /** {@inheritDoc} */ + @Override public void sendMessage(ClusterNode node, Message msg, + IgniteInClosure ackC) throws IgniteSpiException { + if (blockAll || block && node.id().equals(crd.id())) + throw new IgniteSpiException(new SocketException("Test communication exception")); + + super.sendMessage(node, msg, ackC); + } + } + + /** + * + */ + private class DiscoverySpi extends TcpDiscoverySpi { + /** {@inheritDoc} */ + @Override protected void writeToSocket(Socket sock, TcpDiscoveryAbstractMessage msg, byte[] data, + long timeout) throws IOException { + if (blockAll || block && sock.getPort() == 47500) + throw new SocketException("Test discovery exception"); + + super.writeToSocket(sock, msg, data, timeout); + } + + /** {@inheritDoc} */ + @Override protected void writeToSocket(Socket sock, TcpDiscoveryAbstractMessage msg, + long timeout) throws IOException, IgniteCheckedException { + if (blockAll || block && sock.getPort() == 47500) + throw new SocketException("Test discovery exception"); + + super.writeToSocket(sock, msg, timeout); + } + + /** {@inheritDoc} */ + @Override protected void writeToSocket(Socket sock, OutputStream out, TcpDiscoveryAbstractMessage msg, + long timeout) throws IOException, IgniteCheckedException { + if (blockAll || block && sock.getPort() == 47500) + throw new SocketException("Test discovery exception"); + + super.writeToSocket(sock, out, msg, timeout); + } + + /** {@inheritDoc} */ + @Override protected void writeToSocket(TcpDiscoveryAbstractMessage msg, Socket sock, int res, + long timeout) throws IOException { + if (blockAll || block && sock.getPort() == 47500) + throw new SocketException("Test discovery exception"); + + super.writeToSocket(msg, sock, res, timeout); + } + + /** {@inheritDoc} */ + @Override protected Socket openSocket(Socket sock, InetSocketAddress remAddr, + IgniteSpiOperationTimeoutHelper timeoutHelper) throws IOException, IgniteSpiOperationTimeoutException { + if (blockAll || block && sock.getPort() == 47500) + throw new SocketException("Test discovery exception"); + + return super.openSocket(sock, remAddr, timeoutHelper); + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java index 331b581bb7bec..0483a1ce8fbc6 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java @@ -32,6 +32,7 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteClientDisconnectedException; import org.apache.ignite.IgniteInterruptedException; import org.apache.ignite.IgniteMessaging; import org.apache.ignite.IgniteState; @@ -43,6 +44,7 @@ import org.apache.ignite.events.Event; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.IgniteKernal; import org.apache.ignite.internal.IgniteNodeAttributes; import org.apache.ignite.internal.util.GridConcurrentHashSet; import org.apache.ignite.internal.util.lang.GridAbsPredicate; @@ -1788,8 +1790,7 @@ else if (evt.type() == EVT_CLIENT_NODE_RECONNECTED) { clientNodeIds.add(client.cluster().localNode().id()); GridTestUtils.waitForCondition(new GridAbsPredicate() { - @Override - public boolean apply() { + @Override public boolean apply() { return srv.cluster().nodes().size() == 2; } }, awaitTime()); @@ -1799,6 +1800,49 @@ public boolean apply() { assertFalse(err.get()); } + /** + * @throws Exception If failed. + */ + public void testForceClientReconnect() throws Exception { + startServerNodes(1); + + startClientNodes(1); + + Ignite srv = G.ignite("server-0"); + IgniteKernal client = (IgniteKernal)G.ignite("client-0"); + + UUID clientId = F.first(clientNodeIds); + + final CountDownLatch latch = new CountDownLatch(1); + + srv.events().enableLocal(EVT_NODE_JOINED); + + srv.events().localListen(new IgnitePredicate() { + @Override public boolean apply(Event evt) { + latch.countDown(); + + return false; + } + }, EVT_NODE_JOINED); + + client.context().discovery().reconnect(); + + assert latch.await(10, TimeUnit.SECONDS); + + while (true) { + try { + UUID newId = client.localNode().id(); + + assert !clientId.equals(newId) : clientId; + + break; + } + catch (IgniteClientDisconnectedException e) { + e.reconnectFuture().get(10_000); + } + } + } + /** * @param ignite Ignite. * @throws Exception If failed. diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteClientReconnectTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteClientReconnectTestSuite.java index ea8e37bada1d0..67d88e1e29eac 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteClientReconnectTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteClientReconnectTestSuite.java @@ -29,6 +29,7 @@ import org.apache.ignite.internal.IgniteClientReconnectServicesTest; import org.apache.ignite.internal.IgniteClientReconnectStopTest; import org.apache.ignite.internal.IgniteClientReconnectStreamerTest; +import org.apache.ignite.internal.IgniteClientRejoinTest; /** * @@ -52,6 +53,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(IgniteClientReconnectServicesTest.class); suite.addTestSuite(IgniteClientReconnectStreamerTest.class); suite.addTestSuite(IgniteClientReconnectFailoverTest.class); + suite.addTestSuite(IgniteClientRejoinTest.class); return suite; } From c4de164392ddc114c88d5a6eba0ff0b13d32542f Mon Sep 17 00:00:00 2001 From: AMRepo Date: Mon, 20 Mar 2017 16:31:15 +0300 Subject: [PATCH 02/90] IGNITE-518: Unmuted tests that was fixed in ignite-4036. This closes #1636. --- .../expiry/IgniteCacheExpiryPolicyAbstractTest.java | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyAbstractTest.java index b234631e4002a..0b1b8c5fff3a4 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyAbstractTest.java @@ -162,7 +162,8 @@ public void testCreateUpdate0() throws Exception { assertEquals(0, pSize); } - /** * @throws Exception If failed. + /** + * @throws Exception If failed. */ public void testZeroOnCreate() throws Exception { factory = CreatedExpiryPolicy.factoryOf(Duration.ZERO); @@ -849,8 +850,6 @@ private void createUpdate(Integer key, @Nullable TransactionConcurrency txConcur * @throws Exception If failed. */ public void testNearCreateUpdate() throws Exception { - fail("https://issues.apache.org/jira/browse/IGNITE-518"); - if (cacheMode() != PARTITIONED) return; @@ -974,8 +973,6 @@ private void nearPutAll() throws Exception { * @throws Exception If failed. */ public void testNearAccess() throws Exception { - fail("https://issues.apache.org/jira/browse/IGNITE-518"); - if (cacheMode() != PARTITIONED) return; @@ -1023,10 +1020,10 @@ public void testNearAccess() throws Exception { * @throws Exception If failed. */ public void testNearExpiresOnClient() throws Exception { - if(cacheMode() != PARTITIONED) + if (cacheMode() != PARTITIONED) return; - factory = CreatedExpiryPolicy.factoryOf(new Duration(TimeUnit.SECONDS,1)); + factory = CreatedExpiryPolicy.factoryOf(new Duration(TimeUnit.SECONDS, 1)); nearCache = true; From e0c012d977b6db13dfdf2fb8347677998287c1e4 Mon Sep 17 00:00:00 2001 From: Igor Sapego Date: Tue, 21 Mar 2017 17:50:06 +0300 Subject: [PATCH 03/90] IGNITE-4200: Added copying of the C++ binaries. --- assembly/release-fabric-base.xml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/assembly/release-fabric-base.xml b/assembly/release-fabric-base.xml index 7b3d8cf21e5f5..7f05c305ed4b2 100644 --- a/assembly/release-fabric-base.xml +++ b/assembly/release-fabric-base.xml @@ -174,6 +174,12 @@ /platforms/cpp/docs + + + modules/platforms/cpp/bin + /platforms/cpp/bin + + bin From b7ab27301b59bf93fc73b52fdf8e0bcf124fec1d Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Thu, 6 Apr 2017 14:43:50 +0300 Subject: [PATCH 04/90] IGNITE-4832: Prevent service deployment on client by default when configuration is provided on startup. This closes #1748. --- .../service/GridServiceProcessor.java | 10 +- .../GridServiceProcessorAbstractSelfTest.java | 11 ++ ...rviceProcessorMultiNodeConfigSelfTest.java | 74 +++++++++- ...GridServiceProcessorMultiNodeSelfTest.java | 139 +++++++++++++++--- 4 files changed, 202 insertions(+), 32 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java index bd815189fb5ba..a8af9832e9355 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java @@ -70,11 +70,11 @@ import org.apache.ignite.internal.processors.cache.query.CacheQuery; import org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; -import org.apache.ignite.internal.processors.continuous.AbstractContinuousMessage; import org.apache.ignite.internal.processors.task.GridInternal; import org.apache.ignite.internal.processors.timeout.GridTimeoutObject; import org.apache.ignite.internal.util.GridEmptyIterator; import org.apache.ignite.internal.util.GridSpinBusyLock; +import org.apache.ignite.internal.util.SerializableTransient; import org.apache.ignite.internal.util.future.GridCompoundFuture; import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.future.GridFutureAdapter; @@ -92,7 +92,6 @@ import org.apache.ignite.lang.IgniteProductVersion; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.marshaller.Marshaller; -import org.apache.ignite.internal.util.SerializableTransient; import org.apache.ignite.resources.IgniteInstanceResource; import org.apache.ignite.resources.JobContextResource; import org.apache.ignite.resources.LoggerResource; @@ -296,8 +295,13 @@ public void onContinuousProcessorStarted(GridKernalContext ctx) throws IgniteChe if (cfgs != null) { Collection> futs = new ArrayList<>(); - for (ServiceConfiguration c : ctx.config().getServiceConfiguration()) + for (ServiceConfiguration c : cfgs) { + // Deploy only on server nodes by default. + if (c.getNodeFilter() == null) + c.setNodeFilter(ctx.cluster().get().forServers().predicate()); + futs.add(deploy(c)); + } // Await for services to deploy. for (IgniteInternalFuture f : futs) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorAbstractSelfTest.java index 111cb714e8f32..0f79855518e3d 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorAbstractSelfTest.java @@ -129,6 +129,17 @@ protected void startExtraNodes(int cnt) throws Exception { startGrid(nodeCount() + i); } + /** */ + protected void startExtraNodes(int servers, int clients) throws Exception { + startExtraNodes(servers); + + for (int i = 0; i < clients; i++) { + final String nodeName = getTestGridName(nodeCount() + servers + i); + + startGrid(nodeName, getConfiguration(nodeName).setClientMode(true)); + } + } + /** * @throws Exception If failed. */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeConfigSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeConfigSelfTest.java index b819cc93e6fb8..1bd3b035a446c 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeConfigSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeConfigSelfTest.java @@ -19,7 +19,9 @@ import java.util.concurrent.CountDownLatch; import org.apache.ignite.Ignite; +import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.internal.util.lang.GridAbsPredicateX; +import org.apache.ignite.services.Service; import org.apache.ignite.services.ServiceConfiguration; import org.apache.ignite.testframework.GridTestUtils; @@ -33,6 +35,9 @@ public class GridServiceProcessorMultiNodeConfigSelfTest extends GridServiceProc /** Node singleton name. */ private static final String NODE_SINGLE = "serviceConfigEachNode"; + /** Node singleton name. */ + private static final String NODE_SINGLE_BUT_CLIENT = "serviceConfigEachNodeButClient"; + /** Affinity service name. */ private static final String AFFINITY = "serviceConfigAffinity"; @@ -46,7 +51,7 @@ public class GridServiceProcessorMultiNodeConfigSelfTest extends GridServiceProc /** {@inheritDoc} */ @Override protected ServiceConfiguration[] services() { - ServiceConfiguration[] arr = new ServiceConfiguration[3]; + ServiceConfiguration[] arr = new ServiceConfiguration[4]; ServiceConfiguration cfg = new ServiceConfiguration(); @@ -59,7 +64,7 @@ public class GridServiceProcessorMultiNodeConfigSelfTest extends GridServiceProc cfg = new ServiceConfiguration(); - cfg.setName(NODE_SINGLE); + cfg.setName(NODE_SINGLE_BUT_CLIENT); cfg.setMaxPerNodeCount(1); cfg.setService(new DummyService()); @@ -76,6 +81,15 @@ public class GridServiceProcessorMultiNodeConfigSelfTest extends GridServiceProc arr[2] = cfg; + cfg = new ServiceConfiguration(); + + cfg.setName(NODE_SINGLE); + cfg.setMaxPerNodeCount(1); + cfg.setNodeFilter(new CacheConfiguration.IgniteAllNodesPredicate()); + cfg.setService(new DummyService()); + + arr[3] = cfg; + return arr; } @@ -91,6 +105,8 @@ public class GridServiceProcessorMultiNodeConfigSelfTest extends GridServiceProc DummyService.cancelled(CLUSTER_SINGLE) == 0 && DummyService.started(NODE_SINGLE) == nodeCount() && DummyService.cancelled(NODE_SINGLE) == 0 && + DummyService.started(NODE_SINGLE_BUT_CLIENT) == nodeCount() && + DummyService.cancelled(NODE_SINGLE_BUT_CLIENT) == 0 && actualCount(AFFINITY, randomGrid().services().serviceDescriptors()) == 1; } }, @@ -112,6 +128,13 @@ public void testDeployOnEachNodeUpdateTopology() throws Exception { checkDeployOnEachNodeUpdateTopology(NODE_SINGLE); } + /** + * @throws Exception If failed. + */ + public void testDeployOnEachNodeButClientUpdateTopology() throws Exception { + checkDeployOnEachNodeButClientUpdateTopology(NODE_SINGLE_BUT_CLIENT); + } + /** * @throws Exception If failed. */ @@ -120,6 +143,10 @@ public void testAll() throws Exception { DummyService.reset(); + checkDeployOnEachNodeButClientUpdateTopology(NODE_SINGLE_BUT_CLIENT); + + DummyService.reset(); + checkDeployOnEachNodeUpdateTopology(NODE_SINGLE); DummyService.reset(); @@ -152,9 +179,7 @@ public void testAffinityUpdateTopology() throws Exception { private void checkSingletonUpdateTopology(String name) throws Exception { Ignite g = randomGrid(); - int nodeCnt = 2; - - startExtraNodes(nodeCnt); + startExtraNodes(2, 2); try { assertEquals(name, 0, DummyService.started(name)); @@ -165,7 +190,7 @@ private void checkSingletonUpdateTopology(String name) throws Exception { checkCount(name, g.services().serviceDescriptors(), 1); } finally { - stopExtraNodes(nodeCnt); + stopExtraNodes(4); } } @@ -176,17 +201,21 @@ private void checkSingletonUpdateTopology(String name) throws Exception { private void checkDeployOnEachNodeUpdateTopology(String name) throws Exception { Ignite g = randomGrid(); - int newNodes = 2; + int newNodes = 4; CountDownLatch latch = new CountDownLatch(newNodes); DummyService.exeLatch(name, latch); - startExtraNodes(newNodes); + startExtraNodes(2, 2); try { latch.await(); + // Ensure service is deployed. + assertNotNull(grid(nodeCount() + newNodes - 1).services() + .serviceProxy(NODE_SINGLE_BUT_CLIENT, Service.class, false, 2000)); + assertEquals(name, newNodes, DummyService.started(name)); assertEquals(name, 0, DummyService.cancelled(name)); @@ -196,4 +225,33 @@ private void checkDeployOnEachNodeUpdateTopology(String name) throws Exception { stopExtraNodes(newNodes); } } + + /** + * @param name Name. + * @throws Exception If failed. + */ + private void checkDeployOnEachNodeButClientUpdateTopology(String name) throws Exception { + Ignite g = randomGrid(); + + int servers = 2; + int clients = 2; + + CountDownLatch latch = new CountDownLatch(servers); + + DummyService.exeLatch(name, latch); + + startExtraNodes(servers, clients); + + try { + latch.await(); + + assertEquals(name, servers, DummyService.started(name)); + assertEquals(name, 0, DummyService.cancelled(name)); + + checkCount(name, g.services().serviceDescriptors(), nodeCount() + servers); + } + finally { + stopExtraNodes(servers + clients); + } + } } \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeSelfTest.java index 39336ef001a02..f7403dcc820ca 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeSelfTest.java @@ -21,7 +21,10 @@ import junit.framework.TestCase; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteServices; +import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.lang.IgniteFuture; +import org.apache.ignite.services.Service; +import org.apache.ignite.services.ServiceConfiguration; /** * Single node services test. @@ -121,50 +124,144 @@ public void testAffinityDeployUpdateTopology() throws Exception { /** * @throws Exception If failed. */ - public void testDeployOnEachNodeUpdateTopology() throws Exception { - String name = "serviceOnEachNodeUpdateTopology"; + public void testDeployOnEachNodeButClientUpdateTopology() throws Exception { + // Prestart client node. + Ignite client = startGrid("client", getConfiguration("client").setClientMode(true)); - Ignite g = randomGrid(); + try { + final int prestartedNodes = nodeCount() + 1; - CountDownLatch latch = new CountDownLatch(nodeCount()); + String name = "serviceOnEachNodeButClientUpdateTopology"; - DummyService.exeLatch(name, latch); + Ignite g = randomGrid(); - IgniteServices svcs = g.services().withAsync(); + CountDownLatch latch = new CountDownLatch(nodeCount()); - svcs.deployNodeSingleton(name, new DummyService()); + DummyService.exeLatch(name, latch); - IgniteFuture fut = svcs.future(); + IgniteServices svcs = g.services().withAsync(); - info("Deployed service: " + name); + svcs.deployNodeSingleton(name, new DummyService()); - fut.get(); + IgniteFuture fut = svcs.future(); - info("Finished waiting for service future: " + name); + info("Deployed service: " + name); - latch.await(); + fut.get(); - TestCase.assertEquals(name, nodeCount(), DummyService.started(name)); - TestCase.assertEquals(name, 0, DummyService.cancelled(name)); + info("Finished waiting for service future: " + name); - int newNodes = 2; + latch.await(); - latch = new CountDownLatch(newNodes); + // Ensure service is deployed + assertNotNull(client.services().serviceProxy(name, Service.class, false, 2000)); - DummyService.exeLatch(name, latch); + TestCase.assertEquals(name, nodeCount(), DummyService.started(name)); + TestCase.assertEquals(name, 0, DummyService.cancelled(name)); + + int servers = 2; + int clients = 2; + + latch = new CountDownLatch(servers); + + DummyService.exeLatch(name, latch); + + startExtraNodes(servers, clients); + + try { + latch.await(); + + // Ensure service is deployed + assertNotNull(grid(prestartedNodes + servers - 1) + .services().serviceProxy(name, Service.class, false, 2000)); + + TestCase.assertEquals(name, nodeCount() + servers, DummyService.started(name)); + TestCase.assertEquals(name, 0, DummyService.cancelled(name)); + + checkCount(name, g.services().serviceDescriptors(), nodeCount() + servers); + } + finally { + stopExtraNodes(servers + clients); + } + } + finally { + stopGrid("client"); + } + } - startExtraNodes(newNodes); + /** + * @throws Exception If failed. + */ + public void testDeployOnEachNodeUpdateTopology() throws Exception { + // Prestart client node. + Ignite client = startGrid("client", getConfiguration("client").setClientMode(true)); try { + String name = "serviceOnEachNodeUpdateTopology"; + + Ignite g = randomGrid(); + + final int prestartedNodes = nodeCount() + 1; + + CountDownLatch latch = new CountDownLatch(prestartedNodes); + + DummyService.exeLatch(name, latch); + + ServiceConfiguration srvcCfg = new ServiceConfiguration(); + + srvcCfg.setNodeFilter(new CacheConfiguration.IgniteAllNodesPredicate()); + srvcCfg.setName(name); + srvcCfg.setMaxPerNodeCount(1); + srvcCfg.setService(new DummyService()); + + IgniteServices svcs = g.services().withAsync(); + + svcs.deploy(srvcCfg); + + IgniteFuture fut = svcs.future(); + + info("Deployed service: " + name); + + fut.get(); + + info("Finished waiting for service future: " + name); + latch.await(); - TestCase.assertEquals(name, nodeCount() + newNodes, DummyService.started(name)); + // Ensure service is deployed + assertNotNull(client.services().serviceProxy(name, Service.class, false, 2000)); + + TestCase.assertEquals(name, prestartedNodes, DummyService.started(name)); TestCase.assertEquals(name, 0, DummyService.cancelled(name)); - checkCount(name, g.services().serviceDescriptors(), nodeCount() + newNodes); + int servers = 2; + int clients = 2; + + int extraNodes = servers + clients; + + latch = new CountDownLatch(extraNodes); + + DummyService.exeLatch(name, latch); + + startExtraNodes(servers, clients); + + try { + latch.await(); + + // Ensure service is deployed + assertNotNull(client.services().serviceProxy(name, Service.class, false, 2000)); + + TestCase.assertEquals(name, prestartedNodes + extraNodes, DummyService.started(name)); + TestCase.assertEquals(name, 0, DummyService.cancelled(name)); + + checkCount(name, g.services().serviceDescriptors(), prestartedNodes + extraNodes); + } + finally { + stopExtraNodes(extraNodes); + } } finally { - stopExtraNodes(newNodes); + stopGrid("client"); } } } \ No newline at end of file From 443ac9a7aa82af1359a03bcfc8f9212b108300e4 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Wed, 5 Apr 2017 15:01:02 +0300 Subject: [PATCH 05/90] IGNITE-4917: Fixed failure when accessing BinaryObjectBuilder field value serialized with OptimizedMarshaller . This closes #1736. --- .../internal/binary/builder/BinaryBuilderReader.java | 8 ++++++++ .../binary/mutabletest/GridBinaryTestClasses.java | 9 +++++++++ 2 files changed, 17 insertions(+) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderReader.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderReader.java index baaabd6a1a0bd..02264e3311529 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderReader.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderReader.java @@ -476,6 +476,14 @@ public Object getValueQuickly(int pos, int len) { return new BinaryPlainBinaryObject(binaryObj); } + case GridBinaryMarshaller.OPTM_MARSH: { + final BinaryHeapInputStream bin = BinaryHeapInputStream.create(arr, pos + 1); + + final Object obj = BinaryUtils.doReadOptimized(bin, ctx, U.resolveClassLoader(ctx.configuration())); + + return obj; + } + default: throw new BinaryObjectException("Invalid flag value: " + type); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/mutabletest/GridBinaryTestClasses.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/mutabletest/GridBinaryTestClasses.java index 5ddb87dcefb31..0d4de7fa0e215 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/binary/mutabletest/GridBinaryTestClasses.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/mutabletest/GridBinaryTestClasses.java @@ -24,6 +24,7 @@ import java.io.ObjectOutputStream; import java.io.Serializable; import java.math.BigDecimal; +import java.math.BigInteger; import java.sql.Timestamp; import java.util.ArrayList; import java.util.Date; @@ -141,6 +142,9 @@ public static class TestObjectAllTypes implements Serializable { /** */ public Integer i_; + /** */ + public BigInteger bi_; + /** */ public Long l_; @@ -150,6 +154,9 @@ public static class TestObjectAllTypes implements Serializable { /** */ public Double d_; + /** */ + public BigDecimal bd_; + /** */ public Character c_; @@ -267,9 +274,11 @@ public void setDefaultData() { b_ = 11; s_ = 22; i_ = 33; + bi_ = new BigInteger("33000000000000"); l_ = 44L; f_ = 55f; d_ = 66d; + bd_ = new BigDecimal("33000000000000.123456789"); c_ = 'e'; z_ = true; From 4a1415ad01ff9fde30d5c7c02e6d938f1515178d Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Wed, 12 Apr 2017 13:01:25 +0300 Subject: [PATCH 06/90] IGNITE-4907: Fixed excessive service instances can be started with dynamic deployment. This closes #1766. (cherry picked from commit 0f7ef74) --- .../service/GridServiceProcessor.java | 2 +- ...rviceProcessorMultiNodeConfigSelfTest.java | 95 ++++++++++++++++--- ...GridServiceProcessorMultiNodeSelfTest.java | 61 ++++++++++++ 3 files changed, 146 insertions(+), 12 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java index a8af9832e9355..2a363e21e8ef5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java @@ -972,7 +972,7 @@ private void reassign(GridServiceDeployment dep, AffinityTopologyVersion topVer) int perNodeCnt = totalCnt != 0 ? totalCnt / size : maxPerNodeCnt; int remainder = totalCnt != 0 ? totalCnt % size : 0; - if (perNodeCnt > maxPerNodeCnt && maxPerNodeCnt != 0) { + if (perNodeCnt >= maxPerNodeCnt && maxPerNodeCnt != 0) { perNodeCnt = maxPerNodeCnt; remainder = 0; } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeConfigSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeConfigSelfTest.java index 1bd3b035a446c..9da62c0abb702 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeConfigSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeConfigSelfTest.java @@ -17,11 +17,13 @@ package org.apache.ignite.internal.processors.service; +import java.util.ArrayList; +import java.util.List; import java.util.concurrent.CountDownLatch; import org.apache.ignite.Ignite; import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.util.lang.GridAbsPredicateX; -import org.apache.ignite.services.Service; import org.apache.ignite.services.ServiceConfiguration; import org.apache.ignite.testframework.GridTestUtils; @@ -38,6 +40,9 @@ public class GridServiceProcessorMultiNodeConfigSelfTest extends GridServiceProc /** Node singleton name. */ private static final String NODE_SINGLE_BUT_CLIENT = "serviceConfigEachNodeButClient"; + /** Node singleton name. */ + private static final String NODE_SINGLE_WITH_LIMIT = "serviceConfigWithLimit"; + /** Affinity service name. */ private static final String AFFINITY = "serviceConfigAffinity"; @@ -51,7 +56,7 @@ public class GridServiceProcessorMultiNodeConfigSelfTest extends GridServiceProc /** {@inheritDoc} */ @Override protected ServiceConfiguration[] services() { - ServiceConfiguration[] arr = new ServiceConfiguration[4]; + List cfgs = new ArrayList<>(); ServiceConfiguration cfg = new ServiceConfiguration(); @@ -60,7 +65,7 @@ public class GridServiceProcessorMultiNodeConfigSelfTest extends GridServiceProc cfg.setTotalCount(1); cfg.setService(new DummyService()); - arr[0] = cfg; + cfgs.add(cfg); cfg = new ServiceConfiguration(); @@ -68,7 +73,7 @@ public class GridServiceProcessorMultiNodeConfigSelfTest extends GridServiceProc cfg.setMaxPerNodeCount(1); cfg.setService(new DummyService()); - arr[1] = cfg; + cfgs.add(cfg); cfg = new ServiceConfiguration(); @@ -79,7 +84,7 @@ public class GridServiceProcessorMultiNodeConfigSelfTest extends GridServiceProc cfg.setTotalCount(1); cfg.setService(new AffinityService(AFFINITY_KEY)); - arr[2] = cfg; + cfgs.add(cfg); cfg = new ServiceConfiguration(); @@ -88,9 +93,18 @@ public class GridServiceProcessorMultiNodeConfigSelfTest extends GridServiceProc cfg.setNodeFilter(new CacheConfiguration.IgniteAllNodesPredicate()); cfg.setService(new DummyService()); - arr[3] = cfg; + cfgs.add(cfg); + + cfg = new ServiceConfiguration(); - return arr; + cfg.setName(NODE_SINGLE_WITH_LIMIT); + cfg.setMaxPerNodeCount(1); + cfg.setTotalCount(nodeCount() + 1); + cfg.setService(new DummyService()); + + cfgs.add(cfg); + + return cfgs.toArray(new ServiceConfiguration[cfgs.size()]); } /** {@inheritDoc} */ @@ -107,6 +121,8 @@ public class GridServiceProcessorMultiNodeConfigSelfTest extends GridServiceProc DummyService.cancelled(NODE_SINGLE) == 0 && DummyService.started(NODE_SINGLE_BUT_CLIENT) == nodeCount() && DummyService.cancelled(NODE_SINGLE_BUT_CLIENT) == 0 && + DummyService.started(NODE_SINGLE_WITH_LIMIT) >= nodeCount() && + DummyService.cancelled(NODE_SINGLE_WITH_LIMIT) == 0 && actualCount(AFFINITY, randomGrid().services().serviceDescriptors()) == 1; } }, @@ -170,6 +186,59 @@ public void testAffinityUpdateTopology() throws Exception { finally { stopExtraNodes(nodeCnt); } + + checkCount(AFFINITY, g.services().serviceDescriptors(), 1); + } + + /** + * @throws Exception If failed. + */ + public void testDeployLimits() throws Exception { + final Ignite g = randomGrid(); + + final String name = NODE_SINGLE_WITH_LIMIT; + + waitForDeployment(name, nodeCount()); + + checkCount(name, g.services().serviceDescriptors(), nodeCount()); + + int extraNodes = 2; + + CountDownLatch latch = new CountDownLatch(1); + + DummyService.exeLatch(name, latch); + + startExtraNodes(extraNodes); + + try { + latch.await(); + + checkCount(name, g.services().serviceDescriptors(), nodeCount() + 1); + } + finally { + stopExtraNodes(extraNodes); + } + + assertEquals(name, 1, DummyService.cancelled(name)); + + waitForDeployment(name, nodeCount()); + + checkCount(name, g.services().serviceDescriptors(), nodeCount()); + } + + /** + * @param srvcName Service name + * @param expectedDeps Expected number of service deployments + * + */ + private boolean waitForDeployment(final String srvcName, final int expectedDeps) throws IgniteInterruptedCheckedException { + final Ignite g = randomGrid(); + + return GridTestUtils.waitForCondition(new GridAbsPredicateX() { + @Override public boolean applyx() { + return actualCount(srvcName, g.services().serviceDescriptors()) == expectedDeps; + } + }, 1500); } /** @@ -212,10 +281,6 @@ private void checkDeployOnEachNodeUpdateTopology(String name) throws Exception { try { latch.await(); - // Ensure service is deployed. - assertNotNull(grid(nodeCount() + newNodes - 1).services() - .serviceProxy(NODE_SINGLE_BUT_CLIENT, Service.class, false, 2000)); - assertEquals(name, newNodes, DummyService.started(name)); assertEquals(name, 0, DummyService.cancelled(name)); @@ -224,6 +289,10 @@ private void checkDeployOnEachNodeUpdateTopology(String name) throws Exception { finally { stopExtraNodes(newNodes); } + + waitForDeployment(name, nodeCount()); + + checkCount(name, g.services().serviceDescriptors(), nodeCount()); } /** @@ -253,5 +322,9 @@ private void checkDeployOnEachNodeButClientUpdateTopology(String name) throws Ex finally { stopExtraNodes(servers + clients); } + + waitForDeployment(name, nodeCount()); + + checkCount(name, g.services().serviceDescriptors(), nodeCount()); } } \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeSelfTest.java index f7403dcc820ca..d133cf299a756 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeSelfTest.java @@ -264,4 +264,65 @@ public void testDeployOnEachNodeUpdateTopology() throws Exception { stopGrid("client"); } } + + /** + * @throws Exception If failed. + */ + public void testDeployLimits() throws Exception { + String name = "serviceWithLimitsUpdateTopology"; + + Ignite g = randomGrid(); + + final int totalInstances = nodeCount() + 1; + + CountDownLatch latch = new CountDownLatch(nodeCount()); + + DummyService.exeLatch(name, latch); + + ServiceConfiguration srvcCfg = new ServiceConfiguration(); + + srvcCfg.setName(name); + srvcCfg.setMaxPerNodeCount(1); + srvcCfg.setTotalCount(totalInstances); + srvcCfg.setService(new DummyService()); + + IgniteServices svcs = g.services().withAsync(); + + svcs.deploy(srvcCfg); + + IgniteFuture fut = svcs.future(); + + info("Deployed service: " + name); + + fut.get(); + + info("Finished waiting for service future: " + name); + + latch.await(); + + TestCase.assertEquals(name, nodeCount(), DummyService.started(name)); + TestCase.assertEquals(name, 0, DummyService.cancelled(name)); + + checkCount(name, g.services().serviceDescriptors(), nodeCount()); + + int extraNodes = 2; + + latch = new CountDownLatch(1); + + DummyService.exeLatch(name, latch); + + startExtraNodes(2); + + try { + latch.await(); + + TestCase.assertEquals(name, totalInstances, DummyService.started(name)); + TestCase.assertEquals(name, 0, DummyService.cancelled(name)); + + checkCount(name, g.services().serviceDescriptors(), totalInstances); + } + finally { + stopExtraNodes(extraNodes); + } + } } \ No newline at end of file From bf1049741f7a64728bd433f78262ba273f969848 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Mon, 17 Apr 2017 19:00:30 +0300 Subject: [PATCH 07/90] IGNITE-4954 - Configurable expiration timeout for Cassandra session. This closes #1785. --- .../cassandra/datasource/DataSource.java | 26 ++++++++++++++++++- .../session/CassandraSessionImpl.java | 23 +++++++++------- .../cassandra/session/pool/SessionPool.java | 6 ++--- .../session/pool/SessionWrapper.java | 15 ++++++----- 4 files changed, 51 insertions(+), 19 deletions(-) diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/DataSource.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/DataSource.java index 915eebde73f01..1fa2a1df9faa4 100644 --- a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/DataSource.java +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/DataSource.java @@ -46,6 +46,9 @@ * Data source abstraction to specify configuration of the Cassandra session to be used. */ public class DataSource { + /** Default expiration timeout for Cassandra driver session. */ + public static final long DFLT_SESSION_EXPIRATION_TIMEOUT = 300000; // 5 minutes. + /** Number of rows to immediately fetch in CQL statement execution. */ private Integer fetchSize; @@ -123,6 +126,9 @@ public class DataSource { /** Netty options to use for connection. */ private NettyOptions nettyOptions; + /** Expiration timeout for Cassandra driver session. */ + private long sessionExpirationTimeout = DFLT_SESSION_EXPIRATION_TIMEOUT; + /** Cassandra session wrapper instance. */ private volatile CassandraSession ses; @@ -441,6 +447,23 @@ public void setNettyOptions(NettyOptions options) { invalidate(); } + /** + * Sets expiration timeout for Cassandra driver session. Idle sessions that are not + * used during this timeout value will be automatically closed and recreated later + * on demand. + *

+ * If set to {@code 0}, timeout is disabled. + *

+ * Default value is {@link #DFLT_SESSION_EXPIRATION_TIMEOUT}. + * + * @param sessionExpirationTimeout Expiration timeout for Cassandra driver session. + */ + public void setSessionExpirationTimeout(long sessionExpirationTimeout) { + this.sessionExpirationTimeout = sessionExpirationTimeout; + + invalidate(); + } + /** * Creates Cassandra session wrapper if it wasn't created yet and returns it * @@ -523,7 +546,8 @@ public synchronized CassandraSession session(IgniteLogger log) { if (nettyOptions != null) builder = builder.withNettyOptions(nettyOptions); - return ses = new CassandraSessionImpl(builder, fetchSize, readConsistency, writeConsistency, log); + return ses = new CassandraSessionImpl( + builder, fetchSize, readConsistency, writeConsistency, sessionExpirationTimeout, log); } /** diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/CassandraSessionImpl.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/CassandraSessionImpl.java index 95b8581c5c793..cee776bb27363 100644 --- a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/CassandraSessionImpl.java +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/CassandraSessionImpl.java @@ -17,6 +17,13 @@ package org.apache.ignite.cache.store.cassandra.session; +import java.io.IOException; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; +import javax.cache.Cache; import com.datastax.driver.core.BatchStatement; import com.datastax.driver.core.BoundStatement; import com.datastax.driver.core.Cluster; @@ -30,13 +37,6 @@ import com.datastax.driver.core.exceptions.AlreadyExistsException; import com.datastax.driver.core.exceptions.InvalidQueryException; import com.datastax.driver.core.querybuilder.Batch; -import java.io.IOException; -import java.util.HashMap; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.concurrent.atomic.AtomicInteger; -import javax.cache.Cache; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.cache.store.cassandra.common.CassandraHelper; @@ -82,6 +82,9 @@ public class CassandraSessionImpl implements CassandraSession { /** Consistency level for Cassandra WRITE operations (insert/update/delete). */ private ConsistencyLevel writeConsistency; + /** Expiration timeout. */ + private long expirationTimeout; + /** Logger. */ private IgniteLogger log; @@ -101,11 +104,12 @@ public class CassandraSessionImpl implements CassandraSession { * @param log Logger. */ public CassandraSessionImpl(Cluster.Builder builder, Integer fetchSize, ConsistencyLevel readConsistency, - ConsistencyLevel writeConsistency, IgniteLogger log) { + ConsistencyLevel writeConsistency, long expirationTimeout, IgniteLogger log) { this.builder = builder; this.fetchSize = fetchSize; this.readConsistency = readConsistency; this.writeConsistency = writeConsistency; + this.expirationTimeout = expirationTimeout; this.log = log; } @@ -404,7 +408,8 @@ else if (CassandraHelper.isPreparedStatementClusterError(e)) /** {@inheritDoc} */ @Override public synchronized void close() throws IOException { if (decrementSessionRefs() == 0 && ses != null) { - SessionPool.put(this, ses); + SessionPool.put(this, ses, expirationTimeout); + ses = null; } } diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/pool/SessionPool.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/pool/SessionPool.java index fc4a907e6e06e..86db713482160 100644 --- a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/pool/SessionPool.java +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/pool/SessionPool.java @@ -17,13 +17,13 @@ package org.apache.ignite.cache.store.cassandra.session.pool; -import com.datastax.driver.core.Session; import java.lang.Thread.State; import java.util.Collection; import java.util.HashMap; import java.util.LinkedList; import java.util.List; import java.util.Map; +import com.datastax.driver.core.Session; import org.apache.ignite.cache.store.cassandra.session.CassandraSessionImpl; /** @@ -98,14 +98,14 @@ private static class SessionMonitor extends Thread { * @param cassandraSes Session wrapper. * @param driverSes Driver session. */ - public static void put(CassandraSessionImpl cassandraSes, Session driverSes) { + public static void put(CassandraSessionImpl cassandraSes, Session driverSes, long expirationTimeout) { if (cassandraSes == null || driverSes == null) return; SessionWrapper old; synchronized (sessions) { - old = sessions.put(cassandraSes, new SessionWrapper(driverSes)); + old = sessions.put(cassandraSes, new SessionWrapper(driverSes, expirationTimeout)); if (monitorSingleton == null || State.TERMINATED.equals(monitorSingleton.getState())) { monitorSingleton = new SessionMonitor(); diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/pool/SessionWrapper.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/pool/SessionWrapper.java index 7c5722bb8f21f..68b9dd480f8b8 100644 --- a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/pool/SessionWrapper.java +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/pool/SessionWrapper.java @@ -24,12 +24,12 @@ * Wrapper for Cassandra driver session, responsible for monitoring session expiration and its closing. */ public class SessionWrapper { - /** Expiration timeout for Cassandra driver session. */ - public static final long DFLT_EXPIRATION_TIMEOUT = 300000; // 5 minutes. - /** Cassandra driver session. */ private Session ses; + /** Expiration timeout. */ + private long expirationTimeout; + /** Wrapper creation time. */ private long time; @@ -38,9 +38,11 @@ public class SessionWrapper { * * @param ses Cassandra driver session. */ - public SessionWrapper(Session ses) { + public SessionWrapper(Session ses, long expirationTimeout) { this.ses = ses; - this.time = System.currentTimeMillis(); + this.expirationTimeout = expirationTimeout; + + time = System.currentTimeMillis(); } /** @@ -49,7 +51,7 @@ public SessionWrapper(Session ses) { * @return true if session expired. */ public boolean expired() { - return System.currentTimeMillis() - time > DFLT_EXPIRATION_TIMEOUT; + return expirationTimeout > 0 && System.currentTimeMillis() - time > expirationTimeout; } /** @@ -66,6 +68,7 @@ public Session driverSession() { */ public void release() { CassandraHelper.closeSession(ses); + ses = null; } } From f9ecacc625b458539775e6550bd9b7613ed38f21 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Fri, 28 Apr 2017 11:46:23 +0300 Subject: [PATCH 08/90] IGNITE-5077 - Support service security permissions backport from master (cherry picked from commit 6236b5f) --- .../processors/security/SecurityContext.java | 9 ++++++ .../service/GridServiceProcessor.java | 11 +++++++ .../security/SecurityBasicPermissionSet.java | 17 ++++++++++ .../plugin/security/SecurityPermission.java | 13 ++++++-- .../security/SecurityPermissionSet.java | 8 +++++ .../SecurityPermissionSetBuilder.java | 19 +++++++++++ .../SecurityPermissionSetBuilderTest.java | 32 +++++++++++++++---- .../junits/spi/GridSpiAbstractTest.java | 5 +++ 8 files changed, 106 insertions(+), 8 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityContext.java index ef46713a72f63..bf5894ef3f48c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityContext.java @@ -47,6 +47,15 @@ public interface SecurityContext { */ public boolean cacheOperationAllowed(String cacheName, SecurityPermission perm); + /** + * Checks whether service operation is allowed. + * + * @param srvcName Service name. + * @param perm Permission to check. + * @return {@code True} if task operation is allowed. + */ + public boolean serviceOperationAllowed(String srvcName, SecurityPermission perm); + /** * Checks whether system-wide permission is allowed (excluding Visor task operations). * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java index 2a363e21e8ef5..d7b9abc7fc1bd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java @@ -92,6 +92,7 @@ import org.apache.ignite.lang.IgniteProductVersion; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.marshaller.Marshaller; +import org.apache.ignite.plugin.security.SecurityPermission; import org.apache.ignite.resources.IgniteInstanceResource; import org.apache.ignite.resources.JobContextResource; import org.apache.ignite.resources.LoggerResource; @@ -496,6 +497,8 @@ public IgniteInternalFuture deploy(ServiceConfiguration cfg) { validate(cfg); + ctx.security().authorize(cfg.getName(), SecurityPermission.SERVICE_DEPLOY, null); + if (!state.srvcCompatibility) { Marshaller marsh = ctx.config().getMarshaller(); @@ -632,6 +635,8 @@ private ServicesCompatibilityState markCompatibilityStateAsUsed() { * @return Future. */ public IgniteInternalFuture cancel(String name) { + ctx.security().authorize(name, SecurityPermission.SERVICE_CANCEL, null); + while (true) { try { GridFutureAdapter fut = new GridFutureAdapter<>(); @@ -780,6 +785,8 @@ public Collection serviceDescriptors() { */ @SuppressWarnings("unchecked") public T service(String name) { + ctx.security().authorize(name, SecurityPermission.SERVICE_INVOKE, null); + Collection ctxs; synchronized (locSvcs) { @@ -844,6 +851,8 @@ public ServiceContextImpl serviceContext(String name) { @SuppressWarnings("unchecked") public T serviceProxy(ClusterGroup prj, String name, Class svcItf, boolean sticky, long timeout) throws IgniteException { + ctx.security().authorize(name, SecurityPermission.SERVICE_INVOKE, null); + if (hasLocalNode(prj)) { ServiceContextImpl ctx = serviceContext(name); @@ -883,6 +892,8 @@ private boolean hasLocalNode(ClusterGroup prj) { */ @SuppressWarnings("unchecked") public Collection services(String name) { + ctx.security().authorize(name, SecurityPermission.SERVICE_INVOKE, null); + Collection ctxs; synchronized (locSvcs) { diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityBasicPermissionSet.java b/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityBasicPermissionSet.java index 5b50c56321377..7521dff84df4f 100644 --- a/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityBasicPermissionSet.java +++ b/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityBasicPermissionSet.java @@ -38,6 +38,9 @@ public class SecurityBasicPermissionSet implements SecurityPermissionSet { /** Task permissions. */ private Map> taskPerms = new HashMap<>(); + /** Service permissions. */ + private Map> srvcPerms = new HashMap<>(); + /** System permissions. */ private Collection sysPerms = new ArrayList<>(); @@ -62,6 +65,15 @@ public void setTaskPermissions(Map> taskP this.taskPerms = taskPerms; } + /** + * Setter for set service permission map. + * + * @param srvcPerms Service permissions. + */ + public void setServicePermissions(Map> srvcPerms) { + this.srvcPerms = srvcPerms; + } + /** * Setter for set collection system permission. * @@ -90,6 +102,11 @@ public void setDefaultAllowAll(boolean dfltAllowAll) { return taskPerms; } + /** {@inheritDoc} */ + @Override public Map> servicePermissions() { + return srvcPerms; + } + /** {@inheritDoc} */ @Nullable @Override public Collection systemPermissions() { return sysPerms; diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermission.java b/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermission.java index 9f63c1ed1d2a2..54361614e62a4 100644 --- a/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermission.java +++ b/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermission.java @@ -21,7 +21,7 @@ /** * Supported security permissions within grid. Permissions - * are specified on per-cache or per-task level. + * are specified on per-cache, per-task or per-service level. */ public enum SecurityPermission { /** Cache {@code read} permission. */ @@ -55,7 +55,16 @@ public enum SecurityPermission { ADMIN_CACHE, /** Visor admin operations permissions. */ - ADMIN_OPS; + ADMIN_OPS, + + /** Service deploy permission. */ + SERVICE_DEPLOY, + + /** Service cancel permission. */ + SERVICE_CANCEL, + + /** Service invoke permission. */ + SERVICE_INVOKE; /** Enumerated values. */ private static final SecurityPermission[] VALS = values(); diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermissionSet.java b/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermissionSet.java index 99615019c10d2..5e07e421eaf78 100644 --- a/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermissionSet.java +++ b/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermissionSet.java @@ -58,6 +58,14 @@ public interface SecurityPermissionSet extends Serializable, LessNamingBean { */ public Map> cachePermissions(); + /** + * Map of service names to service permissions. Wildcards are allowed at the + * end of service names. + * + * @return Map of service names to service permissions. + */ + public Map> servicePermissions(); + /** * Collection of system-wide permissions (events enable/disable, Visor task execution). * diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilder.java b/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilder.java index 61ad77ce08f5e..cf38c0f0e7e90 100644 --- a/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilder.java +++ b/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilder.java @@ -57,6 +57,9 @@ public class SecurityPermissionSetBuilder { /** Task permissions.*/ private Map> taskPerms = new HashMap<>(); + /** Service permissions.*/ + private Map> srvcPerms = new HashMap<>(); + /** System permissions.*/ private List sysPerms = new ArrayList<>(); @@ -99,6 +102,21 @@ public SecurityPermissionSetBuilder appendTaskPermissions(String name, SecurityP return this; } + /** + * Append permission set form {@link org.apache.ignite.IgniteServices service} with {@code name}. + * + * @param name String for map some service to permission set. + * @param perms Permissions. + * @return SecurityPermissionSetBuilder refer to same permission builder. + */ + public SecurityPermissionSetBuilder appendServicePermissions(String name, SecurityPermission... perms) { + validate(toCollection("SERVICE_"), perms); + + append(srvcPerms, name, toCollection(perms)); + + return this; + } + /** * Append permission set form {@link org.apache.ignite.IgniteCache cache} with {@code name}. * @@ -215,6 +233,7 @@ public SecurityPermissionSet build() { permSet.setDefaultAllowAll(dfltAllowAll); permSet.setCachePermissions(unmodifiableMap(cachePerms)); permSet.setTaskPermissions(unmodifiableMap(taskPerms)); + permSet.setServicePermissions(unmodifiableMap(srvcPerms)); permSet.setSystemPermissions(unmodifiableList(sysPerms)); return permSet; diff --git a/modules/core/src/test/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilderTest.java b/modules/core/src/test/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilderTest.java index f63f9a7da0a01..5443cfd77fd5c 100644 --- a/modules/core/src/test/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilderTest.java +++ b/modules/core/src/test/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilderTest.java @@ -28,6 +28,8 @@ import static org.apache.ignite.plugin.security.SecurityPermission.CACHE_PUT; import static org.apache.ignite.plugin.security.SecurityPermission.CACHE_READ; import static org.apache.ignite.plugin.security.SecurityPermission.CACHE_REMOVE; +import static org.apache.ignite.plugin.security.SecurityPermission.SERVICE_DEPLOY; +import static org.apache.ignite.plugin.security.SecurityPermission.SERVICE_INVOKE; import static org.apache.ignite.plugin.security.SecurityPermission.TASK_CANCEL; import static org.apache.ignite.plugin.security.SecurityPermission.TASK_EXECUTE; import static org.apache.ignite.plugin.security.SecurityPermission.EVENTS_ENABLE; @@ -41,6 +43,7 @@ public class SecurityPermissionSetBuilderTest extends GridCommonAbstractTest { /** * */ + @SuppressWarnings({"ThrowableNotThrown", "ArraysAsListWithZeroOrOneArgument"}) public void testPermissionBuilder() { SecurityBasicPermissionSet exp = new SecurityBasicPermissionSet(); @@ -56,13 +59,18 @@ public void testPermissionBuilder() { exp.setTaskPermissions(permTask); + Map> permSrvc = new HashMap<>(); + permSrvc.put("service1", Arrays.asList(SERVICE_DEPLOY)); + permSrvc.put("service2", Arrays.asList(SERVICE_INVOKE)); + + exp.setServicePermissions(permSrvc); + exp.setSystemPermissions(Arrays.asList(ADMIN_VIEW, EVENTS_ENABLE)); final SecurityPermissionSetBuilder permsBuilder = new SecurityPermissionSetBuilder(); assertThrows(log, new Callable() { - @Override - public Object call() throws Exception { + @Override public Object call() throws Exception { permsBuilder.appendCachePermissions("cache", ADMIN_VIEW); return null; } @@ -71,8 +79,7 @@ public Object call() throws Exception { ); assertThrows(log, new Callable() { - @Override - public Object call() throws Exception { + @Override public Object call() throws Exception { permsBuilder.appendTaskPermissions("task", CACHE_READ); return null; } @@ -81,8 +88,7 @@ public Object call() throws Exception { ); assertThrows(log, new Callable() { - @Override - public Object call() throws Exception { + @Override public Object call() throws Exception { permsBuilder.appendSystemPermissions(TASK_EXECUTE, CACHE_PUT); return null; } @@ -90,6 +96,15 @@ public Object call() throws Exception { "you can assign permission only start with [EVENTS_, ADMIN_], but you try TASK_EXECUTE" ); + assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + permsBuilder.appendSystemPermissions(SERVICE_INVOKE, CACHE_REMOVE); + return null; + } + }, IgniteException.class, + "you can assign permission only start with [EVENTS_, ADMIN_], but you try SERVICE_INVOKE" + ); + permsBuilder.appendCachePermissions( "cache1", CACHE_PUT, CACHE_REMOVE ).appendCachePermissions( @@ -98,12 +113,17 @@ public Object call() throws Exception { "task1", TASK_CANCEL ).appendTaskPermissions( "task2", TASK_EXECUTE + ).appendServicePermissions( + "service1", SERVICE_DEPLOY + ).appendServicePermissions( + "service2", SERVICE_INVOKE ).appendSystemPermissions(ADMIN_VIEW, EVENTS_ENABLE); SecurityPermissionSet actual = permsBuilder.build(); assertEquals(exp.cachePermissions(), actual.cachePermissions()); assertEquals(exp.taskPermissions(), actual.taskPermissions()); + assertEquals(exp.servicePermissions(), actual.servicePermissions()); assertEquals(exp.systemPermissions(), actual.systemPermissions()); assertEquals(exp.defaultAllowAll(), actual.defaultAllowAll()); } diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/spi/GridSpiAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/spi/GridSpiAbstractTest.java index 20b3cf2fdb354..0aeff3c08b78a 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/spi/GridSpiAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/spi/GridSpiAbstractTest.java @@ -717,6 +717,11 @@ private static class SecurityPermissionSetImpl implements SecurityPermissionSet return Collections.emptyMap(); } + /** {@inheritDoc} */ + @Override public Map> servicePermissions() { + return Collections.emptyMap(); + } + /** {@inheritDoc} */ @Nullable @Override public Collection systemPermissions() { return null; From 91c899b909383c78b78b9bf0c8f233b8c75ef29e Mon Sep 17 00:00:00 2001 From: Valentin Kulichenko Date: Fri, 28 Apr 2017 14:48:57 +0200 Subject: [PATCH 09/90] IGNITE-5081 - Removed redundant duplication of permissions in SecurityPermissionSetBuilder --- .../SecurityPermissionSetBuilder.java | 17 ++--- .../SecurityPermissionSetBuilderTest.java | 63 +++++++++++-------- 2 files changed, 46 insertions(+), 34 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilder.java b/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilder.java index cf38c0f0e7e90..abac541ffdb68 100644 --- a/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilder.java +++ b/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilder.java @@ -17,16 +17,17 @@ package org.apache.ignite.plugin.security; -import java.util.Map; -import java.util.List; -import java.util.HashMap; -import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; import org.apache.ignite.IgniteException; +import org.apache.ignite.internal.util.typedef.internal.U; -import static java.util.Collections.unmodifiableList; import static java.util.Collections.unmodifiableMap; +import static java.util.Collections.unmodifiableSet; /** * Provides a convenient way to create a permission set. @@ -61,7 +62,7 @@ public class SecurityPermissionSetBuilder { private Map> srvcPerms = new HashMap<>(); /** System permissions.*/ - private List sysPerms = new ArrayList<>(); + private Set sysPerms = new HashSet<>(); /** Default allow all.*/ private boolean dfltAllowAll; @@ -193,7 +194,7 @@ private void validate(Collection ptrns, SecurityPermission perm) { private final Collection toCollection(T... perms) { assert perms != null; - Collection col = new ArrayList<>(perms.length); + Collection col = U.newHashSet(perms.length); Collections.addAll(col, perms); @@ -234,7 +235,7 @@ public SecurityPermissionSet build() { permSet.setCachePermissions(unmodifiableMap(cachePerms)); permSet.setTaskPermissions(unmodifiableMap(taskPerms)); permSet.setServicePermissions(unmodifiableMap(srvcPerms)); - permSet.setSystemPermissions(unmodifiableList(sysPerms)); + permSet.setSystemPermissions(unmodifiableSet(sysPerms)); return permSet; } diff --git a/modules/core/src/test/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilderTest.java b/modules/core/src/test/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilderTest.java index 5443cfd77fd5c..0ac7bc739f179 100644 --- a/modules/core/src/test/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilderTest.java +++ b/modules/core/src/test/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilderTest.java @@ -17,23 +17,24 @@ package org.apache.ignite.plugin.security; -import java.util.Map; -import java.util.Arrays; -import java.util.HashMap; import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; import java.util.concurrent.Callable; import org.apache.ignite.IgniteException; +import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import static org.apache.ignite.plugin.security.SecurityPermission.ADMIN_VIEW; import static org.apache.ignite.plugin.security.SecurityPermission.CACHE_PUT; import static org.apache.ignite.plugin.security.SecurityPermission.CACHE_READ; import static org.apache.ignite.plugin.security.SecurityPermission.CACHE_REMOVE; import static org.apache.ignite.plugin.security.SecurityPermission.SERVICE_DEPLOY; import static org.apache.ignite.plugin.security.SecurityPermission.SERVICE_INVOKE; +import static org.apache.ignite.plugin.security.SecurityPermission.EVENTS_ENABLE; import static org.apache.ignite.plugin.security.SecurityPermission.TASK_CANCEL; import static org.apache.ignite.plugin.security.SecurityPermission.TASK_EXECUTE; -import static org.apache.ignite.plugin.security.SecurityPermission.EVENTS_ENABLE; -import static org.apache.ignite.plugin.security.SecurityPermission.ADMIN_VIEW; import static org.apache.ignite.testframework.GridTestUtils.assertThrows; /** @@ -41,31 +42,30 @@ */ public class SecurityPermissionSetBuilderTest extends GridCommonAbstractTest { /** - * */ @SuppressWarnings({"ThrowableNotThrown", "ArraysAsListWithZeroOrOneArgument"}) public void testPermissionBuilder() { SecurityBasicPermissionSet exp = new SecurityBasicPermissionSet(); Map> permCache = new HashMap<>(); - permCache.put("cache1", Arrays.asList(CACHE_PUT, CACHE_REMOVE)); - permCache.put("cache2", Arrays.asList(CACHE_READ)); + permCache.put("cache1", permissions(CACHE_PUT, CACHE_REMOVE)); + permCache.put("cache2", permissions(CACHE_READ)); exp.setCachePermissions(permCache); Map> permTask = new HashMap<>(); - permTask.put("task1", Arrays.asList(TASK_CANCEL)); - permTask.put("task2", Arrays.asList(TASK_EXECUTE)); + permTask.put("task1", permissions(TASK_CANCEL)); + permTask.put("task2", permissions(TASK_EXECUTE)); exp.setTaskPermissions(permTask); Map> permSrvc = new HashMap<>(); - permSrvc.put("service1", Arrays.asList(SERVICE_DEPLOY)); - permSrvc.put("service2", Arrays.asList(SERVICE_INVOKE)); + permSrvc.put("service1", permissions(SERVICE_DEPLOY)); + permSrvc.put("service2", permissions(SERVICE_INVOKE)); exp.setServicePermissions(permSrvc); - exp.setSystemPermissions(Arrays.asList(ADMIN_VIEW, EVENTS_ENABLE)); + exp.setSystemPermissions(permissions(ADMIN_VIEW, EVENTS_ENABLE)); final SecurityPermissionSetBuilder permsBuilder = new SecurityPermissionSetBuilder(); @@ -105,19 +105,18 @@ public void testPermissionBuilder() { "you can assign permission only start with [EVENTS_, ADMIN_], but you try SERVICE_INVOKE" ); - permsBuilder.appendCachePermissions( - "cache1", CACHE_PUT, CACHE_REMOVE - ).appendCachePermissions( - "cache2", CACHE_READ - ).appendTaskPermissions( - "task1", TASK_CANCEL - ).appendTaskPermissions( - "task2", TASK_EXECUTE - ).appendServicePermissions( - "service1", SERVICE_DEPLOY - ).appendServicePermissions( - "service2", SERVICE_INVOKE - ).appendSystemPermissions(ADMIN_VIEW, EVENTS_ENABLE); + permsBuilder + .appendCachePermissions("cache1", CACHE_PUT) + .appendCachePermissions("cache1", CACHE_PUT, CACHE_REMOVE) + .appendCachePermissions("cache2", CACHE_READ) + .appendTaskPermissions("task1", TASK_CANCEL) + .appendTaskPermissions("task2", TASK_EXECUTE) + .appendTaskPermissions("task2", TASK_EXECUTE) + .appendServicePermissions("service1", SERVICE_DEPLOY) + .appendServicePermissions("service2", SERVICE_INVOKE) + .appendServicePermissions("service2", SERVICE_INVOKE) + .appendSystemPermissions(ADMIN_VIEW) + .appendSystemPermissions(ADMIN_VIEW, EVENTS_ENABLE); SecurityPermissionSet actual = permsBuilder.build(); @@ -127,4 +126,16 @@ public void testPermissionBuilder() { assertEquals(exp.systemPermissions(), actual.systemPermissions()); assertEquals(exp.defaultAllowAll(), actual.defaultAllowAll()); } + + /** + * @param perms Permissions. + * @return Collection. + */ + private static Collection permissions(SecurityPermission... perms) { + Collection col = U.newHashSet(perms.length); + + Collections.addAll(col, perms); + + return col; + } } From b48a26b9b1e97fb8eb52c2a2f36005770922ac3d Mon Sep 17 00:00:00 2001 From: Valentin Kulichenko Date: Fri, 28 Apr 2017 14:53:33 +0200 Subject: [PATCH 10/90] IGNITE-5080 - Fixes in SecurityBasicPermissionSet --- .../security/SecurityBasicPermissionSet.java | 89 ++++++++++++++----- 1 file changed, 65 insertions(+), 24 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityBasicPermissionSet.java b/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityBasicPermissionSet.java index 7521dff84df4f..44166d936c9aa 100644 --- a/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityBasicPermissionSet.java +++ b/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityBasicPermissionSet.java @@ -17,10 +17,12 @@ package org.apache.ignite.plugin.security; -import java.util.Map; -import java.util.HashMap; -import java.util.ArrayList; import java.util.Collection; +import java.util.HashMap; +import java.util.Map; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.S; import org.jetbrains.annotations.Nullable; @@ -33,16 +35,20 @@ public class SecurityBasicPermissionSet implements SecurityPermissionSet { private static final long serialVersionUID = 0L; /** Cache permissions. */ - private Map> cachePerms = new HashMap<>(); + @GridToStringInclude + private Map> cachePermissions = new HashMap<>(); /** Task permissions. */ - private Map> taskPerms = new HashMap<>(); + @GridToStringInclude + private Map> taskPermissions = new HashMap<>(); /** Service permissions. */ - private Map> srvcPerms = new HashMap<>(); + @GridToStringInclude + private Map> servicePermissions = new HashMap<>(); /** System permissions. */ - private Collection sysPerms = new ArrayList<>(); + @GridToStringInclude + private Collection systemPermissions; /** Default allow all. */ private boolean dfltAllowAll; @@ -50,37 +56,43 @@ public class SecurityBasicPermissionSet implements SecurityPermissionSet { /** * Setter for set cache permission map. * - * @param cachePerms Cache permissions. + * @param cachePermissions Cache permissions. */ - public void setCachePermissions(Map> cachePerms) { - this.cachePerms = cachePerms; + public void setCachePermissions(Map> cachePermissions) { + A.notNull(cachePermissions, "cachePermissions"); + + this.cachePermissions = cachePermissions; } /** * Setter for set task permission map. * - * @param taskPerms Task permissions. + * @param taskPermissions Task permissions. */ - public void setTaskPermissions(Map> taskPerms) { - this.taskPerms = taskPerms; + public void setTaskPermissions(Map> taskPermissions) { + A.notNull(taskPermissions, "taskPermissions"); + + this.taskPermissions = taskPermissions; } /** * Setter for set service permission map. * - * @param srvcPerms Service permissions. + * @param servicePermissions Service permissions. */ - public void setServicePermissions(Map> srvcPerms) { - this.srvcPerms = srvcPerms; + public void setServicePermissions(Map> servicePermissions) { + A.notNull(taskPermissions, "servicePermissions"); + + this.servicePermissions = servicePermissions; } /** - * Setter for set collection system permission. + * Setter for set collection system permission. * - * @param sysPerms System permissions. + * @param systemPermissions System permissions. */ - public void setSystemPermissions(Collection sysPerms) { - this.sysPerms = sysPerms; + public void setSystemPermissions(Collection systemPermissions) { + this.systemPermissions = systemPermissions; } /** @@ -94,22 +106,22 @@ public void setDefaultAllowAll(boolean dfltAllowAll) { /** {@inheritDoc} */ @Override public Map> cachePermissions() { - return cachePerms; + return cachePermissions; } /** {@inheritDoc} */ @Override public Map> taskPermissions() { - return taskPerms; + return taskPermissions; } /** {@inheritDoc} */ @Override public Map> servicePermissions() { - return srvcPerms; + return servicePermissions; } /** {@inheritDoc} */ @Nullable @Override public Collection systemPermissions() { - return sysPerms; + return systemPermissions; } /** {@inheritDoc} */ @@ -117,6 +129,35 @@ public void setDefaultAllowAll(boolean dfltAllowAll) { return dfltAllowAll; } + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + + if (!(o instanceof SecurityBasicPermissionSet)) + return false; + + SecurityBasicPermissionSet other = (SecurityBasicPermissionSet)o; + + return dfltAllowAll == other.dfltAllowAll && + F.eq(cachePermissions, other.cachePermissions) && + F.eq(taskPermissions, other.taskPermissions) && + F.eq(servicePermissions, other.servicePermissions) && + F.eq(systemPermissions, other.systemPermissions); + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + int res = (dfltAllowAll ? 1 : 0); + + res = 31 * res + (cachePermissions != null ? cachePermissions.hashCode() : 0); + res = 31 * res + (taskPermissions != null ? taskPermissions.hashCode() : 0); + res = 31 * res + (servicePermissions != null ? servicePermissions.hashCode() : 0); + res = 31 * res + (systemPermissions != null ? systemPermissions.hashCode() : 0); + + return res; + } + /** {@inheritDoc} */ @Override public String toString() { return S.toString(SecurityBasicPermissionSet.class, this); From f66c23cbb9a6f2c923ebf75c58f00afaf1c0b5f3 Mon Sep 17 00:00:00 2001 From: Evgenii Zhuravlev Date: Wed, 3 May 2017 17:47:45 +0300 Subject: [PATCH 11/90] IGNITE-4939 Receive event before cache initialized fix --- .../processors/cache/GridCacheProcessor.java | 20 +++++++++++ .../cache/IgniteCacheProxyStub.java | 33 +++++++++++++++++++ .../CacheContinuousQueryHandler.java | 3 +- 3 files changed, 55 insertions(+), 1 deletion(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyStub.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 0be2072e3766d..5a54e53dbd3f5 100755 --- 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 @@ -3345,6 +3345,26 @@ public IgniteCacheProxy jcache(@Nullable String name) { return cache; } + /** + * @param name Cache name. + * @param id Cache id. + * @return Cache instance for given name or stub if cache not created yet, but cache descriptor exists. + */ + @SuppressWarnings("unchecked") + public IgniteCacheProxy safeJcache(String name, int id) { + assert name != null; + + IgniteCacheProxy cache = (IgniteCacheProxy)jCacheProxies.get(name); + + if (cache == null) + if (cacheDescriptor(id) != null && CU.isSystemCache(name)) + cache = new IgniteCacheProxyStub(); + else + throw new IllegalArgumentException("Cache is not configured: " + name); + + return cache; + } + /** * @return All configured public cache instances. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyStub.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyStub.java new file mode 100644 index 0000000000000..75489edb98e74 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyStub.java @@ -0,0 +1,33 @@ +/* + * 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.cache; + +/** + * Cache proxy stub. + */ +public class IgniteCacheProxyStub extends IgniteCacheProxy{ + /** */ + private static final long serialVersionUID = 0L; + + /** + * Empty constructor. + */ + public IgniteCacheProxyStub() { + // No-op. + } +} \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java index 83edab444f8c0..1a6577d241774 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java @@ -716,7 +716,8 @@ private void notifyCallback0(UUID nodeId, GridCacheContext cctx = cacheContext(ctx); - final IgniteCache cache = cctx.kernalContext().cache().jcache(cctx.name()); + //returns stub if system cache not created yet + final IgniteCache cache = cctx.kernalContext().cache().safeJcache(cctx.name(), cctx.cacheId()); if (internal) { if (e.isFiltered()) From 45b4d6316145d0b4b46713409f5e8fbe55ff4c41 Mon Sep 17 00:00:00 2001 From: Evgenii Zhuravlev Date: Thu, 4 May 2017 12:11:37 +0300 Subject: [PATCH 12/90] IGNITE-4939 Receive event before cache initialized fix --- .../ignite/internal/processors/cache/GridCacheProcessor.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) 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 5a54e53dbd3f5..ccd7ae0bf500d 100755 --- 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 @@ -3352,9 +3352,7 @@ public IgniteCacheProxy jcache(@Nullable String name) { */ @SuppressWarnings("unchecked") public IgniteCacheProxy safeJcache(String name, int id) { - assert name != null; - - IgniteCacheProxy cache = (IgniteCacheProxy)jCacheProxies.get(name); + IgniteCacheProxy cache = (IgniteCacheProxy)jCacheProxies.get(maskNull(name)); if (cache == null) if (cacheDescriptor(id) != null && CU.isSystemCache(name)) From 075bcfca0ea22633be13cd02647e359ad6fdca16 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Thu, 4 May 2017 12:21:04 +0300 Subject: [PATCH 13/90] Fix flacky service deployment tests. --- .../GridServiceProcessorAbstractSelfTest.java | 17 ++++++ ...rviceProcessorMultiNodeConfigSelfTest.java | 46 +++++++-------- ...GridServiceProcessorMultiNodeSelfTest.java | 56 ++++++++++--------- 3 files changed, 68 insertions(+), 51 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorAbstractSelfTest.java index 0f79855518e3d..6d91f36aab00b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorAbstractSelfTest.java @@ -30,7 +30,9 @@ import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.processors.affinity.GridAffinityProcessor; +import org.apache.ignite.internal.util.lang.GridAbsPredicateX; import org.apache.ignite.internal.util.typedef.CA; import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.lang.IgniteFuture; @@ -519,6 +521,21 @@ protected int actualCount(String svcName, Iterable descs) { return sum; } + /** + * @param srvcName Service name + * @param expectedDeps Expected number of service deployments + * + */ + protected boolean waitForDeployment(final String srvcName, final int expectedDeps) throws IgniteInterruptedCheckedException { + final Ignite g = randomGrid(); + + return GridTestUtils.waitForCondition(new GridAbsPredicateX() { + @Override public boolean applyx() { + return actualCount(srvcName, g.services().serviceDescriptors()) == expectedDeps; + } + }, 1500); + } + /** * Counter service. */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeConfigSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeConfigSelfTest.java index 9da62c0abb702..c78bcaae607ac 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeConfigSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeConfigSelfTest.java @@ -22,7 +22,6 @@ import java.util.concurrent.CountDownLatch; import org.apache.ignite.Ignite; import org.apache.ignite.configuration.CacheConfiguration; -import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.util.lang.GridAbsPredicateX; import org.apache.ignite.services.ServiceConfiguration; import org.apache.ignite.testframework.GridTestUtils; @@ -56,7 +55,6 @@ public class GridServiceProcessorMultiNodeConfigSelfTest extends GridServiceProc /** {@inheritDoc} */ @Override protected ServiceConfiguration[] services() { - List cfgs = new ArrayList<>(); ServiceConfiguration cfg = new ServiceConfiguration(); @@ -65,6 +63,8 @@ public class GridServiceProcessorMultiNodeConfigSelfTest extends GridServiceProc cfg.setTotalCount(1); cfg.setService(new DummyService()); + List cfgs = new ArrayList<>(); + cfgs.add(cfg); cfg = new ServiceConfiguration(); @@ -202,43 +202,31 @@ public void testDeployLimits() throws Exception { checkCount(name, g.services().serviceDescriptors(), nodeCount()); - int extraNodes = 2; - CountDownLatch latch = new CountDownLatch(1); DummyService.exeLatch(name, latch); + int extraNodes = 2; + startExtraNodes(extraNodes); try { latch.await(); + waitForDeployment(name, nodeCount() + 1); + checkCount(name, g.services().serviceDescriptors(), nodeCount() + 1); } finally { stopExtraNodes(extraNodes); } - assertEquals(name, 1, DummyService.cancelled(name)); - waitForDeployment(name, nodeCount()); - checkCount(name, g.services().serviceDescriptors(), nodeCount()); - } - - /** - * @param srvcName Service name - * @param expectedDeps Expected number of service deployments - * - */ - private boolean waitForDeployment(final String srvcName, final int expectedDeps) throws IgniteInterruptedCheckedException { - final Ignite g = randomGrid(); + // Service can be redeployed when nodes is stopping one-by-one. + assertEquals(0, DummyService.started(name) - DummyService.cancelled(name)); - return GridTestUtils.waitForCondition(new GridAbsPredicateX() { - @Override public boolean applyx() { - return actualCount(srvcName, g.services().serviceDescriptors()) == expectedDeps; - } - }, 1500); + checkCount(name, g.services().serviceDescriptors(), nodeCount()); } /** @@ -281,8 +269,12 @@ private void checkDeployOnEachNodeUpdateTopology(String name) throws Exception { try { latch.await(); - assertEquals(name, newNodes, DummyService.started(name)); - assertEquals(name, 0, DummyService.cancelled(name)); + waitForDeployment(name, nodeCount() + newNodes); + + // Since we start extra nodes, there may be extra start and cancel events, + // so we check only the difference between start and cancel and + // not start and cancel events individually. + assertEquals(name, newNodes, DummyService.started(name) - DummyService.cancelled(name)); checkCount(name, g.services().serviceDescriptors(), nodeCount() + newNodes); } @@ -314,8 +306,12 @@ private void checkDeployOnEachNodeButClientUpdateTopology(String name) throws Ex try { latch.await(); - assertEquals(name, servers, DummyService.started(name)); - assertEquals(name, 0, DummyService.cancelled(name)); + waitForDeployment(name, nodeCount() + servers); + + // Since we start extra nodes, there may be extra start and cancel events, + // so we check only the difference between start and cancel and + // not start and cancel events individually. + assertEquals(name, servers, DummyService.started(name) - DummyService.cancelled(name)); checkCount(name, g.services().serviceDescriptors(), nodeCount() + servers); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeSelfTest.java index d133cf299a756..8ecceb9a65d39 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeSelfTest.java @@ -92,7 +92,7 @@ public void testAffinityDeployUpdateTopology() throws Exception { // Store a cache key. g.cache(CACHE_NAME).put(affKey, affKey.toString()); - String name = "serviceAffinityUpdateTopology"; + final String name = "serviceAffinityUpdateTopology"; IgniteServices svcs = g.services().withAsync(); @@ -129,9 +129,7 @@ public void testDeployOnEachNodeButClientUpdateTopology() throws Exception { Ignite client = startGrid("client", getConfiguration("client").setClientMode(true)); try { - final int prestartedNodes = nodeCount() + 1; - - String name = "serviceOnEachNodeButClientUpdateTopology"; + final String name = "serviceOnEachNodeButClientUpdateTopology"; Ignite g = randomGrid(); @@ -156,8 +154,8 @@ public void testDeployOnEachNodeButClientUpdateTopology() throws Exception { // Ensure service is deployed assertNotNull(client.services().serviceProxy(name, Service.class, false, 2000)); - TestCase.assertEquals(name, nodeCount(), DummyService.started(name)); - TestCase.assertEquals(name, 0, DummyService.cancelled(name)); + assertEquals(name, nodeCount(), DummyService.started(name)); + assertEquals(name, 0, DummyService.cancelled(name)); int servers = 2; int clients = 2; @@ -171,12 +169,12 @@ public void testDeployOnEachNodeButClientUpdateTopology() throws Exception { try { latch.await(); - // Ensure service is deployed - assertNotNull(grid(prestartedNodes + servers - 1) - .services().serviceProxy(name, Service.class, false, 2000)); + waitForDeployment(name, servers); - TestCase.assertEquals(name, nodeCount() + servers, DummyService.started(name)); - TestCase.assertEquals(name, 0, DummyService.cancelled(name)); + // Since we start extra nodes, there may be extra start and cancel events, + // so we check only the difference between start and cancel and + // not start and cancel events individually. + assertEquals(name, nodeCount() + servers, DummyService.started(name) - DummyService.cancelled(name)); checkCount(name, g.services().serviceDescriptors(), nodeCount() + servers); } @@ -197,7 +195,7 @@ public void testDeployOnEachNodeUpdateTopology() throws Exception { Ignite client = startGrid("client", getConfiguration("client").setClientMode(true)); try { - String name = "serviceOnEachNodeUpdateTopology"; + final String name = "serviceOnEachNodeUpdateTopology"; Ignite g = randomGrid(); @@ -231,8 +229,8 @@ public void testDeployOnEachNodeUpdateTopology() throws Exception { // Ensure service is deployed assertNotNull(client.services().serviceProxy(name, Service.class, false, 2000)); - TestCase.assertEquals(name, prestartedNodes, DummyService.started(name)); - TestCase.assertEquals(name, 0, DummyService.cancelled(name)); + assertEquals(name, prestartedNodes, DummyService.started(name)); + assertEquals(name, 0, DummyService.cancelled(name)); int servers = 2; int clients = 2; @@ -248,11 +246,13 @@ public void testDeployOnEachNodeUpdateTopology() throws Exception { try { latch.await(); - // Ensure service is deployed - assertNotNull(client.services().serviceProxy(name, Service.class, false, 2000)); + waitForDeployment(name, prestartedNodes + extraNodes); - TestCase.assertEquals(name, prestartedNodes + extraNodes, DummyService.started(name)); - TestCase.assertEquals(name, 0, DummyService.cancelled(name)); + // Since we start extra nodes, there may be extra start and cancel events, + // so we check only the difference between start and cancel and + // not start and cancel events individually. + assertEquals(name, prestartedNodes + extraNodes, + DummyService.started(name) - DummyService.cancelled(name)); checkCount(name, g.services().serviceDescriptors(), prestartedNodes + extraNodes); } @@ -269,7 +269,7 @@ public void testDeployOnEachNodeUpdateTopology() throws Exception { * @throws Exception If failed. */ public void testDeployLimits() throws Exception { - String name = "serviceWithLimitsUpdateTopology"; + final String name = "serviceWithLimitsUpdateTopology"; Ignite g = randomGrid(); @@ -300,24 +300,28 @@ public void testDeployLimits() throws Exception { latch.await(); - TestCase.assertEquals(name, nodeCount(), DummyService.started(name)); - TestCase.assertEquals(name, 0, DummyService.cancelled(name)); + assertEquals(name, nodeCount(), DummyService.started(name)); + assertEquals(name, 0, DummyService.cancelled(name)); checkCount(name, g.services().serviceDescriptors(), nodeCount()); - int extraNodes = 2; - latch = new CountDownLatch(1); DummyService.exeLatch(name, latch); - startExtraNodes(2); + int extraNodes = 2; + + startExtraNodes(extraNodes); try { latch.await(); - TestCase.assertEquals(name, totalInstances, DummyService.started(name)); - TestCase.assertEquals(name, 0, DummyService.cancelled(name)); + waitForDeployment(name, totalInstances); + + // Since we start extra nodes, there may be extra start and cancel events, + // so we check only the difference between start and cancel and + // not start and cancel events individually. + assertEquals(name, totalInstances, DummyService.started(name) - DummyService.cancelled(name)); checkCount(name, g.services().serviceDescriptors(), totalInstances); } From 25c06b50d46937cb39534cdf4147b862217289a2 Mon Sep 17 00:00:00 2001 From: rfqu Date: Tue, 2 May 2017 19:46:44 +0300 Subject: [PATCH 14/90] ignite-4220 Support statements for JDBC and Cassandra store --- .../store/cassandra/CassandraCacheStore.java | 16 ++++- .../session/LoadCacheCustomQueryWorker.java | 26 +++++-- .../tests/IgnitePersistentStoreTest.java | 23 ++++-- .../store/jdbc/CacheAbstractJdbcStore.java | 72 +++++++++++++------ .../CacheJdbcPojoStoreAbstractSelfTest.java | 49 +++++++++++++ 5 files changed, 152 insertions(+), 34 deletions(-) diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java index e8da3a7c8e511..2e1d3ea70318d 100644 --- a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java @@ -20,6 +20,7 @@ import com.datastax.driver.core.BoundStatement; import com.datastax.driver.core.PreparedStatement; import com.datastax.driver.core.Row; +import com.datastax.driver.core.Statement; import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; @@ -103,10 +104,19 @@ public CassandraCacheStore(DataSource dataSrc, KeyValuePersistenceSettings setti CassandraSession ses = getCassandraSession(); for (Object obj : args) { - if (obj == null || !(obj instanceof String) || !((String)obj).trim().toLowerCase().startsWith("select")) - continue; + LoadCacheCustomQueryWorker task = null; - futs.add(pool.submit(new LoadCacheCustomQueryWorker<>(ses, (String) obj, controller, log, clo))); + if (obj instanceof Statement) + task = new LoadCacheCustomQueryWorker<>(ses, (Statement)obj, controller, log, clo); + else if (obj instanceof String) { + String qry = ((String)obj).trim(); + + if (qry.toLowerCase().startsWith("select")) + task = new LoadCacheCustomQueryWorker<>(ses, (String) obj, controller, log, clo); + } + + if (task != null) + futs.add(pool.submit(task)); } for (Future fut : futs) diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/LoadCacheCustomQueryWorker.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/LoadCacheCustomQueryWorker.java index d3ace7d436e81..d186b98cf72fa 100644 --- a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/LoadCacheCustomQueryWorker.java +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/LoadCacheCustomQueryWorker.java @@ -36,8 +36,8 @@ public class LoadCacheCustomQueryWorker implements Callable { /** Cassandra session to execute CQL query */ private final CassandraSession ses; - /** User query. */ - private final String qry; + /** Statement. */ + private final Statement stmt; /** Persistence controller */ private final PersistenceController ctrl; @@ -49,12 +49,28 @@ public class LoadCacheCustomQueryWorker implements Callable { private final IgniteBiInClosure clo; /** + * @param ses Session. + * @param qry Query. + * @param ctrl Control. + * @param log Logger. * @param clo Closure for loaded values. */ public LoadCacheCustomQueryWorker(CassandraSession ses, String qry, PersistenceController ctrl, - IgniteLogger log, IgniteBiInClosure clo) { + IgniteLogger log, IgniteBiInClosure clo) { + this(ses, new SimpleStatement(qry.trim().endsWith(";") ? qry : qry + ';'), ctrl, log, clo); + } + + /** + * @param ses Session. + * @param stmt Statement. + * @param ctrl Control. + * @param log Logger. + * @param clo Closure for loaded values. + */ + public LoadCacheCustomQueryWorker(CassandraSession ses, Statement stmt, PersistenceController ctrl, + IgniteLogger log, IgniteBiInClosure clo) { this.ses = ses; - this.qry = qry.trim().endsWith(";") ? qry : qry + ";"; + this.stmt = stmt; this.ctrl = ctrl; this.log = log; this.clo = clo; @@ -70,7 +86,7 @@ public LoadCacheCustomQueryWorker(CassandraSession ses, String qry, PersistenceC /** {@inheritDoc} */ @Override public Statement getStatement() { - return new SimpleStatement(qry); + return stmt; } /** {@inheritDoc} */ diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreTest.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreTest.java index 5da6ba25dabea..51d08855388c6 100644 --- a/modules/cassandra/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreTest.java +++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreTest.java @@ -17,6 +17,7 @@ package org.apache.ignite.tests; +import com.datastax.driver.core.SimpleStatement; import java.util.Collection; import java.util.Map; import org.apache.ignite.Ignite; @@ -35,6 +36,7 @@ import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; +import org.junit.Assert; import org.springframework.core.io.ClassPathResource; /** @@ -346,20 +348,29 @@ public void loadCacheTest() { LOGGER.info("Running loadCache test"); try (Ignite ignite = Ignition.start("org/apache/ignite/tests/persistence/pojo/ignite-config.xml")) { - IgniteCache personCache3 = ignite.getOrCreateCache(new CacheConfiguration("cache3")); + CacheConfiguration ccfg = new CacheConfiguration<>("cache3"); + + IgniteCache personCache3 = ignite.getOrCreateCache(ccfg); + int size = personCache3.size(CachePeekMode.ALL); LOGGER.info("Initial cache size " + size); LOGGER.info("Loading cache data from Cassandra table"); - personCache3.loadCache(null, new String[] {"select * from test1.pojo_test3 limit 3"}); + String qry = "select * from test1.pojo_test3 limit 3"; + + personCache3.loadCache(null, qry); size = personCache3.size(CachePeekMode.ALL); - if (size != 3) { - throw new RuntimeException("Cache data was incorrectly loaded from Cassandra. " + - "Expected number of records is 3, but loaded number of records is " + size); - } + Assert.assertEquals("Cache data was incorrectly loaded from Cassandra table by '" + qry + "'", 3, size); + + personCache3.clear(); + + personCache3.loadCache(null, new SimpleStatement(qry)); + + size = personCache3.size(CachePeekMode.ALL); + Assert.assertEquals("Cache data was incorrectly loaded from Cassandra table by statement", 3, size); LOGGER.info("Cache data loaded from Cassandra table"); } diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java index e7ce52651a868..e211fadf2fb74 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java @@ -81,7 +81,6 @@ import static org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreFactory.DFLT_PARALLEL_LOAD_CACHE_MINIMUM_THRESHOLD; import static org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreFactory.DFLT_WRITE_ATTEMPTS; import static org.apache.ignite.cache.store.jdbc.JdbcTypesTransformer.NUMERIC_TYPES; -import static org.apache.ignite.cache.store.jdbc.JdbcTypesTransformer.NUMERIC_TYPES; /** * Implementation of {@link CacheStore} backed by JDBC. @@ -753,17 +752,34 @@ protected Integer columnIndex(Map loadColIdxs, String dbName) { } })) throw new CacheLoaderException("Provided key type is not found in store or cache configuration " + - "[cache=" + U.maskName(cacheName) + ", key=" + keyType + "]"); - - String qry = args[i + 1].toString(); + "[cache=" + U.maskName(cacheName) + ", key=" + keyType + ']'); EntryMapping em = entryMapping(cacheName, typeIdForTypeName(kindForName(keyType), keyType)); - if (log.isInfoEnabled()) - log.info("Started load cache using custom query [cache=" + U.maskName(cacheName) + - ", keyType=" + keyType + ", query=" + qry + "]"); + Object arg = args[i + 1]; + + LoadCacheCustomQueryWorker task; + + if (arg instanceof PreparedStatement) { + PreparedStatement stmt = (PreparedStatement)arg; + + if (log.isInfoEnabled()) + log.info("Started load cache using custom statement [cache=" + U.maskName(cacheName) + + ", keyType=" + keyType + ", stmt=" + stmt + ']'); + + task = new LoadCacheCustomQueryWorker<>(em, stmt, clo); + } + else { + String qry = arg.toString(); + + if (log.isInfoEnabled()) + log.info("Started load cache using custom query [cache=" + U.maskName(cacheName) + + ", keyType=" + keyType + ", query=" + qry + ']'); + + task = new LoadCacheCustomQueryWorker<>(em, qry, clo); + } - futs.add(pool.submit(new LoadCacheCustomQueryWorker<>(em, qry, clo))); + futs.add(pool.submit(task)); } } else { @@ -778,7 +794,7 @@ protected Integer columnIndex(Map loadColIdxs, String dbName) { processedKeyTypes.add(keyType); if (log.isInfoEnabled()) - log.info("Started load cache [cache=" + U.maskName(cacheName) + ", keyType=" + keyType + "]"); + log.info("Started load cache [cache=" + U.maskName(cacheName) + ", keyType=" + keyType + ']'); if (parallelLoadCacheMinThreshold > 0) { Connection conn = null; @@ -795,7 +811,7 @@ protected Integer columnIndex(Map loadColIdxs, String dbName) { if (rs.next()) { if (log.isDebugEnabled()) log.debug("Multithread loading entries from db [cache=" + U.maskName(cacheName) + - ", keyType=" + keyType + "]"); + ", keyType=" + keyType + ']'); int keyCnt = em.keyCols.size(); @@ -824,7 +840,7 @@ protected Integer columnIndex(Map loadColIdxs, String dbName) { } catch (SQLException e) { log.warning("Failed to load entries from db in multithreaded mode, will try in single thread " + - "[cache=" + U.maskName(cacheName) + ", keyType=" + keyType + " ]", e); + "[cache=" + U.maskName(cacheName) + ", keyType=" + keyType + ']', e); } finally { U.closeQuiet(conn); @@ -833,7 +849,7 @@ protected Integer columnIndex(Map loadColIdxs, String dbName) { if (log.isDebugEnabled()) log.debug("Single thread loading entries from db [cache=" + U.maskName(cacheName) + - ", keyType=" + keyType + "]"); + ", keyType=" + keyType + ']'); futs.add(pool.submit(loadCacheFull(em, clo))); } @@ -860,7 +876,7 @@ protected Integer columnIndex(Map loadColIdxs, String dbName) { EntryMapping em = entryMapping(session().cacheName(), typeIdForObject(key)); if (log.isDebugEnabled()) - log.debug("Load value from db [table= " + em.fullTableName() + ", key=" + key + "]"); + log.debug("Load value from db [table= " + em.fullTableName() + ", key=" + key + ']'); Connection conn = null; @@ -1954,12 +1970,26 @@ private class LoadCacheCustomQueryWorker implements Callable { /** Entry mapping description. */ private final EntryMapping em; + /** User statement. */ + private PreparedStatement stmt; + /** User query. */ - private final String qry; + private String qry; /** Closure for loaded values. */ private final IgniteBiInClosure clo; + /** + * @param em Entry mapping description. + * @param stmt User statement. + * @param clo Closure for loaded values. + */ + private LoadCacheCustomQueryWorker(EntryMapping em, PreparedStatement stmt, IgniteBiInClosure clo) { + this.em = em; + this.stmt = stmt; + this.clo = clo; + } + /** * @param em Entry mapping description. * @param qry User query. @@ -1975,12 +2005,12 @@ private LoadCacheCustomQueryWorker(EntryMapping em, String qry, IgniteBiInClosur @Override public Void call() throws Exception { Connection conn = null; - PreparedStatement stmt = null; - try { - conn = openConnection(true); + if (stmt == null) { + conn = openConnection(true); - stmt = conn.prepareStatement(qry); + stmt = conn.prepareStatement(qry); + } stmt.setFetchSize(dialect.getFetchSize()); @@ -2006,9 +2036,11 @@ private LoadCacheCustomQueryWorker(EntryMapping em, String qry, IgniteBiInClosur throw new CacheLoaderException("Failed to execute custom query for load cache", e); } finally { - U.closeQuiet(stmt); + if (conn != null) { + U.closeQuiet(stmt); - U.closeQuiet(conn); + U.closeQuiet(conn); + } } } } diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreAbstractSelfTest.java index 1de44f7243fbb..9e59769503f53 100644 --- a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreAbstractSelfTest.java @@ -328,6 +328,55 @@ protected void checkCacheLoadWithSql() { assertEquals(PERSON_CNT, c1.size()); } + /** + * Checks that data was loaded correctly with prepared statement. + */ + protected void checkCacheLoadWithStatement() throws SQLException { + Connection conn = null; + + PreparedStatement stmt = null; + + try { + conn = getConnection(); + + conn.setAutoCommit(true); + + String qry = "select id, org_id, name, birthday, gender from Person"; + + stmt = conn.prepareStatement(qry); + + IgniteCache c1 = grid().cache(CACHE_NAME); + + c1.loadCache(null, "org.apache.ignite.cache.store.jdbc.model.PersonKey", stmt); + + assertEquals(PERSON_CNT, c1.size()); + } + finally { + U.closeQuiet(stmt); + + U.closeQuiet(conn); + } + + } + + /** + * @throws Exception If failed. + */ + public void testLoadCacheWithStatement() throws Exception { + startTestGrid(false, false, false, false, 512); + + checkCacheLoadWithStatement(); + } + + /** + * @throws Exception If failed. + */ + public void testLoadCacheWithStatementTx() throws Exception { + startTestGrid(false, false, false, true, 512); + + checkCacheLoadWithStatement(); + } + /** * @throws Exception If failed. */ From 987c182686962673e70398395cb27e94f894713b Mon Sep 17 00:00:00 2001 From: nikolay_tikhonov Date: Mon, 15 May 2017 11:54:16 +0300 Subject: [PATCH 15/90] Fixed "IGNITE-5214 ConcurrentModificationException with enable DEBUG log level" Signed-off-by: nikolay_tikhonov --- .../continuous/CacheContinuousQueryHandler.java | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java index 1a6577d241774..5f00d580f4231 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java @@ -998,6 +998,7 @@ private static class PartitionRecovery { synchronized (pendingEvts) { if (log.isDebugEnabled()) { + log.debug("Handling event [lastFiredEvt=" + lastFiredEvt + ", curTop=" + curTop + ", entUpdCnt=" + entry.updateCounter() + @@ -1126,15 +1127,15 @@ private static class PartitionRecovery { break; } } - } - if (log.isDebugEnabled()) { - log.debug("Will send to listener the following events [entries=" + entries + - ", lastFiredEvt=" + lastFiredEvt + - ", curTop=" + curTop + - ", entUpdCnt=" + entry.updateCounter() + - ", partId=" + entry.partition() + - ", pendingEvts=" + pendingEvts + ']'); + if (log.isDebugEnabled()) { + log.debug("Will send to listener the following events [entries=" + entries + + ", lastFiredEvt=" + lastFiredEvt + + ", curTop=" + curTop + + ", entUpdCnt=" + entry.updateCounter() + + ", partId=" + entry.partition() + + ", pendingEvts=" + pendingEvts + ']'); + } } return entries; From ebc4a1648a80fbbd485e4c351fce9bee163318f9 Mon Sep 17 00:00:00 2001 From: sboikov Date: Tue, 16 May 2017 11:30:29 +0300 Subject: [PATCH 16/90] DirectByteBufferStreamImpl: converted asserts into exceptions. (cherry picked from commit 560ef60) --- .../v2/DirectByteBufferStreamImplV2.java | 27 ++++++++++++------- 1 file changed, 18 insertions(+), 9 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/direct/stream/v2/DirectByteBufferStreamImplV2.java b/modules/core/src/main/java/org/apache/ignite/internal/direct/stream/v2/DirectByteBufferStreamImplV2.java index d7dc990848094..af45263e36a8b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/direct/stream/v2/DirectByteBufferStreamImplV2.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/direct/stream/v2/DirectByteBufferStreamImplV2.java @@ -1,4 +1,4 @@ -/* + /* * 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. @@ -27,6 +27,7 @@ import java.util.Map; import java.util.RandomAccess; import java.util.UUID; +import org.apache.ignite.IgniteException; import org.apache.ignite.internal.direct.stream.DirectByteBufferStream; import org.apache.ignite.internal.util.GridUnsafe; import org.apache.ignite.internal.util.tostring.GridToStringExclude; @@ -80,7 +81,8 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream { /** */ private static final ArrayCreator BYTE_ARR_CREATOR = new ArrayCreator() { @Override public byte[] create(int len) { - assert len >= 0; + if (len < 0) + throw new IgniteException("Read invalid byte array length: " + len); switch (len) { case 0: @@ -95,7 +97,8 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream { /** */ private static final ArrayCreator SHORT_ARR_CREATOR = new ArrayCreator() { @Override public short[] create(int len) { - assert len >= 0; + if (len < 0) + throw new IgniteException("Read invalid short array length: " + len); switch (len) { case 0: @@ -110,7 +113,8 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream { /** */ private static final ArrayCreator INT_ARR_CREATOR = new ArrayCreator() { @Override public int[] create(int len) { - assert len >= 0; + if (len < 0) + throw new IgniteException("Read invalid int array length: " + len); switch (len) { case 0: @@ -125,7 +129,8 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream { /** */ private static final ArrayCreator LONG_ARR_CREATOR = new ArrayCreator() { @Override public long[] create(int len) { - assert len >= 0; + if (len < 0) + throw new IgniteException("Read invalid long array length: " + len); switch (len) { case 0: @@ -140,7 +145,8 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream { /** */ private static final ArrayCreator FLOAT_ARR_CREATOR = new ArrayCreator() { @Override public float[] create(int len) { - assert len >= 0; + if (len < 0) + throw new IgniteException("Read invalid float array length: " + len); switch (len) { case 0: @@ -155,7 +161,8 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream { /** */ private static final ArrayCreator DOUBLE_ARR_CREATOR = new ArrayCreator() { @Override public double[] create(int len) { - assert len >= 0; + if (len < 0) + throw new IgniteException("Read invalid double array length: " + len); switch (len) { case 0: @@ -170,7 +177,8 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream { /** */ private static final ArrayCreator CHAR_ARR_CREATOR = new ArrayCreator() { @Override public char[] create(int len) { - assert len >= 0; + if (len < 0) + throw new IgniteException("Read invalid char array length: " + len); switch (len) { case 0: @@ -185,7 +193,8 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream { /** */ private static final ArrayCreator BOOLEAN_ARR_CREATOR = new ArrayCreator() { @Override public boolean[] create(int len) { - assert len >= 0; + if (len < 0) + throw new IgniteException("Read invalid boolean array length: " + len); switch (len) { case 0: From 9cd7e0f8d132f9b7c496fe64f75f271ef60da5eb Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Thu, 9 Feb 2017 16:44:41 +0700 Subject: [PATCH 17/90] IGNITE-4676 Fixed hang if closure executed nested internal task with continuation. Added test. (cherry picked from commit e7a5307) --- .../processors/job/GridJobWorker.java | 4 + .../internal/GridContinuousTaskSelfTest.java | 79 +++++++++++++++++++ 2 files changed, 83 insertions(+) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java index 6a00d96a0abc2..acefde72eefab 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java @@ -617,6 +617,10 @@ else if (X.hasCause(e, GridServiceNotFoundException.class) || // Finish here only if not held by this thread. if (!HOLD.get()) finishJob(res, ex, sndRes); + else + // Make sure flag is not set for current thread. + // This may happen in case of nested internal task call with continuation. + HOLD.set(false); ctx.job().currentTaskSession(null); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridContinuousTaskSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridContinuousTaskSelfTest.java index 98e3c5afc8e2d..cec288714c100 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/GridContinuousTaskSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/GridContinuousTaskSelfTest.java @@ -21,10 +21,12 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Timer; import java.util.TimerTask; +import java.util.concurrent.Callable; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCompute; import org.apache.ignite.IgniteException; @@ -43,7 +45,9 @@ import org.apache.ignite.compute.ComputeTaskSessionAttributeListener; import org.apache.ignite.compute.ComputeTaskSessionFullSupport; import org.apache.ignite.compute.ComputeTaskSplitAdapter; +import org.apache.ignite.internal.processors.task.GridInternal; import org.apache.ignite.lang.IgniteClosure; +import org.apache.ignite.resources.IgniteInstanceResource; import org.apache.ignite.resources.JobContextResource; import org.apache.ignite.resources.LoggerResource; import org.apache.ignite.resources.TaskContinuousMapperResource; @@ -51,6 +55,7 @@ import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.apache.ignite.testframework.junits.common.GridCommonTest; +import org.jetbrains.annotations.Nullable; /** * Continuous task test. @@ -195,6 +200,80 @@ public void testMultipleHoldccCalls() throws Exception { } } + /** + * @throws Exception If test failed. + */ + public void testClosureWithNestedInternalTask() throws Exception { + try { + IgniteEx ignite = startGrid(0); + + ComputeTaskInternalFuture fut = ignite.context().closure().callAsync(GridClosureCallMode.BALANCE, new Callable() { + /** */ + @IgniteInstanceResource + private IgniteEx g; + + @Override public String call() throws Exception { + return g.compute(g.cluster()).execute(NestedHoldccTask.class, null); + } + }, ignite.cluster().nodes()); + + assertEquals("DONE", fut.get(3000)); + } + finally { + stopGrid(0, true); + } + } + + /** Test task with continuation. */ + @GridInternal + public static class NestedHoldccTask extends ComputeTaskAdapter { + /** {@inheritDoc} */ + @Nullable @Override public Map map(List subgrid, + @Nullable String arg) throws IgniteException { + Map map = new HashMap<>(); + + for (ClusterNode node : subgrid) + map.put(new NestedHoldccJob(), node); + + return map; + + } + + /** {@inheritDoc} */ + @Nullable @Override public String reduce(List results) throws IgniteException { + return results.get(0).getData(); + } + } + + /** Test job. */ + public static class NestedHoldccJob extends ComputeJobAdapter { + /** */ + @JobContextResource + private ComputeJobContext jobCtx; + + /** */ + private int cnt = 0; + + /** {@inheritDoc} */ + @Override public Object execute() throws IgniteException { + if (cnt < 1) { + cnt++; + + jobCtx.holdcc(); + + new Timer().schedule(new TimerTask() { + @Override public void run() { + jobCtx.callcc(); + } + }, 500); + + return "NOT DONE"; + } + + return "DONE"; + } + } + /** */ @SuppressWarnings({"PublicInnerClass"}) public static class TestMultipleHoldccCallsClosure implements IgniteClosure { From 43bcc15127bd3fd7ac4e277da6da9e5fb6a855c0 Mon Sep 17 00:00:00 2001 From: Vasiliy Sisko Date: Thu, 30 Mar 2017 11:08:10 +0700 Subject: [PATCH 18/90] IGNITE-4838 Fixed internal task detection logic. Added tests. (cherry picked from commit ba68c6c) --- .../processors/task/GridTaskProcessor.java | 9 ++++- .../internal/GridTaskExecutionSelfTest.java | 34 +++++++++++++++++++ ...cutionWithoutPeerClassLoadingSelfTest.java | 31 +++++++++++++++++ .../IgniteComputeGridTestSuite.java | 2 ++ 4 files changed, 75 insertions(+), 1 deletion(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/GridTaskExecutionWithoutPeerClassLoadingSelfTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java index d32b51c0867ca..935686456e593 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java @@ -608,6 +608,13 @@ else if (task != null) { if (subjId == null) subjId = ctx.localNodeId(); + boolean internal = false; + + if (dep == null || taskCls == null) + assert deployEx != null; + else + internal = dep.internalTask(task, taskCls); + // Creates task session with task name and task version. GridTaskSessionImpl ses = ctx.session().createTaskSession( sesId, @@ -621,7 +628,7 @@ else if (task != null) { Collections.emptyList(), Collections.emptyMap(), fullSup, - dep != null && dep.internalTask(task, taskCls), + internal, subjId); ComputeTaskInternalFuture fut = new ComputeTaskInternalFuture<>(ses, ctx); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridTaskExecutionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridTaskExecutionSelfTest.java index 996acd686f3cd..cc6a1eae6b2c6 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/GridTaskExecutionSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/GridTaskExecutionSelfTest.java @@ -22,8 +22,10 @@ import org.apache.ignite.GridTestTask; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCompute; +import org.apache.ignite.IgniteDeploymentException; import org.apache.ignite.compute.ComputeJobContext; import org.apache.ignite.compute.ComputeTaskFuture; +import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteCallable; import org.apache.ignite.lang.IgniteFuture; @@ -45,6 +47,20 @@ public GridTaskExecutionSelfTest() { super(false); } + /** */ + protected boolean peerClassLoadingEnabled() { + return true; + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + cfg.setPeerClassLoadingEnabled(peerClassLoadingEnabled()); + + return cfg; + } + /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { startGrid(1); @@ -144,4 +160,22 @@ public void testJobIdCollision() throws Exception { for (IgniteFuture fut : futs) fut.get(); } + + /** + * Test execution of non-existing task by name IGNITE-4838. + * + * @throws Exception If failed. + */ + public void testExecuteTaskWithInvalidName() throws Exception { + try { + ComputeTaskFuture fut = ignite.compute().execute("invalid.task.name", null); + + fut.get(); + + assert false : "Should never be reached due to exception thrown."; + } + catch (IgniteDeploymentException e) { + info("Received correct exception: " + e); + } + } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridTaskExecutionWithoutPeerClassLoadingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridTaskExecutionWithoutPeerClassLoadingSelfTest.java new file mode 100644 index 0000000000000..45e65cd9db017 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/GridTaskExecutionWithoutPeerClassLoadingSelfTest.java @@ -0,0 +1,31 @@ +/* + * 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 org.apache.ignite.testframework.junits.common.GridCommonTest; + +/** + * Task execution test. + */ +@GridCommonTest(group = "Kernal Self") +public class GridTaskExecutionWithoutPeerClassLoadingSelfTest extends GridTaskExecutionSelfTest { + /** {@inheritDoc} */ + @Override protected boolean peerClassLoadingEnabled() { + return false; + } +} 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 8a501fdb801d6..ae64c95fb30b6 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 @@ -51,6 +51,7 @@ import org.apache.ignite.internal.GridStopWithWaitSelfTest; import org.apache.ignite.internal.GridTaskCancelSingleNodeSelfTest; import org.apache.ignite.internal.GridTaskExecutionSelfTest; +import org.apache.ignite.internal.GridTaskExecutionWithoutPeerClassLoadingSelfTest; import org.apache.ignite.internal.GridTaskFailoverAffinityRunTest; import org.apache.ignite.internal.GridTaskFailoverSelfTest; import org.apache.ignite.internal.GridTaskFutureImplStopGridSelfTest; @@ -100,6 +101,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(GridCancelUnusedJobSelfTest.class); suite.addTestSuite(GridTaskJobRejectSelfTest.class); suite.addTestSuite(GridTaskExecutionSelfTest.class); + suite.addTestSuite(GridTaskExecutionWithoutPeerClassLoadingSelfTest.class); suite.addTestSuite(GridFailoverSelfTest.class); suite.addTestSuite(GridTaskListenerSelfTest.class); suite.addTestSuite(GridFailoverTopologySelfTest.class); From 2a818d36395dd1af23acf444adf396b2e2edbede Mon Sep 17 00:00:00 2001 From: Konstantin Dudkov Date: Mon, 22 May 2017 16:28:07 +0300 Subject: [PATCH 19/90] Fixed "IGNITE-4205 CassandraCacheStore should start IiteThread threads in loadCache() method" Signed-off-by: nikolay_tikhonov --- .../store/cassandra/CassandraCacheStore.java | 13 ++- .../tests/IgnitePersistentStoreTest.java | 62 +++++++++++++ .../loadall_blob/ignite-config.xml | 90 +++++++++++++++++++ .../loadall_blob/persistence-settings.xml | 29 ++++++ .../store/jdbc/CacheAbstractJdbcStore.java | 6 +- 5 files changed, 198 insertions(+), 2 deletions(-) create mode 100644 modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/loadall_blob/ignite-config.xml create mode 100644 modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/loadall_blob/persistence-settings.xml diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java index 2e1d3ea70318d..dabf1b09a588e 100644 --- a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java @@ -31,6 +31,7 @@ import javax.cache.Cache; import javax.cache.integration.CacheLoaderException; import javax.cache.integration.CacheWriterException; +import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.cache.store.CacheStore; @@ -47,7 +48,9 @@ import org.apache.ignite.lang.IgniteBiInClosure; import org.apache.ignite.logger.NullLogger; import org.apache.ignite.resources.CacheStoreSessionResource; +import org.apache.ignite.resources.IgniteInstanceResource; import org.apache.ignite.resources.LoggerResource; +import org.apache.ignite.thread.IgniteThreadFactory; /** * Implementation of {@link CacheStore} backed by Cassandra database. @@ -59,6 +62,14 @@ public class CassandraCacheStore implements CacheStore { /** Connection attribute property name. */ private static final String ATTR_CONN_PROP = "CASSANDRA_STORE_CONNECTION"; + /** Thread name. */ + private static final String CACHE_LOADER_THREAD_NAME = "cassandra-cache-loader"; + + /** Auto-injected ignite instance. */ + @SuppressWarnings("unused") + @IgniteInstanceResource + private Ignite ignite; + /** Auto-injected store session. */ @CacheStoreSessionResource private CacheStoreSession storeSes; @@ -99,7 +110,7 @@ public CassandraCacheStore(DataSource dataSrc, KeyValuePersistenceSettings setti Collection> futs = new ArrayList<>(args.length); try { - pool = Executors.newFixedThreadPool(maxPoolSize); + pool = Executors.newFixedThreadPool(maxPoolSize, new IgniteThreadFactory(ignite.name(), CACHE_LOADER_THREAD_NAME)); CassandraSession ses = getCassandraSession(); diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreTest.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreTest.java index 51d08855388c6..75dff66de8f84 100644 --- a/modules/cassandra/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreTest.java +++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreTest.java @@ -23,9 +23,11 @@ import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.Ignition; +import org.apache.ignite.binary.BinaryObject; import org.apache.ignite.cache.CachePeekMode; import org.apache.ignite.cache.store.CacheStore; import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.binary.BinaryMarshaller; import org.apache.ignite.internal.processors.cache.CacheEntryImpl; import org.apache.ignite.tests.pojos.Person; import org.apache.ignite.tests.pojos.PersonId; @@ -237,6 +239,35 @@ public void blobStrategyTest() { } } + + /** */ + @Test + public void blobBinaryLoadCacheTest() { + Ignition.stopAll(true); + + try (Ignite ignite = Ignition.start("org/apache/ignite/tests/persistence/loadall_blob/ignite-config.xml")) { + IgniteCache personCache = ignite.getOrCreateCache("cache2"); + + assert ignite.configuration().getMarshaller() instanceof BinaryMarshaller; + + personCache.put(1L, new PojoPerson(1, "name")); + + assert personCache.withKeepBinary().get(1L) instanceof BinaryObject; + } + + Ignition.stopAll(true); + + try (Ignite ignite = Ignition.start("org/apache/ignite/tests/persistence/loadall_blob/ignite-config.xml")) { + IgniteCache personCache = ignite.getOrCreateCache("cache2"); + + personCache.loadCache(null, null); + + PojoPerson person = personCache.get(1L); + + LOGGER.info("loadCache tests passed"); + } + } + /** */ @Test public void pojoStrategyTest() { @@ -377,4 +408,35 @@ public void loadCacheTest() { LOGGER.info("loadCache test passed"); } + + + /** */ + public static class PojoPerson { + /** */ + private int id; + + /** */ + private String name; + + /** */ + public PojoPerson() { + // No-op. + } + + /** */ + public PojoPerson(int id, String name) { + this.id = id; + this.name = name; + } + + /** */ + public int getId() { + return id; + } + + /** */ + public String getName() { + return name; + } + } } diff --git a/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/loadall_blob/ignite-config.xml b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/loadall_blob/ignite-config.xml new file mode 100644 index 0000000000000..115e263e017d8 --- /dev/null +++ b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/loadall_blob/ignite-config.xml @@ -0,0 +1,90 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 127.0.0.1:47500..47509 + + + + + + + + diff --git a/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/loadall_blob/persistence-settings.xml b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/loadall_blob/persistence-settings.xml new file mode 100644 index 0000000000000..e872201fcc180 --- /dev/null +++ b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/loadall_blob/persistence-settings.xml @@ -0,0 +1,29 @@ + + + + + + + + + diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java index e211fadf2fb74..817b1a5da8531 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java @@ -72,6 +72,7 @@ import org.apache.ignite.resources.CacheStoreSessionResource; import org.apache.ignite.resources.IgniteInstanceResource; import org.apache.ignite.resources.LoggerResource; +import org.apache.ignite.thread.IgniteThreadFactory; import org.apache.ignite.transactions.Transaction; import org.jetbrains.annotations.Nullable; @@ -121,6 +122,9 @@ * */ public abstract class CacheAbstractJdbcStore implements CacheStore, LifecycleAware { + /** Thread name. */ + private static final String CACHE_LOADER_THREAD_NAME = "jdbc-cache-loader"; + /** Connection attribute property name. */ protected static final String ATTR_CONN_PROP = "JDBC_STORE_CONNECTION"; @@ -730,7 +734,7 @@ protected Integer columnIndex(Map loadColIdxs, String dbName) { String cacheName = session().cacheName(); try { - pool = Executors.newFixedThreadPool(maxPoolSize); + pool = Executors.newFixedThreadPool(maxPoolSize, new IgniteThreadFactory(ignite.name(), CACHE_LOADER_THREAD_NAME)); Collection> futs = new ArrayList<>(); From 04fadd4a499239176ba21c390d93e30809abb4c1 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Tue, 23 May 2017 15:42:20 +0300 Subject: [PATCH 20/90] IGNITE-5223 Allow use local binary metadata cache if it's possible --- .../apache/ignite/IgniteSystemProperties.java | 8 + .../ignite/internal/binary/BinaryContext.java | 4 + .../internal/binary/BinaryReaderExImpl.java | 23 +- .../CacheObjectBinaryProcessorImpl.java | 98 ++++-- ...inaryMarshallerLocalMetadataCacheTest.java | 297 ++++++++++++++++++ .../IgniteBinaryObjectsTestSuite.java | 3 + 6 files changed, 406 insertions(+), 27 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerLocalMetadataCacheTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java index d77b2fb2838d4..713defeda552c 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -515,6 +515,14 @@ public final class IgniteSystemProperties { */ public static final String IGNITE_UNWRAP_BINARY_FOR_INDEXING_SPI = "IGNITE_UNWRAP_BINARY_FOR_INDEXING_SPI"; + /** + * Use local metadata cache instead of distributed one. May be used only when binary objects schema + * are not modified and all classes available on each node. Classes that implements Binarylizable are + * not supported. + * @deprecated Should be removed in Apache Ignite 2.0. + */ + public static final String IGNITE_USE_LOCAL_BINARY_MARSHALLER_CACHE = "IGNITE_USE_LOCAL_BINARY_MARSHALLER_CACHE"; + /** * Enforces singleton. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java index 7b21dfbed51f0..9f66b3ed50046 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java @@ -612,6 +612,7 @@ else if (cpElement.isFile()) { /** * @param cls Class. + * @param deserialize If {@code false}, metadata will be updated. * @return Class descriptor. * @throws BinaryObjectException In case of error. */ @@ -664,6 +665,7 @@ else if (!desc.registered()) { * @param userType User type or not. * @param typeId Type ID. * @param ldr Class loader. + * @param deserialize If {@code false}, metadata will be updated. * @return Class descriptor. */ public BinaryClassDescriptor descriptorForTypeId( @@ -719,6 +721,7 @@ public BinaryClassDescriptor descriptorForTypeId( * Creates and registers {@link BinaryClassDescriptor} for the given {@code class}. * * @param cls Class. + * @param deserialize If {@code false}, metadata will be updated. * @return Class descriptor. */ private BinaryClassDescriptor registerClassDescriptor(Class cls, boolean deserialize) { @@ -759,6 +762,7 @@ private BinaryClassDescriptor registerClassDescriptor(Class cls, boolean dese * Creates and registers {@link BinaryClassDescriptor} for the given user {@code class}. * * @param cls Class. + * @param deserialize If {@code false}, metadata will be updated. * @return Class descriptor. */ private BinaryClassDescriptor registerUserClassDescriptor(Class cls, boolean deserialize) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryReaderExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryReaderExImpl.java index 775f237787cfc..ad2e736fa5f34 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryReaderExImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryReaderExImpl.java @@ -26,6 +26,7 @@ import java.util.Date; import java.util.Map; import java.util.UUID; +import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.binary.BinaryCollectionFactory; import org.apache.ignite.binary.BinaryInvalidTypeException; import org.apache.ignite.binary.BinaryMapFactory; @@ -1722,15 +1723,23 @@ public BinarySchema getOrCreateSchema() { if (fieldIdLen != BinaryUtils.FIELD_ID_LEN) { BinaryTypeImpl type = (BinaryTypeImpl)ctx.metadata(typeId); - if (type == null || type.metadata() == null) - throw new BinaryObjectException("Cannot find metadata for object with compact footer: " + - typeId); + if (type == null || type.metadata() == null || type.metadata().schemas().isEmpty()) { + if (IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_USE_LOCAL_BINARY_MARSHALLER_CACHE, false)) { + BinaryClassDescriptor desc = ctx.descriptorForTypeId(true, typeId, getClass().getClassLoader(), false); - for (BinarySchema typeSchema : type.metadata().schemas()) { - if (schemaId == typeSchema.schemaId()) { - schema = typeSchema; + schema = desc.schema(); + } + else + throw new BinaryObjectException("Cannot find metadata for object with compact footer: " + + typeId); + } + else { + for (BinarySchema typeSchema : type.metadata().schemas()) { + if (schemaId == typeSchema.schemaId()) { + schema = typeSchema; - break; + break; + } } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java index 1d60c422a6417..6b691c2236e52 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java @@ -39,6 +39,7 @@ import org.apache.ignite.IgniteBinary; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.binary.BinaryBasicNameMapper; import org.apache.ignite.binary.BinaryField; import org.apache.ignite.binary.BinaryObject; @@ -126,6 +127,9 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm /** */ private volatile IgniteCacheProxy metaDataCache; + /** */ + private final ConcurrentHashMap8 locMetadataCache; + /** */ private final ConcurrentHashMap8 clientMetaDataCache; @@ -175,6 +179,10 @@ public CacheObjectBinaryProcessorImpl(GridKernalContext ctx) { clientNode = this.ctx.clientNode(); clientMetaDataCache = clientNode ? new ConcurrentHashMap8() : null; + + boolean useLocCache = IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_USE_LOCAL_BINARY_MARSHALLER_CACHE, false); + + locMetadataCache = useLocCache ? new ConcurrentHashMap8() : null; } /** {@inheritDoc} */ @@ -190,7 +198,7 @@ public CacheObjectBinaryProcessorImpl(GridKernalContext ctx) { BinaryMetadata newMeta0 = ((BinaryTypeImpl)newMeta).metadata(); - if (metaDataCache == null) { + if (metaDataCache == null && locMetadataCache == null) { BinaryMetadata oldMeta = metaBuf.get(typeId); BinaryMetadata mergedMeta = BinaryUtils.mergeMetadata(oldMeta, newMeta0); @@ -213,13 +221,13 @@ public CacheObjectBinaryProcessorImpl(GridKernalContext ctx) { return; } - assert metaDataCache != null; + assert metaDataCache != null || locMetadataCache != null; CacheObjectBinaryProcessorImpl.this.addMeta(typeId, newMeta0.wrap(binaryCtx)); } @Override public BinaryType metadata(int typeId) throws BinaryObjectException { - if (metaDataCache == null) + if (metaDataCache == null && locMetadataCache == null) U.awaitQuiet(startLatch); return CacheObjectBinaryProcessorImpl.this.metadata(typeId); @@ -289,6 +297,12 @@ public CacheObjectBinaryProcessorImpl(GridKernalContext ctx) { /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public void onUtilityCacheStarted() throws IgniteCheckedException { + if (locMetadataCache != null) { + startLatch.countDown(); + + return; + } + IgniteCacheProxy proxy = ctx.cache().jcache(CU.UTILITY_CACHE_NAME); boolean old = proxy.context().deploy().ignoreOwnership(true); @@ -573,22 +587,31 @@ public GridBinaryMarshaller marshaller() { final BinaryMetadataKey key = new BinaryMetadataKey(typeId); - try { - BinaryMetadata oldMeta = metaDataCache.localPeek(key); - BinaryMetadata mergedMeta = BinaryUtils.mergeMetadata(oldMeta, newMeta0); + if (locMetadataCache != null) { + locMetadataCache.merge(key, newMeta0, new ConcurrentHashMap8.BiFun() { + @Override public BinaryMetadata apply(BinaryMetadata curMeta, BinaryMetadata newMeta) { + return BinaryUtils.mergeMetadata(curMeta, newMeta); + } + }); + } + else { + try { + BinaryMetadata oldMeta = metaDataCache.localPeek(key); + BinaryMetadata mergedMeta = BinaryUtils.mergeMetadata(oldMeta, newMeta0); - AffinityTopologyVersion topVer = ctx.cache().context().lockedTopologyVersion(null); + AffinityTopologyVersion topVer = ctx.cache().context().lockedTopologyVersion(null); - if (topVer == null) - topVer = ctx.cache().context().exchange().readyAffinityVersion(); + if (topVer == null) + topVer = ctx.cache().context().exchange().readyAffinityVersion(); - BinaryObjectException err = metaDataCache.invoke(topVer, key, new MetadataProcessor(mergedMeta)); + BinaryObjectException err = metaDataCache.invoke(topVer, key, new MetadataProcessor(mergedMeta)); - if (err != null) - throw err; - } - catch (CacheException e) { - throw new BinaryObjectException("Failed to update meta data for type: " + newMeta.typeName(), e); + if (err != null) + throw err; + } + catch (CacheException e) { + throw new BinaryObjectException("Failed to update meta data for type: " + newMeta.typeName(), e); + } } } @@ -601,17 +624,28 @@ public GridBinaryMarshaller marshaller() { if (typeMeta != null) return typeMeta; - BinaryMetadata meta = metaDataCache.getTopologySafe(new BinaryMetadataKey(typeId)); + BinaryMetadata meta; + + if (locMetadataCache != null) + meta = locMetadataCache.get(new BinaryMetadataKey(typeId)); + else + meta = metaDataCache.getTopologySafe(new BinaryMetadataKey(typeId)); return meta != null ? meta.wrap(binaryCtx) : null; } else { BinaryMetadataKey key = new BinaryMetadataKey(typeId); - BinaryMetadata meta = metaDataCache.localPeek(key); + BinaryMetadata meta; + + if (locMetadataCache != null) + meta = locMetadataCache.get(key); + else { + meta = metaDataCache.localPeek(key); - if (meta == null && !metaDataCache.context().preloader().syncFuture().isDone()) - meta = metaDataCache.getTopologySafe(key); + if (meta == null && !metaDataCache.context().preloader().syncFuture().isDone()) + meta = metaDataCache.getTopologySafe(key); + } return meta != null ? meta.wrap(binaryCtx) : null; } @@ -630,7 +664,20 @@ public GridBinaryMarshaller marshaller() { for (Integer typeId : typeIds) keys.add(new BinaryMetadataKey(typeId)); - Map meta = metaDataCache.getAll(keys); + Map meta; + + if (locMetadataCache != null) { + meta = new HashMap<>(); + + for (BinaryMetadataKey key : keys) { + BinaryMetadata metadata = locMetadataCache.get(key); + + if (metadata != null) + meta.put(key, metadata); + } + } + else + meta = metaDataCache.getAll(keys); Map res = U.newHashMap(meta.size()); @@ -654,6 +701,17 @@ public GridBinaryMarshaller marshaller() { } }); else { + if (locMetadataCache != null) { + ConcurrentHashMap8.ValuesView vals = locMetadataCache.values(); + + ArrayList res = new ArrayList<>(vals.size()); + + for (BinaryMetadata metadata : vals) + res.add(metadata.wrap(binaryCtx)); + + return res; + } + return F.viewReadOnly(metaDataCache.entrySetx(metaPred), new C1, BinaryType>() { private static final long serialVersionUID = 0L; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerLocalMetadataCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerLocalMetadataCacheTest.java new file mode 100644 index 0000000000000..ad3c5f3e0a83b --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerLocalMetadataCacheTest.java @@ -0,0 +1,297 @@ +/* + * 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.binary; + +import java.io.Externalizable; +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import java.math.BigInteger; +import java.util.HashMap; +import java.util.Map; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteSystemProperties; +import org.apache.ignite.binary.BinaryObject; +import org.apache.ignite.cluster.ClusterGroup; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteCallable; +import org.apache.ignite.testframework.junits.IgniteTestResources; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Tests IGNITE_USE_LOCAL_BINARY_MARSHALLER_CACHE property. + */ +public class BinaryMarshallerLocalMetadataCacheTest extends GridCommonAbstractTest { + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName, + IgniteTestResources rsrcs) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName, rsrcs); + + cfg.setMarshaller(new BinaryMarshaller()); + + cfg.setClientMode(gridName.startsWith("client")); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + System.setProperty(IgniteSystemProperties.IGNITE_USE_LOCAL_BINARY_MARSHALLER_CACHE, "true"); + + startGrid(0); + startGrid(1); + startGrid("client"); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + try { + stopAllGrids(); + } + finally { + System.clearProperty(IgniteSystemProperties.IGNITE_USE_LOCAL_BINARY_MARSHALLER_CACHE); + } + } + + /** + * @throws Exception If failed. + */ + public void testComputeLocalMetadata() throws Exception { + final BinaryObject obj = grid(0).binary().toBinary(new OptimizedContainer(new Optimized())); + + ClusterGroup remotes = grid(0).cluster().forRemotes(); + + OptimizedContainer res = grid(0).compute(remotes).call(new IgniteCallable() { + @Override public OptimizedContainer call() throws Exception { + + return obj.deserialize(); + } + }); + + OptimizedContainer res2 = grid(0).compute(remotes).call(new IgniteCallable() { + @Override public OptimizedContainer call() throws Exception { + + return obj.deserialize(); + } + }); + + System.out.println(res); + System.out.println(res2); + } + + /** + * @throws Exception If failed. + */ + public void testCacheLocalMetadata() throws Exception { + IgniteCache cache = grid("client").createCache("cache"); + + Map data = new HashMap<>(); + + for (int i = 0; i < 1000; i++) + data.put(new Key(i), new OptimizedContainer(new Optimized(String.valueOf(i)))); + + for (int i = 1000; i < 2000; i++) + data.put(new Key(i), new Simple(i, String.valueOf(i), new BigInteger(String.valueOf(i), 10))); + + cache.putAll(data); + + checkCache(cache, data); + checkCache(grid(0).cache("cache"), data); + checkCache(grid(1).cache("cache"), data); + } + + /** + * @param cache Cache. + * @param data Data. + */ + private void checkCache(IgniteCache cache, + Map data) { + for (Map.Entry entry : cache.getAll(data.keySet()).entrySet()) + assertEquals(data.get(entry.getKey()), entry.getValue()); + } + + /** + * + */ + private static class Key { + /** */ + private Integer i; + + /** + * @param i I. + */ + public Key(Integer i) { + this.i = i; + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + Key key = (Key)o; + + return i != null ? i.equals(key.i) : key.i == null; + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return i != null ? i.hashCode() : 0; + } + } + + /** + * + */ + private static class OptimizedContainer { + /** */ + private Optimized optim; + + /** + * @param optim Val. + */ + public OptimizedContainer(Optimized optim) { + this.optim = optim; + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + OptimizedContainer container = (OptimizedContainer)o; + + return optim != null ? optim.equals(container.optim) : container.optim == null; + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return optim != null ? optim.hashCode() : 0; + } + } + + /** + * + */ + private static class Optimized implements Externalizable { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + + /** */ + private String fld; + + /** + * @param fld Fld. + */ + public Optimized(String fld) { + this.fld = fld; + } + + /** + * Default constructor (required by Externalizable). + */ + public Optimized() { + } + + /** {@inheritDoc} */ + @Override public void writeExternal(ObjectOutput out) throws IOException { + U.writeUTFStringNullable(out, fld); + } + + /** {@inheritDoc} */ + @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { + fld = U.readUTFStringNullable(in); + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + Optimized optimized = (Optimized)o; + + return fld != null ? fld.equals(optimized.fld) : optimized.fld == null; + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return fld != null ? fld.hashCode() : 0; + } + } + + /** + * + */ + private static class Simple { + /** I. */ + private int i; + + /** String. */ + private String str; + + /** Big integer. */ + private BigInteger bigInteger; + + /** + * @param i I. + * @param str String. + * @param bigInteger Big integer. + */ + public Simple(int i, String str, BigInteger bigInteger) { + this.i = i; + this.str = str; + this.bigInteger = bigInteger; + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + + if (o == null || getClass() != o.getClass()) + return false; + + Simple simple = (Simple)o; + + if (i != simple.i) + return false; + + if (str != null ? !str.equals(simple.str) : simple.str != null) + return false; + + return bigInteger != null ? bigInteger.equals(simple.bigInteger) : simple.bigInteger == null; + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + int res = i; + + res = 31 * res + (str != null ? str.hashCode() : 0); + res = 31 * res + (bigInteger != null ? bigInteger.hashCode() : 0); + + return res; + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsTestSuite.java index c1d9974837fb3..29e43e4593415 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsTestSuite.java @@ -26,6 +26,7 @@ import org.apache.ignite.internal.binary.BinaryFieldsOffheapSelfTest; import org.apache.ignite.internal.binary.BinaryFooterOffsetsHeapSelfTest; import org.apache.ignite.internal.binary.BinaryFooterOffsetsOffheapSelfTest; +import org.apache.ignite.internal.binary.BinaryMarshallerLocalMetadataCacheTest; import org.apache.ignite.internal.binary.BinaryMarshallerSelfTest; import org.apache.ignite.internal.binary.BinaryObjectBuilderAdditionalSelfTest; import org.apache.ignite.internal.binary.BinaryObjectBuilderDefaultMappersSelfTest; @@ -147,6 +148,8 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(GridCacheBinaryObjectUserClassloaderSelfTest.class); + suite.addTestSuite(BinaryMarshallerLocalMetadataCacheTest.class); + return suite; } } From b2040b7a95e421609bcf7ae05b56dc623310b409 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Tue, 23 May 2017 16:14:08 +0300 Subject: [PATCH 21/90] IGNITE-5259 Minor serialization fix --- modules/core/pom.xml | 3 + .../apache/ignite/IgniteSystemProperties.java | 12 ++- .../ignite/internal/IgniteNodeAttributes.java | 6 ++ .../discovery/GridDiscoveryManager.java | 39 ++++++++ .../CacheContinuousQueryHandler.java | 1 - .../top/GridTopologyCommandHandler.java | 2 + .../processors/security/SecurityUtils.java | 92 ++++++++++++++++++ .../security/SecurityBasicPermissionSet.java | 41 +++++++- .../ignite/spi/discovery/tcp/ServerImpl.java | 97 +++++++++++++++++-- ...ridDiscoveryManagerAttributesSelfTest.java | 70 ++++++++++++- .../discovery/tcp/TestReconnectProcessor.java | 47 ++++++++- 11 files changed, 390 insertions(+), 20 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityUtils.java diff --git a/modules/core/pom.xml b/modules/core/pom.xml index 4c4343a9c41a1..4a14407ab131f 100644 --- a/modules/core/pom.xml +++ b/modules/core/pom.xml @@ -234,6 +234,9 @@ **/*.java + + src/test/resources + diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java index 713defeda552c..6827e0c248290 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -503,7 +503,7 @@ public final class IgniteSystemProperties { /** * Whether Ignite can access unaligned memory addresses. *

- * Defaults to {@code} false, meaning that unaligned access will be performed only on x86 architecture. + * Defaults to {@code false}, meaning that unaligned access will be performed only on x86 architecture. */ public static final String IGNITE_UNALIGNED_MEMORY_ACCESS = "IGNITE_UNALIGNED_MEMORY_ACCESS"; @@ -523,6 +523,16 @@ public final class IgniteSystemProperties { */ public static final String IGNITE_USE_LOCAL_BINARY_MARSHALLER_CACHE = "IGNITE_USE_LOCAL_BINARY_MARSHALLER_CACHE"; + /** + * When set to {@code true}, Ignite switches to compatibility mode with versions that don't + * support service security permissions. In this case security permissions will be ignored + * (if they set). + *

+ * Default is {@code false}, which means that service security permissions will be respected. + *

+ */ + public static final String IGNITE_SECURITY_COMPATIBILITY_MODE = "IGNITE_SECURITY_COMPATIBILITY_MODE"; + /** * Enforces singleton. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java index e7c984f22864e..436792459af4a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java @@ -136,6 +136,9 @@ public final class IgniteNodeAttributes { /** Security subject for authenticated node. */ public static final String ATTR_SECURITY_SUBJECT = ATTR_PREFIX + ".security.subject"; + /** V2 security subject for authenticated node. */ + public static final String ATTR_SECURITY_SUBJECT_V2 = ATTR_PREFIX + ".security.subject.v2"; + /** Client mode flag. */ public static final String ATTR_CLIENT_MODE = ATTR_PREFIX + ".cache.client"; @@ -160,6 +163,9 @@ public final class IgniteNodeAttributes { /** Ignite services compatibility mode (can be {@code null}). */ public static final String ATTR_SERVICES_COMPATIBILITY_MODE = ATTR_PREFIX + ".services.compatibility.enabled"; + /** Ignite security compatibility mode. */ + public static final String ATTR_SECURITY_COMPATIBILITY_MODE = ATTR_PREFIX + ".security.compatibility.enabled"; + /** * Enforces singleton. */ 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 2ec10705bedbb..b3ba83da68a8d 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 @@ -121,6 +121,7 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_BINARY_MARSHALLER_USE_STRING_SERIALIZATION_VER_2; import static org.apache.ignite.IgniteSystemProperties.IGNITE_DISCOVERY_HISTORY_SIZE; import static org.apache.ignite.IgniteSystemProperties.IGNITE_OPTIMIZED_MARSHALLER_USE_DEFAULT_SUID; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_SECURITY_COMPATIBILITY_MODE; import static org.apache.ignite.IgniteSystemProperties.IGNITE_SERVICES_COMPATIBILITY_MODE; import static org.apache.ignite.IgniteSystemProperties.getInteger; import static org.apache.ignite.events.EventType.EVT_CLIENT_NODE_DISCONNECTED; @@ -136,9 +137,12 @@ import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER_USE_BINARY_STRING_SER_VER_2; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER_USE_DFLT_SUID; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_PEER_CLASSLOADING; +import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_SECURITY_COMPATIBILITY_MODE; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_SERVICES_COMPATIBILITY_MODE; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_USER_NAME; import static org.apache.ignite.internal.IgniteVersionUtils.VER; +import static org.apache.ignite.internal.processors.security.SecurityUtils.SERVICE_PERMISSIONS_SINCE; +import static org.apache.ignite.internal.processors.security.SecurityUtils.isSecurityCompatibilityMode; import static org.apache.ignite.plugin.segmentation.SegmentationPolicy.NOOP; /** @@ -442,6 +446,9 @@ private void updateClientNodes(UUID leftNodeId) { spi.setMetricsProvider(createMetricsProvider()); if (ctx.security().enabled()) { + if (isSecurityCompatibilityMode()) + ctx.addNodeAttribute(ATTR_SECURITY_COMPATIBILITY_MODE, true); + spi.setAuthenticator(new DiscoverySpiNodeAuthenticator() { @Override public SecurityContext authenticateNode(ClusterNode node, SecurityCredentials cred) { try { @@ -1044,6 +1051,7 @@ private void checkAttributes(Iterable nodes) throws IgniteCheckedEx boolean locDelayAssign = locNode.attribute(ATTR_LATE_AFFINITY_ASSIGNMENT); Boolean locSrvcCompatibilityEnabled = locNode.attribute(ATTR_SERVICES_COMPATIBILITY_MODE); + Boolean locSecurityCompatibilityEnabled = locNode.attribute(ATTR_SECURITY_COMPATIBILITY_MODE); for (ClusterNode n : nodes) { int rmtJvmMajVer = nodeJavaMajorVersion(n); @@ -1157,6 +1165,37 @@ else if (Boolean.FALSE.equals(locSrvcCompatibilityEnabled)) { ", rmtNodeAddrs=" + U.addressesAsString(n) + ", locNodeId=" + locNode.id() + ", rmtNodeId=" + n.id() + ']'); } + + if (n.version().compareToIgnoreTimestamp(SERVICE_PERMISSIONS_SINCE) >= 0 + && ctx.security().enabled() // Matters only if security enabled. + ) { + Boolean rmtSecurityCompatibilityEnabled = n.attribute(ATTR_SECURITY_COMPATIBILITY_MODE); + + if (!F.eq(locSecurityCompatibilityEnabled, rmtSecurityCompatibilityEnabled)) { + throw new IgniteCheckedException("Local node's " + IGNITE_SECURITY_COMPATIBILITY_MODE + + " property value differs from remote node's value " + + "(to make sure all nodes in topology have identical Ignite security compatibility mode enabled, " + + "configure system property explicitly) " + + "[locSecurityCompatibilityEnabled=" + locSecurityCompatibilityEnabled + + ", rmtSecurityCompatibilityEnabled=" + rmtSecurityCompatibilityEnabled + + ", locNodeAddrs=" + U.addressesAsString(locNode) + + ", rmtNodeAddrs=" + U.addressesAsString(n) + + ", locNodeId=" + locNode.id() + ", rmtNodeId=" + n.id() + ']'); + } + } + + if (n.version().compareToIgnoreTimestamp(SERVICE_PERMISSIONS_SINCE) < 0 + && ctx.security().enabled() // Matters only if security enabled. + && (locSecurityCompatibilityEnabled == null || !locSecurityCompatibilityEnabled)) { + throw new IgniteCheckedException("Remote node does not support service security permissions. " + + "To be able to join to it, local node must be started with " + IGNITE_SECURITY_COMPATIBILITY_MODE + + " system property set to \"true\". " + + "[locSecurityCompatibilityEnabled=" + locSecurityCompatibilityEnabled + + ", locNodeAddrs=" + U.addressesAsString(locNode) + + ", rmtNodeAddrs=" + U.addressesAsString(n) + + ", locNodeId=" + locNode.id() + ", rmtNodeId=" + n.id() + ", " + + ", rmtNodeVer" + n.version() + ']'); + } } if (log.isDebugEnabled()) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java index 5f00d580f4231..17f4308c0709b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java @@ -998,7 +998,6 @@ private static class PartitionRecovery { synchronized (pendingEvts) { if (log.isDebugEnabled()) { - log.debug("Handling event [lastFiredEvt=" + lastFiredEvt + ", curTop=" + curTop + ", entUpdCnt=" + entry.updateCounter() + diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java index 297785ea760f2..3c68fbfdba92e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java @@ -58,6 +58,7 @@ import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_REST_TCP_PORT; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_SECURITY_SUBJECT; +import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_SECURITY_SUBJECT_V2; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_TX_CONFIG; import static org.apache.ignite.internal.processors.rest.GridRestCommand.NODE; import static org.apache.ignite.internal.processors.rest.GridRestCommand.TOPOLOGY; @@ -290,6 +291,7 @@ private GridClientNodeBean createNodeBean(ClusterNode node, boolean mtr, boolean attrs.remove(ATTR_CACHE); attrs.remove(ATTR_TX_CONFIG); attrs.remove(ATTR_SECURITY_SUBJECT); + attrs.remove(ATTR_SECURITY_SUBJECT_V2); attrs.remove(ATTR_SECURITY_CREDENTIALS); for (Iterator> i = attrs.entrySet().iterator(); i.hasNext();) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityUtils.java new file mode 100644 index 0000000000000..1016335888b3d --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityUtils.java @@ -0,0 +1,92 @@ +/* + * 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.security; + +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; +import org.apache.ignite.IgniteSystemProperties; +import org.apache.ignite.lang.IgniteProductVersion; +import org.apache.ignite.plugin.security.SecurityPermission; + +/** + * Security utilities. + */ +public class SecurityUtils { + /** Version since service security supported. */ + public static final IgniteProductVersion SERVICE_PERMISSIONS_SINCE = IgniteProductVersion.fromString("1.7.11"); + + /** Default serialization version. */ + private final static int DFLT_SERIALIZE_VERSION = isSecurityCompatibilityMode() ? 1 : 2; + + /** Current serialization version. */ + private static final ThreadLocal SERIALIZE_VERSION = new ThreadLocal(){ + @Override protected Integer initialValue() { + return DFLT_SERIALIZE_VERSION; + } + }; + + /** + * Private constructor. + */ + private SecurityUtils() { + } + + /** + * @return Security compatibility mode flag. + */ + public static boolean isSecurityCompatibilityMode() { + return IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_SECURITY_COMPATIBILITY_MODE, false); + } + + /** + * @param ver Serialize version. + */ + public static void serializeVersion(int ver) { + SERIALIZE_VERSION.set(ver); + } + + /** + * @return Serialize version. + */ + public static int serializeVersion() { + return SERIALIZE_VERSION.get(); + } + + /** + * Sets default serialize version {@link #DFLT_SERIALIZE_VERSION}. + */ + public static void restoreDefaultSerializeVersion() { + serializeVersion(DFLT_SERIALIZE_VERSION); + } + + /** + * @return Allow all service permissions. + */ + public static Map> compatibleServicePermissions() { + Map> srvcPerms = new HashMap<>(); + + srvcPerms.put("*", Arrays.asList( + SecurityPermission.SERVICE_CANCEL, + SecurityPermission.SERVICE_DEPLOY, + SecurityPermission.SERVICE_INVOKE)); + + return srvcPerms; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityBasicPermissionSet.java b/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityBasicPermissionSet.java index 44166d936c9aa..370eadd801f83 100644 --- a/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityBasicPermissionSet.java +++ b/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityBasicPermissionSet.java @@ -17,15 +17,24 @@ package org.apache.ignite.plugin.security; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.Map; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.F; 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.jetbrains.annotations.Nullable; +import static org.apache.ignite.internal.processors.security.SecurityUtils.compatibleServicePermissions; +import static org.apache.ignite.internal.processors.security.SecurityUtils.isSecurityCompatibilityMode; +import static org.apache.ignite.internal.processors.security.SecurityUtils.serializeVersion; + /** * Simple implementation of {@link SecurityPermissionSet} interface. Provides * convenient way to specify permission set in the XML configuration. @@ -44,7 +53,9 @@ public class SecurityBasicPermissionSet implements SecurityPermissionSet { /** Service permissions. */ @GridToStringInclude - private Map> servicePermissions = new HashMap<>(); + private transient Map> servicePermissions = isSecurityCompatibilityMode() + ? compatibleServicePermissions() + : new HashMap>(); /** System permissions. */ @GridToStringInclude @@ -158,6 +169,34 @@ public void setDefaultAllowAll(boolean dfltAllowAll) { return res; } + /** + * @param out Out. + */ + private void writeObject(ObjectOutputStream out) throws IOException { + out.defaultWriteObject(); + + if (serializeVersion() >= 2) + U.writeMap(out, servicePermissions); + } + + /** + * @param in In. + */ + private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { + in.defaultReadObject(); + + if (serializeVersion() >= 2) + servicePermissions = U.readMap(in); + + if (servicePermissions == null) { + // Allow all for compatibility mode + if (serializeVersion() < 2) + servicePermissions = compatibleServicePermissions(); + else + servicePermissions = Collections.emptyMap(); + } + } + /** {@inheritDoc} */ @Override public String toString() { return S.toString(SecurityBasicPermissionSet.class, this); diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java index afd1c2ba7fc66..58b362ffd0f15 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java @@ -72,6 +72,7 @@ import org.apache.ignite.internal.events.DiscoveryCustomEvent; import org.apache.ignite.internal.processors.cache.CacheAffinitySharedManager; import org.apache.ignite.internal.processors.security.SecurityContext; +import org.apache.ignite.internal.processors.security.SecurityUtils; import org.apache.ignite.internal.processors.service.GridServiceProcessor; import org.apache.ignite.internal.util.GridBoundedLinkedHashSet; import org.apache.ignite.internal.util.GridConcurrentHashSet; @@ -947,7 +948,8 @@ private void localAuthentication(SecurityCredentials locCred){ Map attrs = new HashMap<>(locNode.attributes()); - attrs.put(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT, U.marshal(spi.marshaller(), subj)); + attrs.put(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT_V2, U.marshal(spi.marshaller(), subj)); + attrs.put(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT, marshalWithSecurityVersion(subj, 1)); locNode.setAttributes(attrs); @@ -983,7 +985,16 @@ private boolean sendJoinRequestMessage() throws IgniteSpiException { for (InetSocketAddress addr : addrs) { try { - Integer res = sendMessageDirectly(joinReq, addr); + Integer res; + + try { + SecurityUtils.serializeVersion(1); + + res = sendMessageDirectly(joinReq, addr); + } + finally { + SecurityUtils.restoreDefaultSerializeVersion(); + } assert res != null; @@ -1921,6 +1932,39 @@ private void processMessageFailedNodes(TcpDiscoveryAbstractMessage msg) { } } + /** + * @param obj Object. + * @param ver Security serialize version. + * @return Marshaled object. + */ + private byte[] marshalWithSecurityVersion(Object obj, int ver) throws IgniteCheckedException { + try { + SecurityUtils.serializeVersion(ver); + + return U.marshal(spi.marshaller(), obj); + } + finally { + SecurityUtils.restoreDefaultSerializeVersion(); + } + } + + /** + * @param bytes Marshaled object. + * @param ver Security serialize version. + * @return Unmarshaled object. + */ + private T unmarshalWithSecurityVersion(byte[] bytes, int ver) throws IgniteCheckedException { + try { + if (ver > 0) + SecurityUtils.serializeVersion(ver); + + return spi.marshaller().unmarshal(bytes, U.resolveClassLoader(spi.ignite().configuration())); + } + finally { + SecurityUtils.restoreDefaultSerializeVersion(); + } + } + /** * Discovery messages history used for client reconnect. */ @@ -2995,6 +3039,8 @@ else if (!spi.failureDetectionTimeoutEnabled() && (e instanceof pendingMsgs.customDiscardId); try { + SecurityUtils.serializeVersion(1); + long tstamp = U.currentTimeMillis(); if (timeoutHelper == null) @@ -3033,6 +3079,8 @@ else if (!spi.failureDetectionTimeoutEnabled() && (e instanceof } } finally { + SecurityUtils.restoreDefaultSerializeVersion(); + clearNodeAddedMessage(msg); } @@ -3423,7 +3471,8 @@ else if (log.isDebugEnabled()) // Stick in authentication subject to node (use security-safe attributes for copy). Map attrs = new HashMap<>(node.getAttributes()); - attrs.put(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT, U.marshal(spi.marshaller(), subj)); + attrs.put(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT_V2, U.marshal(spi.marshaller(), subj)); + attrs.put(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT, marshalWithSecurityVersion(subj, 1)); node.setAttributes(attrs); } @@ -4073,9 +4122,22 @@ else if (!locNodeId.equals(node.id()) && ring.node(node.id()) != null) { else { SecurityContext subj = spi.nodeAuth.authenticateNode(node, cred); - SecurityContext coordSubj = U.unmarshal(spi.marshaller(), - node.attribute(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT), - U.resolveClassLoader(spi.ignite().configuration())); + byte[] subjBytes = node.attribute(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT); + byte[] subjBytesV2 = node.attribute(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT_V2); + + SecurityContext coordSubj; + + try { + if (subjBytesV2 == null) + SecurityUtils.serializeVersion(1); + + coordSubj = U.unmarshal(spi.marshaller(), + subjBytesV2 != null ? subjBytesV2 : subjBytes, + U.resolveClassLoader(spi.ignite().configuration())); + } + finally { + SecurityUtils.restoreDefaultSerializeVersion(); + } if (!permissionsEqual(coordSubj.subject().permissions(), subj.subject().permissions())) { // Node has not pass authentication. @@ -4158,13 +4220,23 @@ else if (!locNodeId.equals(node.id()) && ring.node(node.id()) != null) { new TcpDiscoveryAuthFailedMessage(locNodeId, spi.locHost); try { - ClassLoader cl = U.resolveClassLoader(spi.ignite().configuration()); - byte[] rmSubj = node.attribute(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT); byte[] locSubj = locNode.attribute(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT); - SecurityContext rmCrd = spi.marshaller().unmarshal(rmSubj, cl); - SecurityContext locCrd = spi.marshaller().unmarshal(locSubj, cl); + byte[] rmSubjV2 = node.attribute(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT_V2); + byte[] locSubjV2 = locNode.attribute(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT_V2); + + int ver = 1; // Compatible version. + + if (rmSubjV2 != null && locSubjV2 != null) { + rmSubj = rmSubjV2; + locSubj = locSubjV2; + + ver = 0; // Default version. + } + + SecurityContext rmCrd = unmarshalWithSecurityVersion(rmSubj, ver); + SecurityContext locCrd = unmarshalWithSecurityVersion(locSubj, ver); if (!permissionsEqual(locCrd.subject().permissions(), rmCrd.subject().permissions())) { @@ -5812,6 +5884,8 @@ else if (e.hasCause(ObjectStreamException.class) || while (!isInterrupted()) { try { + SecurityUtils.serializeVersion(1); + TcpDiscoveryAbstractMessage msg = U.unmarshal(spi.marshaller(), in, U.resolveClassLoader(spi.ignite().configuration())); @@ -6062,6 +6136,9 @@ else if (msg instanceof TcpDiscoveryLoopbackProblemMessage) { return; } + finally { + SecurityUtils.restoreDefaultSerializeVersion(); + } } } finally { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java index ba8fa5b6e15a3..f0096db7bc8a6 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java @@ -21,14 +21,18 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.configuration.DeploymentMode; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.binary.BinaryMarshaller; import org.apache.ignite.marshaller.optimized.OptimizedMarshaller; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.TestReconnectPluginProvider; +import org.apache.ignite.spi.discovery.tcp.TestReconnectProcessor; 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; import static org.apache.ignite.IgniteSystemProperties.IGNITE_OPTIMIZED_MARSHALLER_USE_DEFAULT_SUID; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_SECURITY_COMPATIBILITY_MODE; import static org.apache.ignite.IgniteSystemProperties.IGNITE_SERVICES_COMPATIBILITY_MODE; import static org.apache.ignite.IgniteSystemProperties.IGNITE_BINARY_MARSHALLER_USE_STRING_SERIALIZATION_VER_2; import static org.apache.ignite.configuration.DeploymentMode.CONTINUOUS; @@ -258,18 +262,69 @@ public void testServiceCompatibilityEnabled() throws Exception { * @throws Exception If failed. */ private void doTestServiceCompatibilityEnabled(Object first, Object second, boolean fail) throws Exception { + doTestCompatibilityEnabled(IGNITE_SERVICES_COMPATIBILITY_MODE, first, second, fail); + } + + /** + * @throws Exception If failed. + */ + public void testSecurityCompatibilityEnabled() throws Exception { + TestReconnectPluginProvider.enabled = true; + TestReconnectProcessor.enabled = true; + + try { + doTestSecurityCompatibilityEnabled(true, null, true); + doTestSecurityCompatibilityEnabled(true, false, true); + doTestSecurityCompatibilityEnabled(false, true, true); + doTestSecurityCompatibilityEnabled(null, true, true); + + doTestSecurityCompatibilityEnabled(null, null, false); + doTestSecurityCompatibilityEnabled(null, false, false); + doTestSecurityCompatibilityEnabled(false, false, false); + doTestSecurityCompatibilityEnabled(false, null, false); + doTestSecurityCompatibilityEnabled(true, true, false); + } + finally { + TestReconnectPluginProvider.enabled = false; + TestReconnectProcessor.enabled = false; + } + } + + /** + * @param first Service compatibility enabled flag for first node. + * @param second Service compatibility enabled flag for second node. + * @param fail Fail flag. + * @throws Exception If failed. + */ + private void doTestSecurityCompatibilityEnabled(Object first, Object second, boolean fail) throws Exception { + doTestCompatibilityEnabled(IGNITE_SECURITY_COMPATIBILITY_MODE, first, second, fail); + } + + /** + * @param prop System property. + * @param first Service compatibility enabled flag for first node. + * @param second Service compatibility enabled flag for second node. + * @param fail Fail flag. + * @throws Exception If failed. + */ + private void doTestCompatibilityEnabled(String prop, Object first, Object second, boolean fail) throws Exception { + String backup = System.getProperty(prop); try { if (first != null) - System.setProperty(IGNITE_SERVICES_COMPATIBILITY_MODE, String.valueOf(first)); + System.setProperty(prop, String.valueOf(first)); else - System.clearProperty(IGNITE_SERVICES_COMPATIBILITY_MODE); + System.clearProperty(prop); - startGrid(0); + IgniteEx ignite = startGrid(0); + + // Ignore if disabled security plugin used. + if (IGNITE_SECURITY_COMPATIBILITY_MODE.equals(prop) && !ignite.context().security().enabled()) + return; if (second != null) - System.setProperty(IGNITE_SERVICES_COMPATIBILITY_MODE, String.valueOf(second)); + System.setProperty(prop, String.valueOf(second)); else - System.clearProperty(IGNITE_SERVICES_COMPATIBILITY_MODE); + System.clearProperty(prop); try { startGrid(1); @@ -284,6 +339,11 @@ private void doTestServiceCompatibilityEnabled(Object first, Object second, bool } finally { stopAllGrids(); + + if (backup != null) + System.setProperty(prop, backup); + else + System.clearProperty(prop); } } diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TestReconnectProcessor.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TestReconnectProcessor.java index f0ed35c4ccfab..2476bd3d787bf 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TestReconnectProcessor.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TestReconnectProcessor.java @@ -17,11 +17,13 @@ package org.apache.ignite.spi.discovery.tcp; +import java.io.Serializable; import java.util.Collection; import java.util.UUID; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.IgniteNodeAttributes; import org.apache.ignite.internal.processors.GridProcessorAdapter; import org.apache.ignite.internal.processors.security.GridSecurityProcessor; import org.apache.ignite.internal.processors.security.SecurityContext; @@ -37,6 +39,9 @@ * Updates node attributes on disconnect. */ public class TestReconnectProcessor extends GridProcessorAdapter implements GridSecurityProcessor { + /** Enabled flag. */ + public static boolean enabled; + /** * @param ctx Kernal context. */ @@ -44,10 +49,15 @@ protected TestReconnectProcessor(GridKernalContext ctx) { super(ctx); } + /** {@inheritDoc} */ + @Override public void start() throws IgniteCheckedException { + ctx.addNodeAttribute(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS, new SecurityCredentials()); + } + /** {@inheritDoc} */ @Override public SecurityContext authenticateNode(ClusterNode node, SecurityCredentials cred) throws IgniteCheckedException { - return null; + return new TestSecurityContext(); } /** {@inheritDoc} */ @@ -83,11 +93,44 @@ protected TestReconnectProcessor(GridKernalContext ctx) { /** {@inheritDoc} */ @Override public boolean enabled() { - return false; + return enabled; } /** {@inheritDoc} */ @Override public void onDisconnected(IgniteFuture reconnectFut) throws IgniteCheckedException { ctx.addNodeAttribute("test", "2"); } + + /** + * + */ + private static class TestSecurityContext implements SecurityContext, Serializable { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Override public SecuritySubject subject() { + return null; + } + + /** {@inheritDoc} */ + @Override public boolean taskOperationAllowed(String taskClsName, SecurityPermission perm) { + return true; + } + + /** {@inheritDoc} */ + @Override public boolean cacheOperationAllowed(String cacheName, SecurityPermission perm) { + return true; + } + + /** {@inheritDoc} */ + @Override public boolean serviceOperationAllowed(String srvcName, SecurityPermission perm) { + return true; + } + + /** {@inheritDoc} */ + @Override public boolean systemOperationAllowed(SecurityPermission perm) { + return true; + } + } } From b77428d12658b3ab2cdd43ca61ed71d329e83283 Mon Sep 17 00:00:00 2001 From: sboikov Date: Tue, 10 Jan 2017 16:59:17 +0300 Subject: [PATCH 22/90] Do not evict removed entries, otherwise removes can be lost. (cherry picked from commit 55ac6e7) --- .../internal/processors/cache/GridCacheMapEntry.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java index 58b4ae3130c9d..c8b8cd115fefc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java @@ -4300,6 +4300,10 @@ private synchronized CacheEntryImplEx wrapVersionedWithValue() { if (obsoleteVersionExtras() != null) return true; + // TODO GG-11241: need keep removed entries in heap map, otherwise removes can be lost. + if (cctx.deferredDelete() && deletedUnlocked()) + return false; + CacheObject prev = saveOldValueUnlocked(false); if (!hasReaders() && markObsolete0(obsoleteVer, false, null)) { @@ -4358,6 +4362,10 @@ private synchronized CacheEntryImplEx wrapVersionedWithValue() { // Version has changed since entry passed the filter. Do it again. continue; + // TODO GG-11241: need keep removed entries in heap map, otherwise removes can be lost. + if (cctx.deferredDelete() && deletedUnlocked()) + return false; + CacheObject prevVal = saveValueForIndexUnlocked(); if (!hasReaders() && markObsolete0(obsoleteVer, false, null)) { From 29187ef6b663eafe67eaaaf38e4c09fc244ac7aa Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Wed, 24 May 2017 17:31:27 +0300 Subject: [PATCH 23/90] Do not evict removed entries, otherwise removes can be lost. Rollback due to test failings. --- .../internal/processors/cache/GridCacheMapEntry.java | 8 -------- 1 file changed, 8 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java index c8b8cd115fefc..58b4ae3130c9d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java @@ -4300,10 +4300,6 @@ private synchronized CacheEntryImplEx wrapVersionedWithValue() { if (obsoleteVersionExtras() != null) return true; - // TODO GG-11241: need keep removed entries in heap map, otherwise removes can be lost. - if (cctx.deferredDelete() && deletedUnlocked()) - return false; - CacheObject prev = saveOldValueUnlocked(false); if (!hasReaders() && markObsolete0(obsoleteVer, false, null)) { @@ -4362,10 +4358,6 @@ private synchronized CacheEntryImplEx wrapVersionedWithValue() { // Version has changed since entry passed the filter. Do it again. continue; - // TODO GG-11241: need keep removed entries in heap map, otherwise removes can be lost. - if (cctx.deferredDelete() && deletedUnlocked()) - return false; - CacheObject prevVal = saveValueForIndexUnlocked(); if (!hasReaders() && markObsolete0(obsoleteVer, false, null)) { From 442aac2507210d39b7f30ab8f8d9a3dbe2610cae Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Wed, 24 May 2017 18:32:11 +0300 Subject: [PATCH 24/90] IGNITE-5225: Fix NPE caused by changes in IGNITE-4577. (cherry picked from commit d463840) --- .../ignite/internal/util/IgniteUtils.java | 4 ++-- .../communication/tcp/TcpCommunicationSpi.java | 17 +++++++++++------ 2 files changed, 13 insertions(+), 8 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java index ba118cb118cea..ca29adf81e1d3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java @@ -1813,11 +1813,11 @@ public static synchronized boolean isLocalHostChanged() throws IOException { * @return List of reachable addresses. */ public static List filterReachable(Collection addrs) { - final int reachTimeout = 2000; - if (addrs.isEmpty()) return Collections.emptyList(); + final int reachTimeout = 2000; + if (addrs.size() == 1) { InetAddress addr = F.first(addrs); 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 81454f827e58f..6dba6b2b5fd0c 100644 --- 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 @@ -2336,22 +2336,27 @@ protected GridCommunicationClient createTcpClient(ClusterNode node) throws Ignit Set allInetAddrs = U.newHashSet(addrs.size()); - for (InetSocketAddress addr : addrs) - allInetAddrs.add(addr.getAddress()); + for (InetSocketAddress addr : addrs) { + // Skip unresolved as addr.getAddress() can return null. + if(!addr.isUnresolved()) + allInetAddrs.add(addr.getAddress()); + } List reachableInetAddrs = U.filterReachable(allInetAddrs); if (reachableInetAddrs.size() < allInetAddrs.size()) { LinkedHashSet addrs0 = U.newLinkedHashSet(addrs.size()); + List unreachableInetAddr = new ArrayList<>(allInetAddrs.size() - reachableInetAddrs.size()); + for (InetSocketAddress addr : addrs) { if (reachableInetAddrs.contains(addr.getAddress())) addrs0.add(addr); + else + unreachableInetAddr.add(addr); } - for (InetSocketAddress addr : addrs) { - if (!reachableInetAddrs.contains(addr.getAddress())) - addrs0.add(addr); - } + + addrs0.addAll(unreachableInetAddr); addrs = addrs0; } From b1736c0bd87d6cfb65f9ef422241e0f1aba04c8d Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Wed, 24 May 2017 18:48:52 +0300 Subject: [PATCH 25/90] Fixed thread pools incorrect shutdown. (cherry picked from commit 66cef22) --- .../processors/cache/GridCacheAdapter.java | 12 ++--- .../ignite/internal/util/IgniteUtils.java | 45 ++++++++++--------- 2 files changed, 30 insertions(+), 27 deletions(-) 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 11bf34b221bb1..189f602328894 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 @@ -1132,14 +1132,14 @@ public List> splitClearLocally(boolean srv, bool if (!F.isEmpty(jobs)) { ExecutorService execSvc = null; - if (jobs.size() > 1) { - execSvc = Executors.newFixedThreadPool(jobs.size() - 1); + try { + if (jobs.size() > 1) { + execSvc = Executors.newFixedThreadPool(jobs.size() - 1); - for (int i = 1; i < jobs.size(); i++) - execSvc.submit(jobs.get(i)); - } + for (int i = 1; i < jobs.size(); i++) + execSvc.submit(jobs.get(i)); + } - try { jobs.get(0).run(); } finally { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java index ca29adf81e1d3..c2efb953c9135 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java @@ -1833,33 +1833,36 @@ public static List filterReachable(Collection addrs) { ExecutorService executor = Executors.newFixedThreadPool(Math.min(10, addrs.size())); - for (final InetAddress addr : addrs) { - futs.add(executor.submit(new Runnable() { - @Override public void run() { - if (reachable(addr, reachTimeout)) { - synchronized (res) { - res.add(addr); + try { + for (final InetAddress addr : addrs) { + futs.add(executor.submit(new Runnable() { + @Override public void run() { + if (reachable(addr, reachTimeout)) { + synchronized (res) { + res.add(addr); + } } } - } - })); - } - - for (Future fut : futs) { - try { - fut.get(); + })); } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new IgniteException("Thread has been interrupted.", e); - } - catch (ExecutionException e) { - throw new IgniteException(e); + for (Future fut : futs) { + try { + fut.get(); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + + throw new IgniteException("Thread has been interrupted.", e); + } + catch (ExecutionException e) { + throw new IgniteException(e); + } } } - - executor.shutdown(); + finally { + executor.shutdown(); + } return res; } From d77a134fffee431cd7fa0bae2349419bc97ec1cf Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Tue, 30 May 2017 19:00:47 +0300 Subject: [PATCH 26/90] IGNITE-5342 - Skip permission check for TASK_EXECUTE for service jobs --- .../processors/affinity/GridAffinityProcessor.java | 2 +- .../internal/processors/cache/GridCacheAdapter.java | 6 +++--- .../datastructures/CacheDataStructuresManager.java | 4 ++-- .../processors/closure/GridClosureProcessor.java | 11 ++++++++--- .../processors/service/GridServiceProcessor.java | 4 ++-- .../internal/processors/service/GridServiceProxy.java | 4 ++-- .../internal/processors/task/GridTaskProcessor.java | 6 ++++-- .../processors/task/GridTaskThreadContextKey.java | 5 ++++- .../internal/IgniteComputeTopologyExceptionTest.java | 2 +- 9 files changed, 27 insertions(+), 17 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java index b6efafbab394c..1be59784db6ed 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java @@ -497,7 +497,7 @@ private Map> keysToNodes(@Nullable final String c private AffinityInfo affinityInfoFromNode(@Nullable String cacheName, AffinityTopologyVersion topVer, ClusterNode n) throws IgniteCheckedException { GridTuple3 t = ctx.closure() - .callAsyncNoFailover(BROADCAST, affinityJob(cacheName, topVer), F.asList(n), true/*system pool*/, 0).get(); + .callAsyncNoFailover(BROADCAST, affinityJob(cacheName, topVer), F.asList(n), true/*system pool*/, 0, false).get(); AffinityFunction f = (AffinityFunction)unmarshall(ctx, n.id(), t.get1()); AffinityKeyMapper m = (AffinityKeyMapper)unmarshall(ctx, n.id(), t.get2()); 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 189f602328894..ac1d2682d5f2e 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 @@ -84,8 +84,8 @@ import org.apache.ignite.internal.processors.cache.affinity.GridCacheAffinityImpl; import org.apache.ignite.internal.processors.cache.distributed.IgniteExternalizableExpiryPolicy; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheAdapter; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtInvalidPartitionException; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition; import org.apache.ignite.internal.processors.cache.dr.GridCacheDrInfo; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxAdapter; @@ -3665,14 +3665,14 @@ private IgniteInternalFuture runLoadKeysCallable(final Set keys, new LoadKeysCallableV2<>(ctx.name(), keys, update, plc, keepBinary), nodes, true, - 0); + 0, false); } else { return ctx.closures().callAsyncNoFailover(BROADCAST, new LoadKeysCallable<>(ctx.name(), keys, update, plc), nodes, true, - 0); + 0, false); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java index c1983df4f4d86..366a4a920b9ee 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java @@ -492,7 +492,7 @@ public void removeSetData(IgniteUuid id) throws IgniteCheckedException { new BlockSetCallable(cctx.name(), id), nodes, true, - 0).get(); + 0, false).get(); } catch (IgniteCheckedException e) { if (e.hasCause(ClusterTopologyCheckedException.class)) { @@ -516,7 +516,7 @@ else if (!pingNodes(nodes)) { new RemoveSetDataCallable(cctx.name(), id, topVer), nodes, true, - 0).get(); + 0, false).get(); } catch (IgniteCheckedException e) { if (e.hasCause(ClusterTopologyCheckedException.class)) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java index 20fb6a0d0f485..aea7fe0a1e89b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java @@ -80,6 +80,7 @@ import static org.apache.ignite.compute.ComputeJobResultPolicy.FAILOVER; import static org.apache.ignite.compute.ComputeJobResultPolicy.REDUCE; import static org.apache.ignite.internal.processors.task.GridTaskThreadContextKey.TC_NO_FAILOVER; +import static org.apache.ignite.internal.processors.task.GridTaskThreadContextKey.TC_SKIP_AUTH; import static org.apache.ignite.internal.processors.task.GridTaskThreadContextKey.TC_SUBGRID; import static org.apache.ignite.internal.processors.task.GridTaskThreadContextKey.TC_TIMEOUT; @@ -514,11 +515,12 @@ public ComputeTaskInternalFuture affinityRun(@NotNull Collection cach } /** + * @param Type. * @param mode Distribution mode. * @param job Closure to execute. * @param nodes Grid nodes. * @param sys If {@code true}, then system pool will be used. - * @param Type. + * @param skipAuth Skip authorization check. * @return Grid future for collection of closure results. */ public IgniteInternalFuture callAsyncNoFailover( @@ -526,8 +528,8 @@ public IgniteInternalFuture callAsyncNoFailover( @Nullable Callable job, @Nullable Collection nodes, boolean sys, - long timeout - ) { + long timeout, + boolean skipAuth) { assert mode != null; assert timeout >= 0 : timeout; @@ -543,6 +545,9 @@ public IgniteInternalFuture callAsyncNoFailover( ctx.task().setThreadContext(TC_NO_FAILOVER, true); ctx.task().setThreadContext(TC_SUBGRID, nodes); + if (skipAuth) + ctx.task().setThreadContext(TC_SKIP_AUTH, true); + if (timeout > 0) ctx.task().setThreadContext(TC_TIMEOUT, timeout); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java index d7b9abc7fc1bd..25a8edbfc7c17 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java @@ -721,8 +721,8 @@ public Map serviceTopology(String name, long timeout) throws Igni call, Collections.singletonList(node), false, - timeout - ).get(); + timeout, + true).get(); } else return serviceTopology(cache, name); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java index aa609340b3bfb..2286cff9717aa 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java @@ -182,8 +182,8 @@ else if (U.isToStringMethod(mtd)) new ServiceProxyCallable(mtd.getName(), name, mtd.getParameterTypes(), args), Collections.singleton(node), false, - waitTimeout - ).get(); + waitTimeout, + true).get(); } } catch (GridServiceNotFoundException | ClusterTopologyCheckedException e) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java index 935686456e593..12213581b3286 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java @@ -82,6 +82,7 @@ import static org.apache.ignite.internal.GridTopic.TOPIC_TASK; import static org.apache.ignite.internal.GridTopic.TOPIC_TASK_CANCEL; import static org.apache.ignite.internal.managers.communication.GridIoPolicy.SYSTEM_POOL; +import static org.apache.ignite.internal.processors.task.GridTaskThreadContextKey.TC_SKIP_AUTH; import static org.apache.ignite.internal.processors.task.GridTaskThreadContextKey.TC_SUBGRID; import static org.apache.ignite.internal.processors.task.GridTaskThreadContextKey.TC_SUBJ_ID; import static org.apache.ignite.internal.processors.task.GridTaskThreadContextKey.TC_TASK_NAME; @@ -477,8 +478,6 @@ private ComputeTaskInternalFuture startTask( else taskClsName = taskCls != null ? taskCls.getName() : taskName; - ctx.security().authorize(taskClsName, SecurityPermission.TASK_EXECUTE, null); - // Get values from thread-local context. Map map = thCtx.get(); @@ -488,6 +487,9 @@ private ComputeTaskInternalFuture startTask( // Reset thread-local context. thCtx.set(null); + if (map.get(TC_SKIP_AUTH) == null) + ctx.security().authorize(taskClsName, SecurityPermission.TASK_EXECUTE, null); + Long timeout = (Long)map.get(TC_TIMEOUT); long timeout0 = timeout == null || timeout == 0 ? Long.MAX_VALUE : timeout; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskThreadContextKey.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskThreadContextKey.java index 3bb19241a4ebf..a45f851d902a1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskThreadContextKey.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskThreadContextKey.java @@ -34,5 +34,8 @@ public enum GridTaskThreadContextKey { TC_TIMEOUT, /** Security subject ID. */ - TC_SUBJ_ID + TC_SUBJ_ID, + + /** Skip authorization for the task. */ + TC_SKIP_AUTH } \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteComputeTopologyExceptionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteComputeTopologyExceptionTest.java index 3ed91e805d656..a82373b618507 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteComputeTopologyExceptionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteComputeTopologyExceptionTest.java @@ -89,7 +89,7 @@ public void testCorrectCheckedException() throws Exception { }, nodes, false, - 0); + 0, false); try { fut.get(); From 744a81ba937ba83ecdefa7c71f198d92d21527bb Mon Sep 17 00:00:00 2001 From: Anton Vinogradov Date: Wed, 31 May 2017 15:27:33 +0300 Subject: [PATCH 27/90] IGNITE-5232 [BACKPORT] GridDhtPartitionDemander.requestPartitions invokes sendMessages consequently, which lead to significant increase of node start time on large clusters with ssl --- .../preloader/GridDhtPartitionDemander.java | 131 ++++++++++-------- 1 file changed, 71 insertions(+), 60 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java index a6808c73577e1..daae1e2aa144a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java @@ -331,41 +331,21 @@ Runnable addAssignments(final GridDhtPreloaderAssignments assigns, return new Runnable() { @Override public void run() { - try { - if (next != null) - fut.listen(new CI1>() { - @Override public void apply(IgniteInternalFuture f) { - try { - if (f.get()) // Not cancelled. - next.run(); // Starts next cache rebalancing (according to the order). - } - catch (IgniteCheckedException ignored) { - if (log.isDebugEnabled()) - log.debug(ignored.getMessage()); - } + if (next != null) + fut.listen(new CI1>() { + @Override public void apply(IgniteInternalFuture f) { + try { + if (f.get()) // Not cancelled. + next.run(); // Starts next cache rebalancing (according to the order). } - }); - - requestPartitions(fut, assigns); - } - catch (IgniteCheckedException e) { - ClusterTopologyCheckedException cause = e.getCause(ClusterTopologyCheckedException.class); - - if (cause != null) - log.warning("Failed to send initial demand request to node. " + e.getMessage()); - else - log.error("Failed to send initial demand request to node.", e); - - fut.cancel(); - } - catch (Throwable th) { - log.error("Runtime error caught during initial demand request sending.", th); - - fut.cancel(); + catch (IgniteCheckedException ignored) { + if (log.isDebugEnabled()) + log.debug(ignored.getMessage()); + } + } + }); - if (th instanceof Error) - throw th; - } + requestPartitions(fut, assigns); } }; } @@ -404,9 +384,8 @@ else if (delay > 0) { * @return Partitions were requested. */ private void requestPartitions( - RebalanceFuture fut, - GridDhtPreloaderAssignments assigns - ) throws IgniteCheckedException { + final RebalanceFuture fut, + GridDhtPreloaderAssignments assigns){ if (topologyChanged(fut)) { fut.cancel(); @@ -438,7 +417,7 @@ private void requestPartitions( int lsnrCnt = cctx.gridConfig().getRebalanceThreadPoolSize(); - List> sParts = new ArrayList<>(lsnrCnt); + final List> sParts = new ArrayList<>(lsnrCnt); for (int cnt = 0; cnt < lsnrCnt; cnt++) sParts.add(new HashSet()); @@ -453,42 +432,74 @@ private void requestPartitions( for (cnt = 0; cnt < lsnrCnt; cnt++) { if (!sParts.get(cnt).isEmpty()) { // Create copy. - GridDhtPartitionDemandMessage initD = new GridDhtPartitionDemandMessage(d, sParts.get(cnt)); + final GridDhtPartitionDemandMessage initD = new GridDhtPartitionDemandMessage(d, sParts.get(cnt)); initD.topic(rebalanceTopics.get(cnt)); initD.updateSequence(fut.updateSeq); initD.timeout(cctx.config().getRebalanceTimeout()); - synchronized (fut) { - if (!fut.isDone()) { - // Future can be already cancelled at this moment and all failovers happened. - // New requests will not be covered by failovers. - cctx.io().sendOrderedMessage(node, - rebalanceTopics.get(cnt), initD, cctx.ioPolicy(), initD.timeout()); - } - } + final int finalCnt = cnt; - if (log.isDebugEnabled()) - log.debug("Requested rebalancing [from node=" + node.id() + ", listener index=" + - cnt + ", partitions count=" + sParts.get(cnt).size() + - " (" + partitionsList(sParts.get(cnt)) + ")]"); + cctx.kernalContext().closure().runLocalSafe(new Runnable() { + @Override public void run() { + try { + if (!fut.isDone()) { + cctx.io().sendOrderedMessage(node, + rebalanceTopics.get(finalCnt), initD, cctx.ioPolicy(), initD.timeout()); + + // Cleanup required in case partitions demanded in parallel with cancellation. + synchronized (fut) { + if (fut.isDone()) + fut.cleanupRemoteContexts(node.id()); + } + + if (log.isDebugEnabled()) + log.debug("Requested rebalancing [from node=" + node.id() + ", listener index=" + + finalCnt + ", partitions count=" + sParts.get(finalCnt).size() + + " (" + partitionsList(sParts.get(finalCnt)) + ")]"); + } + } + catch (IgniteCheckedException e) { + ClusterTopologyCheckedException cause = e.getCause(ClusterTopologyCheckedException.class); + + if (cause != null) + log.warning("Failed to send initial demand request to node. " + e.getMessage()); + else + log.error("Failed to send initial demand request to node.", e); + + fut.cancel(); + } + catch (Throwable th) { + log.error("Runtime error caught during initial demand request sending.", th); + + fut.cancel(); + } + } + }, /*system pool*/true); } } } else { - U.log(log, "Starting rebalancing (old api) [cache=" + cctx.name() + - ", mode=" + cfg.getRebalanceMode() + - ", fromNode=" + node.id() + - ", partitionsCount=" + parts.size() + - ", topology=" + fut.topologyVersion() + - ", updateSeq=" + fut.updateSeq + "]"); + try { + U.log(log, "Starting rebalancing (old api) [cache=" + cctx.name() + + ", mode=" + cfg.getRebalanceMode() + + ", fromNode=" + node.id() + + ", partitionsCount=" + parts.size() + + ", topology=" + fut.topologyVersion() + + ", updateSeq=" + fut.updateSeq + "]"); - d.timeout(cctx.config().getRebalanceTimeout()); - d.workerId(0);//old api support. + d.timeout(cctx.config().getRebalanceTimeout()); + d.workerId(0);//old api support. + + worker = new DemandWorker(dmIdx.incrementAndGet(), fut); - worker = new DemandWorker(dmIdx.incrementAndGet(), fut); + worker.run(node, d); + } + catch (Throwable th) { + log.error("Runtime error caught during initial demand request sending.", th); - worker.run(node, d); + fut.cancel(); + } } } } From 8220fb121eec86c18a711816f3db478b1d31a4e6 Mon Sep 17 00:00:00 2001 From: agura Date: Wed, 24 May 2017 19:55:09 +0300 Subject: [PATCH 28/90] ignite-5283 Fix of transaction recovery on backup when primary node failed --- .../GridDistributedLockRequest.java | 20 ++ .../distributed/dht/GridDhtLockFuture.java | 1 + .../distributed/dht/GridDhtLockRequest.java | 4 + .../dht/GridDhtTransactionalCacheAdapter.java | 3 +- .../dht/GridDhtTxPrepareFuture.java | 2 + .../dht/GridDhtTxPrepareRequest.java | 25 +- .../distributed/dht/GridDhtTxRemote.java | 24 +- .../cache/transactions/IgniteTxHandler.java | 3 +- .../IgniteTxCachePrimarySyncTest.java | 35 ++- .../IgniteCachePutRetryAbstractSelfTest.java | 48 ++++ .../dht/TxRecoveryStoreEnabledTest.java | 227 ++++++++++++++++++ .../IgniteCacheTxRecoverySelfTestSuite.java | 3 + 12 files changed, 375 insertions(+), 20 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/TxRecoveryStoreEnabledTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockRequest.java index 9639a9a7288ee..3142c8ba836ac 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockRequest.java @@ -48,6 +48,9 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage { /** */ private static final long serialVersionUID = 0L; + /** */ + private static final int STORE_USED_FLAG_MASK = 0x04; + /** Sender node ID. */ private UUID nodeId; @@ -262,6 +265,23 @@ public boolean keepBinary() { return (flags & KEEP_BINARY_FLAG_MASK) != 0; } + /** + * @return Flag indicating whether transaction use cache store. + */ + public boolean storeUsed() { + return (flags & STORE_USED_FLAG_MASK) != 0; + } + + /** + * @param storeUsed Store used value. + */ + public void storeUsed(boolean storeUsed) { + if (storeUsed) + flags = (byte)(flags | STORE_USED_FLAG_MASK); + else + flags &= ~STORE_USED_FLAG_MASK; + } + /** * @return Transaction isolation or null if not in transaction. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java index 686a4c6bed872..ddeb1ad1da507 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java @@ -873,6 +873,7 @@ private void map(Iterable entries) { inTx() ? tx.taskNameHash() : 0, read ? accessTtl : -1L, skipStore, + cctx.store().configured(), keepBinary, cctx.deploymentEnabled()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockRequest.java index 95c6dfc06ce44..289ad93d884fa 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockRequest.java @@ -120,6 +120,7 @@ public GridDhtLockRequest() { * @param taskNameHash Task name hash code. * @param accessTtl TTL for read operation. * @param skipStore Skip store flag. + * @param storeUsed Cache store configured flag. * @param keepBinary Keep binary flag. * @param addDepInfo Deployment info flag. */ @@ -144,6 +145,7 @@ public GridDhtLockRequest( int taskNameHash, long accessTtl, boolean skipStore, + boolean storeUsed, boolean keepBinary, boolean addDepInfo ) { @@ -166,6 +168,8 @@ public GridDhtLockRequest( this.topVer = topVer; + storeUsed(storeUsed); + nearKeys = nearCnt == 0 ? Collections.emptyList() : new ArrayList(nearCnt); invalidateEntries = new BitSet(dhtCnt == 0 ? nearCnt : dhtCnt); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java index a9e3bc496721c..13a88e9720ed7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java @@ -245,7 +245,8 @@ protected GridDhtTransactionalCacheAdapter(GridCacheContext ctx, GridCache req.timeout(), req.txSize(), req.subjectId(), - req.taskNameHash()); + req.taskNameHash(), + !req.skipStore() && req.storeUsed()); tx = ctx.tm().onCreated(null, tx); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java index a759194453a78..4283ccb878a47 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java @@ -1247,6 +1247,7 @@ private void prepare0() { tx.subjectId(), tx.taskNameHash(), tx.activeCachesDeploymentEnabled(), + tx.storeUsed(), retVal); int idx = 0; @@ -1359,6 +1360,7 @@ private void prepare0() { tx.subjectId(), tx.taskNameHash(), tx.activeCachesDeploymentEnabled(), + tx.storeUsed(), retVal); for (IgniteTxEntry entry : nearMapping.entries()) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java index a8f20876abd29..9fa20d8b75b65 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java @@ -55,6 +55,9 @@ public class GridDhtTxPrepareRequest extends GridDistributedTxPrepareRequest { /** */ public static final int NEED_RETURN_VALUE_FLAG_MASK = 0x01; + /** */ + public static final int STORE_USED_FLAG_MASK = 0x02; + /** Max order. */ private UUID nearNodeId; @@ -124,8 +127,9 @@ public GridDhtTxPrepareRequest() { * @param txNodes Transaction nodes mapping. * @param nearXidVer Near transaction ID. * @param last {@code True} if this is last prepare request for node. - * @param retVal Need return value flag. * @param addDepInfo Deployment info flag. + * @param storeUsed Cache store used flag. + * @param retVal Need return value flag. */ public GridDhtTxPrepareRequest( IgniteUuid futId, @@ -142,6 +146,7 @@ public GridDhtTxPrepareRequest( UUID subjId, int taskNameHash, boolean addDepInfo, + boolean storeUsed, boolean retVal) { super(tx, timeout, null, dhtWrites, txNodes, onePhaseCommit, addDepInfo); @@ -157,6 +162,7 @@ public GridDhtTxPrepareRequest( this.subjId = subjId; this.taskNameHash = taskNameHash; + storeUsed(storeUsed); needReturnValue(retVal); invalidateNearEntries = new BitSet(dhtWrites == null ? 0 : dhtWrites.size()); @@ -181,6 +187,23 @@ public void needReturnValue(boolean retVal) { flags &= ~NEED_RETURN_VALUE_FLAG_MASK; } + /** + * @return Flag indicating whether transaction use cache store. + */ + public boolean storeUsed() { + return (flags & STORE_USED_FLAG_MASK) != 0; + } + + /** + * @param storeUsed Store used value. + */ + public void storeUsed(boolean storeUsed) { + if (storeUsed) + flags = (byte)(flags | STORE_USED_FLAG_MASK); + else + flags &= ~STORE_USED_FLAG_MASK; + } + /** * @return {@code True} if this is last prepare request for node. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxRemote.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxRemote.java index 6ad20c7566553..8942ef9d9178c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxRemote.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxRemote.java @@ -64,6 +64,9 @@ public class GridDhtTxRemote extends GridDistributedTxRemoteAdapter { /** Near transaction ID. */ private GridCacheVersion nearXidVer; + /** Store used. */ + private boolean storeUsed; + /** * Empty constructor required for {@link Externalizable}. */ @@ -74,6 +77,7 @@ public GridDhtTxRemote() { /** * This constructor is meant for optimistic transactions. * + * @param ctx Cache context. * @param nearNodeId Near node ID. * @param rmtFutId Remote future ID. * @param nodeId Node ID. @@ -85,10 +89,10 @@ public GridDhtTxRemote() { * @param isolation Transaction isolation. * @param invalidate Invalidate flag. * @param timeout Timeout. - * @param ctx Cache context. * @param txSize Expected transaction size. * @param nearXidVer Near transaction ID. * @param txNodes Transaction nodes mapping. + * @param storeUsed Cache store used flag. */ public GridDhtTxRemote( GridCacheSharedContext ctx, @@ -109,8 +113,8 @@ public GridDhtTxRemote( Map> txNodes, @Nullable UUID subjId, int taskNameHash, - boolean single - ) { + boolean single, + boolean storeUsed) { super( ctx, nodeId, @@ -134,6 +138,7 @@ public GridDhtTxRemote( this.rmtFutId = rmtFutId; this.nearXidVer = nearXidVer; this.txNodes = txNodes; + this.storeUsed = storeUsed; txState = single ? new IgniteTxRemoteSingleStateImpl() : new IgniteTxRemoteStateImpl( @@ -148,6 +153,7 @@ public GridDhtTxRemote( /** * This constructor is meant for pessimistic transactions. * + * @param ctx Cache context. * @param nearNodeId Near node ID. * @param rmtFutId Remote future ID. * @param nodeId Node ID. @@ -160,8 +166,8 @@ public GridDhtTxRemote( * @param isolation Transaction isolation. * @param invalidate Invalidate flag. * @param timeout Timeout. - * @param ctx Cache context. * @param txSize Expected transaction size. + * @param storeUsed Cache store used flag. */ public GridDhtTxRemote( GridCacheSharedContext ctx, @@ -180,8 +186,8 @@ public GridDhtTxRemote( long timeout, int txSize, @Nullable UUID subjId, - int taskNameHash - ) { + int taskNameHash, + boolean storeUsed) { super( ctx, nodeId, @@ -204,6 +210,7 @@ public GridDhtTxRemote( this.nearXidVer = nearXidVer; this.nearNodeId = nearNodeId; this.rmtFutId = rmtFutId; + this.storeUsed = storeUsed; txState = new IgniteTxRemoteStateImpl( Collections.emptyMap(), @@ -226,6 +233,11 @@ public void transactionNodes(Map> txNodes) { return true; } + /** {@inheritDoc} */ + @Override public boolean storeUsed() { + return storeUsed; + } + /** {@inheritDoc} */ @Override public UUID eventNodeId() { return nearNodeId(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java index d56415652bc73..71bf08cfdfdbb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java @@ -1417,7 +1417,8 @@ protected void sendReply(UUID nodeId, GridDhtTxFinishRequest req, boolean commit req.transactionNodes(), req.subjectId(), req.taskNameHash(), - single); + single, + req.storeUsed()); tx.writeVersion(req.writeVersion()); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxCachePrimarySyncTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxCachePrimarySyncTest.java index 3bc22ef8c05c7..020ec5c8646d1 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxCachePrimarySyncTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxCachePrimarySyncTest.java @@ -23,6 +23,7 @@ import java.util.List; import java.util.Map; import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; import javax.cache.Cache; import javax.cache.configuration.Factory; import javax.cache.integration.CacheLoaderException; @@ -62,6 +63,7 @@ import org.apache.ignite.transactions.Transaction; import org.apache.ignite.transactions.TransactionConcurrency; import org.apache.ignite.transactions.TransactionIsolation; +import org.jsr166.ConcurrentHashMap8; import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_ASYNC; @@ -1096,19 +1098,30 @@ private static class TestStoreFactory implements Factory create() { - return new CacheStoreAdapter() { - @Override public Object load(Object key) throws CacheLoaderException { - return null; - } + return new TestCacheStore(); + } + } - @Override public void write(Cache.Entry entry) throws CacheWriterException { - // No-op. - } + /** + * + */ + private static class TestCacheStore extends CacheStoreAdapter { + /** Store map. */ + private static final Map STORE_MAP = new ConcurrentHashMap(); - @Override public void delete(Object key) throws CacheWriterException { - // No-op. - } - }; + /** {@inheritDoc} */ + @Override public Object load(Object key) throws CacheLoaderException { + return STORE_MAP.get(key); + } + + /** {@inheritDoc} */ + @Override public void write(Cache.Entry entry) throws CacheWriterException { + STORE_MAP.put(entry.getKey(), entry.getValue()); + } + + /** {@inheritDoc} */ + @Override public void delete(Object key) throws CacheWriterException { + STORE_MAP.remove(key); } } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java index c3d194b0c4f27..fc9017927d3bc 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java @@ -24,6 +24,7 @@ import java.util.Random; import java.util.Set; import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicBoolean; import javax.cache.Cache; import javax.cache.configuration.Factory; @@ -56,6 +57,7 @@ import org.apache.ignite.spi.swapspace.inmemory.GridTestSwapSpaceSpi; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.jsr166.ConcurrentHashMap8; import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.CLOCK; import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC; @@ -650,4 +652,50 @@ private static class TestStoreFactory implements Factory { }; } } + + +/* + private static class TestStoreFactory implements Factory { + */ +/** {@inheritDoc} *//* + + @Override public CacheStore create() { + return new TestCacheStore(); + } + } + + */ +/** + * + *//* + + private static class TestCacheStore extends CacheStoreAdapter { + */ +/** Store map. *//* + + private static Map STORE_MAP = new ConcurrentHashMap(); + + */ +/** {@inheritDoc} *//* + + @Override public Object load(Object key) throws CacheLoaderException { + return STORE_MAP.get(key); + } + + */ +/** {@inheritDoc} *//* + + @Override public void write(Cache.Entry entry) throws CacheWriterException { + STORE_MAP.put(entry.getKey(), entry.getValue()); + } + + */ +/** {@inheritDoc} *//* + + @Override public void delete(Object key) throws CacheWriterException { + STORE_MAP.remove(key); + } + } +*/ + } \ No newline at end of file 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 new file mode 100644 index 0000000000000..73f0268c42ef3 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/TxRecoveryStoreEnabledTest.java @@ -0,0 +1,227 @@ +/* + * 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.cache.distributed.dht; + +import java.util.concurrent.CountDownLatch; +import javax.cache.Cache; +import javax.cache.configuration.Factory; +import javax.cache.integration.CacheLoaderException; +import javax.cache.integration.CacheWriterException; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.CacheWriteSynchronizationMode; +import org.apache.ignite.cache.store.CacheStore; +import org.apache.ignite.cache.store.CacheStoreAdapter; +import org.apache.ignite.cluster.ClusterNode; +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.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; + +import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC; +import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; +import static org.apache.ignite.transactions.TransactionIsolation.READ_COMMITTED; + +/** + * + */ +public class TxRecoveryStoreEnabledTest extends GridCommonAbstractTest { + /** Nodes count. */ + private static final int NODES_CNT = 2; + + /** Latch. */ + private static CountDownLatch latch; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setCommunicationSpi(new TestCommunicationSpi()); + cfg.setDiscoverySpi(new TestDiscoverySpi()); + + CacheConfiguration ccfg = defaultCacheConfiguration(); + + ccfg.setNearConfiguration(null); + ccfg.setCacheMode(CacheMode.PARTITIONED); + ccfg.setBackups(1); + ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL); + ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); + ccfg.setCacheStoreFactory(new TestCacheStoreFactory()); + ccfg.setReadThrough(true); + ccfg.setWriteThrough(true); + ccfg.setWriteBehindEnabled(false); + + cfg.setCacheConfiguration(ccfg); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + latch = new CountDownLatch(1); + + startGrids(NODES_CNT); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** + * @throws Exception If failed. + */ + public void testOptimistic() throws Exception { + checkTxRecovery(OPTIMISTIC); + } + + /** + * @throws Exception If failed. + */ + public void testPessimistic() throws Exception { + checkTxRecovery(PESSIMISTIC); + } + + /** + * @throws Exception If failed. + */ + private void checkTxRecovery(TransactionConcurrency concurrency) throws Exception { + final Ignite node0 = ignite(0); + Ignite node1 = ignite(1); + + IgniteInternalFuture fut = multithreadedAsync(new Runnable() { + @Override public void run() { + try { + latch.await(); + + IgniteConfiguration cfg = node0.configuration(); + + ((TestCommunicationSpi)cfg.getCommunicationSpi()).block(); + ((TestDiscoverySpi)cfg.getDiscoverySpi()).simulateNodeFailure(); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + }, 1); + + IgniteCache cache0 = node0.cache(null); + + Integer key = primaryKey(cache0); + + try (Transaction tx = node0.transactions().txStart(concurrency, READ_COMMITTED)) { + cache0.put(key, key); + + tx.commit(); + } + catch (Exception e) { + // No-op. + } + + fut.get(); + + IgniteCache cache1 = node1.cache(null); + + assertNull(cache1.get(key)); + } + + /** + * + */ + private static class TestCacheStoreFactory implements Factory { + /** {@inheritDoc} */ + @Override public CacheStore create() { + return new TestCacheStore(); + } + } + + /** + * + */ + private static class TestCacheStore extends CacheStoreAdapter { + /** {@inheritDoc} */ + @Override public void sessionEnd(boolean commit) { + if (latch.getCount() > 0) { // Need wait only on primary node. + latch.countDown(); + + try { + U.sleep(3000); + } + catch (IgniteInterruptedCheckedException e) { + Thread.currentThread().interrupt(); + } + } + } + + /** {@inheritDoc} */ + @Override public Integer load(Integer key) throws CacheLoaderException { + return null; + } + + /** {@inheritDoc} */ + @Override public void write(Cache.Entry entry) throws CacheWriterException { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void delete(Object key) throws CacheWriterException { + // no-op. + } + } + + /** + * + */ + private static class TestDiscoverySpi extends TcpDiscoverySpi { + /** {@inheritDoc} */ + @Override protected void simulateNodeFailure() { + super.simulateNodeFailure(); + } + } + + /** + * + */ + private static class TestCommunicationSpi extends TcpCommunicationSpi { + /** Block. */ + private volatile boolean block; + + /** {@inheritDoc} */ + @Override public void sendMessage(ClusterNode node, Message msg) throws IgniteSpiException { + if (!block) + super.sendMessage(node, msg); + } + + /** + * + */ + private void block() { + block = true; + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTxRecoverySelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTxRecoverySelfTestSuite.java index 7363c7cc1f2a8..c7c8db67fe31e 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTxRecoverySelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTxRecoverySelfTestSuite.java @@ -25,6 +25,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCachePartitionedNearDisabledPrimaryNodeFailureRecoveryTest; import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCachePartitionedPrimaryNodeFailureRecoveryTest; import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCachePartitionedTwoBackupsPrimaryNodeFailureRecoveryTest; +import org.apache.ignite.internal.processors.cache.distributed.dht.TxRecoveryStoreEnabledTest; import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheNearTxPessimisticOriginatingNodeFailureSelfTest; import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCacheReplicatedTxOriginatingNodeFailureSelfTest; import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCacheReplicatedTxPessimisticOriginatingNodeFailureSelfTest; @@ -54,6 +55,8 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(GridCacheNearTxPessimisticOriginatingNodeFailureSelfTest.class); suite.addTestSuite(GridCacheReplicatedTxPessimisticOriginatingNodeFailureSelfTest.class); + suite.addTestSuite(TxRecoveryStoreEnabledTest.class); + return suite; } } \ No newline at end of file From f03252f9b2c6f0e777f307fd85cc8bd20ab21423 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Tue, 6 Jun 2017 16:17:01 +0300 Subject: [PATCH 29/90] IGNITE-5259 Minor serialization fix (cherry picked from commit 374cba8) --- .../ignite/spi/discovery/tcp/ClientImpl.java | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java index 02ba56a884b33..b075dc1fb5289 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java @@ -57,6 +57,7 @@ import org.apache.ignite.internal.IgniteClientDisconnectedCheckedException; import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.IgniteNodeAttributes; +import org.apache.ignite.internal.processors.security.SecurityUtils; import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.typedef.F; @@ -516,7 +517,16 @@ else if (state == DISCONNECTED) { InetSocketAddress addr = it.next(); - T3 sockAndRes = sendJoinRequest(recon, addr); + T3 sockAndRes; + + SecurityUtils.serializeVersion(1); + + try { + sockAndRes = sendJoinRequest(recon, addr); + } + finally { + SecurityUtils.restoreDefaultSerializeVersion(); + } if (sockAndRes == null) { it.remove(); @@ -969,6 +979,8 @@ private void forceStopRead() throws InterruptedException { while (!isInterrupted()) { TcpDiscoveryAbstractMessage msg; + SecurityUtils.serializeVersion(1); + try { msg = U.unmarshal(spi.marshaller(), in, U.resolveClassLoader(spi.ignite().configuration())); } @@ -994,6 +1006,9 @@ private void forceStopRead() throws InterruptedException { continue; } + finally { + SecurityUtils.restoreDefaultSerializeVersion(); + } msg.senderNodeId(rmtNodeId); From c2c237d1222557d3e6b35d9a51a61a4c78e56782 Mon Sep 17 00:00:00 2001 From: Sergey Kalashnikov Date: Fri, 3 Feb 2017 11:41:14 +0300 Subject: [PATCH 30/90] IGNITE-4196: Added means to specify port for H2 debug console. This closes #1486. (cherry picked from commit b246260) --- .../java/org/apache/ignite/IgniteSystemProperties.java | 8 ++++++++ .../internal/processors/query/h2/IgniteH2Indexing.java | 8 +++++++- 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java index 6827e0c248290..f35e9e620070a 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -320,6 +320,14 @@ public final class IgniteSystemProperties { */ public static final String IGNITE_H2_DEBUG_CONSOLE = "IGNITE_H2_DEBUG_CONSOLE"; + /** + * This property allows to specify user defined port which H2 indexing SPI will use + * to start H2 debug console on. If this property is not set or set to 0, H2 debug + * console will use system-provided dynamic port. + * This property is only relevant when {@link #IGNITE_H2_DEBUG_CONSOLE} property is set. + */ + public static final String IGNITE_H2_DEBUG_CONSOLE_PORT = "IGNITE_H2_DEBUG_CONSOLE_PORT"; + /** * If this property is set to {@code true} then shared memory space native debug will be enabled. */ diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java index 62b47b85c6507..8b2993f263cf7 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java @@ -171,8 +171,10 @@ import org.jsr166.ConcurrentHashMap8; import static org.apache.ignite.IgniteSystemProperties.IGNITE_H2_DEBUG_CONSOLE; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_H2_DEBUG_CONSOLE_PORT; import static org.apache.ignite.IgniteSystemProperties.IGNITE_H2_INDEXING_CACHE_CLEANUP_PERIOD; import static org.apache.ignite.IgniteSystemProperties.IGNITE_H2_INDEXING_CACHE_THREAD_USAGE_TIMEOUT; +import static org.apache.ignite.IgniteSystemProperties.getInteger; import static org.apache.ignite.IgniteSystemProperties.getString; import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.SQL; import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.SQL_FIELDS; @@ -1751,11 +1753,15 @@ public GridReduceQueryExecutor reduceQueryExecutor() { if (getString(IGNITE_H2_DEBUG_CONSOLE) != null) { Connection c = DriverManager.getConnection(dbUrl); + int port = getInteger(IGNITE_H2_DEBUG_CONSOLE_PORT, 0); + WebServer webSrv = new WebServer(); - Server web = new Server(webSrv, "-webPort", "0"); + Server web = new Server(webSrv, "-webPort", Integer.toString(port)); web.start(); String url = webSrv.addSession(c); + U.quietAndInfo(log, "H2 debug console URL: " + url); + try { Server.openBrowser(url); } From 5dd74ff635de50ff9561ccdb51bdeb620f60c3db Mon Sep 17 00:00:00 2001 From: sboikov Date: Tue, 10 Jan 2017 16:59:17 +0300 Subject: [PATCH 31/90] Do not evict removed entries, otherwise removes can be lost. (cherry picked from commit 55ac6e7) --- .../internal/processors/cache/GridCacheMapEntry.java | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java index 58b4ae3130c9d..ea01bca4bd50b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java @@ -29,7 +29,6 @@ import javax.cache.expiry.ExpiryPolicy; import javax.cache.processor.EntryProcessor; import javax.cache.processor.EntryProcessorResult; - import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; @@ -4300,6 +4299,10 @@ private synchronized CacheEntryImplEx wrapVersionedWithValue() { if (obsoleteVersionExtras() != null) return true; + // TODO GG-11241: need keep removed entries in heap map, otherwise removes can be lost. + if (cctx.deferredDelete() && deletedUnlocked()) + return false; + CacheObject prev = saveOldValueUnlocked(false); if (!hasReaders() && markObsolete0(obsoleteVer, false, null)) { @@ -4358,6 +4361,10 @@ private synchronized CacheEntryImplEx wrapVersionedWithValue() { // Version has changed since entry passed the filter. Do it again. continue; + // TODO GG-11241: need keep removed entries in heap map, otherwise removes can be lost. + if (cctx.deferredDelete() && deletedUnlocked()) + return false; + CacheObject prevVal = saveValueForIndexUnlocked(); if (!hasReaders() && markObsolete0(obsoleteVer, false, null)) { From 5858efd406bb54352de14a0a7e7f21c2ac7bf899 Mon Sep 17 00:00:00 2001 From: sboikov Date: Fri, 16 Dec 2016 19:23:29 +0300 Subject: [PATCH 32/90] IGNITE-2412 - Do not acquire asyncSemaphore for synchronous operations (cherry-picked from master) (cherry picked from commit 82b4073) --- .../processors/cache/GridCacheAdapter.java | 703 +++++++----------- .../dht/atomic/GridDhtAtomicCache.java | 469 +++++++----- .../dht/colocated/GridDhtColocatedCache.java | 13 - .../distributed/near/GridNearAtomicCache.java | 65 +- .../local/atomic/GridLocalAtomicCache.java | 173 +---- ...bledMultiNodeLongTxTimeoutFullApiTest.java | 2 +- ...ckMessageSystemPoolStarvationSelfTest.java | 13 +- 7 files changed, 596 insertions(+), 842 deletions(-) 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 ac1d2682d5f2e..47c9998239777 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 @@ -248,16 +248,16 @@ public abstract class GridCacheAdapter implements IgniteInternalCache ctx, @Nullable GridCache metrics = new CacheMetricsImpl(ctx); - localMxBean = new CacheLocalMetricsMXBeanImpl(this); + locMxBean = new CacheLocalMetricsMXBeanImpl(this); clusterMxBean = new CacheClusterMetricsMXBeanImpl(this); FileSystemConfiguration[] igfsCfgs = gridCfg.getFileSystemConfiguration(); @@ -456,49 +456,49 @@ public boolean isDht() { public abstract GridCachePreloader preloader(); /** {@inheritDoc} */ - @Override public Affinity affinity() { + @Override public final Affinity affinity() { return aff; } /** {@inheritDoc} */ @SuppressWarnings({"unchecked", "RedundantCast"}) - @Override public IgniteInternalCache cache() { + @Override public final IgniteInternalCache cache() { return (IgniteInternalCache)this; } /** {@inheritDoc} */ - @Override public GridCacheProxyImpl forSubjectId(UUID subjId) { + @Override public final GridCacheProxyImpl forSubjectId(UUID subjId) { CacheOperationContext opCtx = new CacheOperationContext(false, subjId, false, null, false, null); return new GridCacheProxyImpl<>(ctx, this, opCtx); } /** {@inheritDoc} */ - @Override public boolean skipStore() { + @Override public final boolean skipStore() { return false; } /** {@inheritDoc} */ - @Override public GridCacheProxyImpl setSkipStore(boolean skipStore) { + @Override public final GridCacheProxyImpl setSkipStore(boolean skipStore) { CacheOperationContext opCtx = new CacheOperationContext(true, null, false, null, false, null); return new GridCacheProxyImpl<>(ctx, this, opCtx); } /** {@inheritDoc} */ - @Override public GridCacheProxyImpl keepBinary() { + @Override public final GridCacheProxyImpl keepBinary() { CacheOperationContext opCtx = new CacheOperationContext(false, null, true, null, false, null); return new GridCacheProxyImpl<>((GridCacheContext)ctx, (GridCacheAdapter)this, opCtx); } /** {@inheritDoc} */ - @Nullable @Override public ExpiryPolicy expiry() { + @Nullable @Override public final ExpiryPolicy expiry() { return null; } /** {@inheritDoc} */ - @Override public GridCacheProxyImpl withExpiryPolicy(ExpiryPolicy plc) { + @Override public final GridCacheProxyImpl withExpiryPolicy(ExpiryPolicy plc) { assert !CU.isUtilityCache(ctx.name()); assert !CU.isAtomicsCache(ctx.name()); assert !CU.isMarshallerCache(ctx.name()); @@ -509,14 +509,14 @@ public boolean isDht() { } /** {@inheritDoc} */ - @Override public IgniteInternalCache withNoRetries() { + @Override public final IgniteInternalCache withNoRetries() { CacheOperationContext opCtx = new CacheOperationContext(false, null, false, null, true, null); return new GridCacheProxyImpl<>(ctx, this, opCtx); } /** {@inheritDoc} */ - @Override public CacheConfiguration configuration() { + @Override public final CacheConfiguration configuration() { return ctx.config(); } @@ -617,7 +617,7 @@ public void onKernalStop() { } /** {@inheritDoc} */ - @Override public boolean isEmpty() { + @Override public final boolean isEmpty() { try { return localSize(CachePeekModes.ONHEAP_ONLY) == 0; } @@ -627,7 +627,7 @@ public void onKernalStop() { } /** {@inheritDoc} */ - @Override public boolean containsKey(K key) { + @Override public final boolean containsKey(K key) { try { return containsKeyAsync(key).get(); } @@ -654,7 +654,7 @@ public void onKernalStop() { } /** {@inheritDoc} */ - @Override public boolean containsKeys(Collection keys) { + @Override public final boolean containsKeys(Collection keys) { try { return containsKeysAsync(keys).get(); } @@ -664,7 +664,7 @@ public void onKernalStop() { } /** {@inheritDoc} */ - @Override public IgniteInternalFuture containsKeysAsync(final Collection keys) { + @Override public final IgniteInternalFuture containsKeysAsync(final Collection keys) { A.notNull(keys, "keys"); return getAllAsync( @@ -695,7 +695,7 @@ public void onKernalStop() { } /** {@inheritDoc} */ - @Override public Iterable> localEntries(CachePeekMode[] peekModes) throws IgniteCheckedException { + @Override public final Iterable> localEntries(CachePeekMode[] peekModes) throws IgniteCheckedException { assert peekModes != null; ctx.checkSecurity(SecurityPermission.CACHE_READ); @@ -752,7 +752,7 @@ public String toString() { /** {@inheritDoc} */ @SuppressWarnings("ForLoopReplaceableByForEach") - @Nullable @Override public V localPeek(K key, + @Nullable @Override public final V localPeek(K key, CachePeekMode[] peekModes, @Nullable IgniteCacheExpiryPolicy plc) throws IgniteCheckedException { @@ -894,7 +894,7 @@ public String toString() { * * @param ldr Class loader to undeploy. */ - public void onUndeploy(ClassLoader ldr) { + public final void onUndeploy(ClassLoader ldr) { ctx.deploy().onUndeploy(ldr, context()); } @@ -903,7 +903,7 @@ public void onUndeploy(ClassLoader ldr) { * @param key Entry key. * @return Entry or null. */ - @Nullable public GridCacheEntryEx peekEx(KeyCacheObject key) { + @Nullable public final GridCacheEntryEx peekEx(KeyCacheObject key) { return entry0(key, ctx.affinity().affinityTopologyVersion(), false, false); } @@ -912,7 +912,7 @@ public void onUndeploy(ClassLoader ldr) { * @param key Entry key. * @return Entry or null. */ - @Nullable public GridCacheEntryEx peekEx(Object key) { + @Nullable public final GridCacheEntryEx peekEx(Object key) { return entry0(ctx.toCacheKeyObject(key), ctx.affinity().affinityTopologyVersion(), false, false); } @@ -920,7 +920,7 @@ public void onUndeploy(ClassLoader ldr) { * @param key Entry key. * @return Entry (never {@code null}). */ - public GridCacheEntryEx entryEx(Object key) { + public final GridCacheEntryEx entryEx(Object key) { return entryEx(ctx.toCacheKeyObject(key), false); } @@ -928,7 +928,7 @@ public GridCacheEntryEx entryEx(Object key) { * @param key Entry key. * @return Entry (never {@code null}). */ - public GridCacheEntryEx entryEx(KeyCacheObject key) { + public final GridCacheEntryEx entryEx(KeyCacheObject key) { return entryEx(key, false); } @@ -983,24 +983,24 @@ public GridCacheEntryEx entryEx(KeyCacheObject key, AffinityTopologyVersion topV /** * @return Set of internal cached entry representations. */ - public Iterable entries() { + public final Iterable entries() { return allEntries(); } /** * @return Set of internal cached entry representations. */ - public Iterable allEntries() { + public final Iterable allEntries() { return map.entries(); } /** {@inheritDoc} */ - @Override public Set> entrySet() { + @Override public final Set> entrySet() { return entrySet((CacheEntryPredicate[])null); } /** {@inheritDoc} */ - @Override public Set> entrySetx(final CacheEntryPredicate... filter) { + @Override public final Set> entrySetx(final CacheEntryPredicate... filter) { boolean keepBinary = ctx.keepBinary(); return new EntrySet(map.entrySet(filter), keepBinary); @@ -1012,17 +1012,17 @@ public Iterable allEntries() { } /** {@inheritDoc} */ - @Override public Set keySet() { + @Override public final Set keySet() { return new KeySet(map.entrySet()); } /** {@inheritDoc} */ - @Override public Set keySetx() { + @Override public final Set keySetx() { return keySet(); } /** {@inheritDoc} */ - @Override public Set primaryKeySet() { + @Override public final Set primaryKeySet() { return new KeySet(map.entrySet(CU.cachePrimary(ctx.grid().affinity(ctx.name()), ctx.localNode()))); } @@ -1044,7 +1044,7 @@ public Iterable allEntries() { * @param filter Filters. * @return Collection of cached values. */ - public Iterable values(final CacheEntryPredicate... filter) { + public final Iterable values(final CacheEntryPredicate... filter) { return new Iterable() { @Override public Iterator iterator() { return new Iterator() { @@ -1070,12 +1070,12 @@ public Iterable values(final CacheEntryPredicate... filter) { * * @param key Entry key. */ - public void removeIfObsolete(KeyCacheObject key) { + public final void removeIfObsolete(KeyCacheObject key) { assert key != null; GridCacheMapEntry entry = map.getEntry(key); - if (entry.obsolete()) + if (entry != null && entry.obsolete()) removeEntry(entry); } @@ -1259,11 +1259,11 @@ public void clearLocally(Collection keys, boolean readers) { /** * @param entry Removes entry from cache if currently mapped value is the same as passed. */ - public void removeEntry(GridCacheEntryEx entry) { - boolean removed = map.removeEntry(entry); + public final void removeEntry(GridCacheEntryEx entry) { + boolean rmvd = map.removeEntry(entry); if (log.isDebugEnabled()) { - if (removed) + if (rmvd) log.debug("Removed entry from cache: " + entry); else log.debug("Remove will not be done for key (entry got replaced or removed): " + entry.key()); @@ -1298,7 +1298,7 @@ private boolean evictx(K key, GridCacheVersion ver, } /** {@inheritDoc} */ - @Override public V getForcePrimary(K key) throws IgniteCheckedException { + @Override public final V getForcePrimary(K key) throws IgniteCheckedException { String taskName = ctx.kernalContext().job().currentTaskName(); return getAllAsync( @@ -1315,7 +1315,7 @@ private boolean evictx(K key, GridCacheVersion ver, } /** {@inheritDoc} */ - @Override public IgniteInternalFuture getForcePrimaryAsync(final K key) { + @Override public final IgniteInternalFuture getForcePrimaryAsync(final K key) { String taskName = ctx.kernalContext().job().currentTaskName(); return getAllAsync( @@ -1336,7 +1336,7 @@ private boolean evictx(K key, GridCacheVersion ver, } /** {@inheritDoc} */ - public V getTopologySafe(K key) throws IgniteCheckedException { + public final V getTopologySafe(K key) throws IgniteCheckedException { String taskName = ctx.kernalContext().job().currentTaskName(); return getAllAsync( @@ -1353,12 +1353,12 @@ public V getTopologySafe(K key) throws IgniteCheckedException { } /** {@inheritDoc} */ - @Nullable @Override public Map getAllOutTx(Set keys) throws IgniteCheckedException { + @Nullable @Override public final Map getAllOutTx(Set keys) throws IgniteCheckedException { return getAllOutTxAsync(keys).get(); } /** {@inheritDoc} */ - @Override public IgniteInternalFuture> getAllOutTxAsync(Set keys) { + @Override public final IgniteInternalFuture> getAllOutTxAsync(Set keys) { String taskName = ctx.kernalContext().job().currentTaskName(); return getAllAsync(keys, @@ -1372,15 +1372,6 @@ public V getTopologySafe(K key) throws IgniteCheckedException { false); } - /** - * @param key Key. - * @param topVer Topology version. - * @return Entry. - */ - @Nullable protected GridCacheEntryEx entryExSafe(KeyCacheObject key, AffinityTopologyVersion topVer) { - return entryEx(key); - } - /** {@inheritDoc} */ @Nullable @Override public V get(K key) throws IgniteCheckedException { A.notNull(key, "key"); @@ -1521,14 +1512,14 @@ public V getTopologySafe(K key) throws IgniteCheckedException { } /** {@inheritDoc} */ - @Override public Map getAll(@Nullable Collection keys) throws IgniteCheckedException { + @Override public final Map getAll(@Nullable Collection keys) throws IgniteCheckedException { A.notNull(keys, "keys"); boolean statsEnabled = ctx.config().isStatisticsEnabled(); long start = statsEnabled ? System.nanoTime() : 0L; - Map map = getAll(keys, !ctx.keepBinary(), false); + Map map = getAll0(keys, !ctx.keepBinary(), false); if (ctx.config().getInterceptor() != null) map = interceptGet(keys, map); @@ -1548,7 +1539,7 @@ public V getTopologySafe(K key) throws IgniteCheckedException { long start = statsEnabled ? System.nanoTime() : 0L; - Map map = (Map)getAll(keys, !ctx.keepBinary(), true); + Map map = (Map)getAll0(keys, !ctx.keepBinary(), true); Collection> res = new HashSet<>(); @@ -2156,7 +2147,7 @@ protected final IgniteInternalFuture> getAllAsync0( } /** {@inheritDoc} */ - @Override public V getAndPut(K key, V val) throws IgniteCheckedException { + @Override public final V getAndPut(K key, V val) throws IgniteCheckedException { return getAndPut(key, val, null); } @@ -2178,7 +2169,24 @@ protected final IgniteInternalFuture> getAllAsync0( if (keyCheck) validateCacheKey(key); - V prevVal = syncOp(new SyncOp(true) { + V prevVal = getAndPut0(key, val, filter); + + if (statsEnabled) + metrics0().addPutAndGetTimeNanos(System.nanoTime() - start); + + return prevVal; + } + + /** + * @param key Key. + * @param val Value. + * @param filter Optional filter. + * @return Previous value. + * @throws IgniteCheckedException If failed. + */ + protected V getAndPut0(final K key, final V val, @Nullable final CacheEntryPredicate filter) + throws IgniteCheckedException { + return syncOp(new SyncOp(true) { @Override public V op(IgniteTxLocalAdapter tx) throws IgniteCheckedException { return (V)tx.putAsync(ctx, null, key, val, true, filter).get().value(); } @@ -2187,15 +2195,10 @@ protected final IgniteInternalFuture> getAllAsync0( return "put [key=" + key + ", val=" + val + ", filter=" + filter + ']'; } }); - - if (statsEnabled) - metrics0().addPutAndGetTimeNanos(System.nanoTime() - start); - - return prevVal; } /** {@inheritDoc} */ - @Override public IgniteInternalFuture getAndPutAsync(K key, V val) { + @Override public final IgniteInternalFuture getAndPutAsync(K key, V val) { return getAndPutAsync(key, val, null); } @@ -2205,11 +2208,16 @@ protected final IgniteInternalFuture> getAllAsync0( * @param filter Filter. * @return Put operation future. */ - public IgniteInternalFuture getAndPutAsync(K key, V val, @Nullable CacheEntryPredicate filter) { + protected final IgniteInternalFuture getAndPutAsync(K key, V val, @Nullable CacheEntryPredicate filter) { final boolean statsEnabled = ctx.config().isStatisticsEnabled(); final long start = statsEnabled ? System.nanoTime() : 0L; + A.notNull(key, "key", val, "val"); + + if (keyCheck) + validateCacheKey(key); + IgniteInternalFuture fut = getAndPutAsync0(key, val, filter); if (statsEnabled) @@ -2224,13 +2232,10 @@ public IgniteInternalFuture getAndPutAsync(K key, V val, @Nullable CacheEntry * @param filter Optional filter. * @return Put operation future. */ - public IgniteInternalFuture getAndPutAsync0(final K key, final V val, - @Nullable final CacheEntryPredicate filter) { - A.notNull(key, "key", val, "val"); - - if (keyCheck) - validateCacheKey(key); - + public IgniteInternalFuture getAndPutAsync0(final K key, + final V val, + @Nullable final CacheEntryPredicate filter) + { return asyncOp(new AsyncOp() { @Override public IgniteInternalFuture op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) { return tx.putAsync(ctx, readyTopVer, key, val, true, filter) @@ -2244,7 +2249,7 @@ public IgniteInternalFuture getAndPutAsync0(final K key, final V val, } /** {@inheritDoc} */ - @Override public boolean put(final K key, final V val) throws IgniteCheckedException { + @Override public final boolean put(final K key, final V val) throws IgniteCheckedException { return put(key, val, null); } @@ -2268,7 +2273,26 @@ public boolean put(final K key, final V val, final CacheEntryPredicate filter) if (keyCheck) validateCacheKey(key); - Boolean stored = syncOp(new SyncOp(true) { + boolean stored = put0(key, val, filter); + + if (statsEnabled && stored) + metrics0().addPutTimeNanos(System.nanoTime() - start); + + return stored; + } + + /** + * @param key Key. + * @param val Value. + * @param filter Filter. + * @return {@code True} if optional filter passed and value was stored in cache, + * {@code false} otherwise. Note that this method will return {@code true} if filter is not + * specified. + * @throws IgniteCheckedException If put operation failed. + */ + protected boolean put0(final K key, final V val, final CacheEntryPredicate filter) + throws IgniteCheckedException { + Boolean res = syncOp(new SyncOp(true) { @Override public Boolean op(IgniteTxLocalAdapter tx) throws IgniteCheckedException { return tx.putAsync(ctx, null, key, val, false, filter).get().success(); } @@ -2278,10 +2302,9 @@ public boolean put(final K key, final V val, final CacheEntryPredicate filter) } }); - if (statsEnabled) - metrics0().addPutTimeNanos(System.nanoTime() - start); + assert res != null; - return stored; + return res; } /** {@inheritDoc} */ @@ -2323,7 +2346,7 @@ public boolean put(final K key, final V val, final CacheEntryPredicate filter) } /** {@inheritDoc} */ - @Nullable @Override public EntryProcessorResult invoke(@Nullable AffinityTopologyVersion topVer, + @Nullable @Override public final EntryProcessorResult invoke(@Nullable AffinityTopologyVersion topVer, K key, EntryProcessor entryProcessor, Object... args) throws IgniteCheckedException { @@ -2556,7 +2579,7 @@ private EntryProcessorResult invoke0( } /** {@inheritDoc} */ - @Override public IgniteInternalFuture putAsync(K key, V val) { + @Override public final IgniteInternalFuture putAsync(K key, V val) { return putAsync(key, val, null); } @@ -2566,7 +2589,12 @@ private EntryProcessorResult invoke0( * @param filter Filter. * @return Put future. */ - public IgniteInternalFuture putAsync(K key, V val, @Nullable CacheEntryPredicate filter) { + public final IgniteInternalFuture putAsync(K key, V val, @Nullable CacheEntryPredicate filter) { + A.notNull(key, "key", val, "val"); + + if (keyCheck) + validateCacheKey(key); + final boolean statsEnabled = ctx.config().isStatisticsEnabled(); final long start = statsEnabled ? System.nanoTime() : 0L; @@ -2587,11 +2615,6 @@ public IgniteInternalFuture putAsync(K key, V val, @Nullable CacheEntry */ public IgniteInternalFuture putAsync0(final K key, final V val, @Nullable final CacheEntryPredicate filter) { - A.notNull(key, "key", val, "val"); - - if (keyCheck) - validateCacheKey(key); - return asyncOp(new AsyncOp() { @Override public IgniteInternalFuture op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) { return tx.putAsync(ctx, @@ -2619,267 +2642,82 @@ public IgniteInternalFuture putAsync0(final K key, final V val, } /** {@inheritDoc} */ - @Nullable @Override public V getAndPutIfAbsent(final K key, final V val) throws IgniteCheckedException { - A.notNull(key, "key", val, "val"); - - if (keyCheck) - validateCacheKey(key); - - return syncOp(new SyncOp(true) { - @Override public V op(IgniteTxLocalAdapter tx) throws IgniteCheckedException { - return (V)tx.putAsync(ctx, null, key, val, true, ctx.noVal()).get().value(); - } - - @Override public String toString() { - return "putIfAbsent [key=" + key + ", val=" + val + ']'; - } - }); + @Nullable @Override public final V getAndPutIfAbsent(final K key, final V val) throws IgniteCheckedException { + return getAndPut(key, val, ctx.noVal()); } /** {@inheritDoc} */ - @Override public IgniteInternalFuture getAndPutIfAbsentAsync(final K key, final V val) { - final boolean statsEnabled = ctx.config().isStatisticsEnabled(); - - final long start = statsEnabled ? System.nanoTime() : 0L; - - A.notNull(key, "key", val, "val"); - - if (keyCheck) - validateCacheKey(key); - - IgniteInternalFuture fut = asyncOp(new AsyncOp() { - @Override public IgniteInternalFuture op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) { - return tx.putAsync(ctx, readyTopVer, key, val, true, ctx.noVal()) - .chain((IgniteClosure, V>)RET2VAL); - } - - @Override public String toString() { - return "putIfAbsentAsync [key=" + key + ", val=" + val + ']'; - } - }); - - if (statsEnabled) - fut.listen(new UpdatePutTimeStatClosure(metrics0(), start)); - - return fut; + @Override public final IgniteInternalFuture getAndPutIfAbsentAsync(final K key, final V val) { + return getAndPutAsync(key, val, ctx.noVal()); } /** {@inheritDoc} */ - @Override public boolean putIfAbsent(final K key, final V val) throws IgniteCheckedException { - boolean statsEnabled = ctx.config().isStatisticsEnabled(); - - long start = statsEnabled ? System.nanoTime() : 0L; - - A.notNull(key, "key", val, "val"); - - if (keyCheck) - validateCacheKey(key); - - Boolean stored = syncOp(new SyncOp(true) { - @Override public Boolean op(IgniteTxLocalAdapter tx) throws IgniteCheckedException { - return tx.putAsync(ctx, null, key, val, false, ctx.noVal()).get().success(); - } - - @Override public String toString() { - return "putxIfAbsent [key=" + key + ", val=" + val + ']'; - } - }); - - if (statsEnabled && stored) - metrics0().addPutTimeNanos(System.nanoTime() - start); - - return stored; + @Override public final boolean putIfAbsent(final K key, final V val) throws IgniteCheckedException { + return put(key, val, ctx.noVal()); } /** {@inheritDoc} */ - @Override public IgniteInternalFuture putIfAbsentAsync(final K key, final V val) { - final boolean statsEnabled = ctx.config().isStatisticsEnabled(); - - final long start = statsEnabled ? System.nanoTime() : 0L; - - A.notNull(key, "key", val, "val"); - - if (keyCheck) - validateCacheKey(key); - - IgniteInternalFuture fut = asyncOp(new AsyncOp() { - @Override public IgniteInternalFuture op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) { - return tx.putAsync(ctx, - readyTopVer, - key, - val, - false, - ctx.noVal()).chain( - (IgniteClosure, Boolean>)RET2FLAG); - } - - @Override public String toString() { - return "putxIfAbsentAsync [key=" + key + ", val=" + val + ']'; - } - }); - - if (statsEnabled) - fut.listen(new UpdatePutTimeStatClosure(metrics0(), start)); - - return fut; + @Override public final IgniteInternalFuture putIfAbsentAsync(final K key, final V val) { + return putAsync(key, val, ctx.noVal()); } /** {@inheritDoc} */ - @Nullable @Override public V getAndReplace(final K key, final V val) throws IgniteCheckedException { - A.notNull(key, "key", val, "val"); - - if (keyCheck) - validateCacheKey(key); - - return syncOp(new SyncOp(true) { - @Override public V op(IgniteTxLocalAdapter tx) throws IgniteCheckedException { - return (V)tx.putAsync(ctx, null, key, val, true, ctx.hasVal()).get().value(); - } - - @Override public String toString() { - return "replace [key=" + key + ", val=" + val + ']'; - } - }); + @Nullable @Override public final V getAndReplace(final K key, final V val) throws IgniteCheckedException { + return getAndPut(key, val, ctx.hasVal()); } /** {@inheritDoc} */ - @Override public IgniteInternalFuture getAndReplaceAsync(final K key, final V val) { - final boolean statsEnabled = ctx.config().isStatisticsEnabled(); - - final long start = statsEnabled ? System.nanoTime() : 0L; - - A.notNull(key, "key", val, "val"); - - if (keyCheck) - validateCacheKey(key); - - IgniteInternalFuture fut = asyncOp(new AsyncOp() { - @Override public IgniteInternalFuture op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) { - return tx.putAsync(ctx, readyTopVer, key, val, true, ctx.hasVal()).chain( - (IgniteClosure, V>)RET2VAL); - } - - @Override public String toString() { - return "replaceAsync [key=" + key + ", val=" + val + ']'; - } - }); - - if (statsEnabled) - fut.listen(new UpdatePutAndGetTimeStatClosure(metrics0(), start)); - - return fut; + @Override public final IgniteInternalFuture getAndReplaceAsync(final K key, final V val) { + return getAndPutAsync(key, val, ctx.hasVal()); } /** {@inheritDoc} */ - @Override public boolean replace(final K key, final V val) throws IgniteCheckedException { - A.notNull(key, "key", val, "val"); - - if (keyCheck) - validateCacheKey(key); - - return syncOp(new SyncOp(true) { - @Override public Boolean op(IgniteTxLocalAdapter tx) throws IgniteCheckedException { - return tx.putAsync(ctx, null, key, val, false, ctx.hasVal()).get().success(); - } - - @Override public String toString() { - return "replacex [key=" + key + ", val=" + val + ']'; - } - }); + @Override public final boolean replace(final K key, final V val) throws IgniteCheckedException { + return put(key, val, ctx.hasVal()); } /** {@inheritDoc} */ - @Override public IgniteInternalFuture replaceAsync(final K key, final V val) { - A.notNull(key, "key", val, "val"); - - if (keyCheck) - validateCacheKey(key); - - return asyncOp(new AsyncOp() { - @Override public IgniteInternalFuture op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) { - return tx.putAsync(ctx, readyTopVer, key, val, false, ctx.hasVal()).chain( - (IgniteClosure, Boolean>) RET2FLAG); - } - - @Override public String toString() { - return "replacexAsync [key=" + key + ", val=" + val + ']'; - } - }); + @Override public final IgniteInternalFuture replaceAsync(final K key, final V val) { + return putAsync(key, val, ctx.hasVal()); } /** {@inheritDoc} */ - @Override public boolean replace(final K key, final V oldVal, final V newVal) throws IgniteCheckedException { - A.notNull(key, "key", oldVal, "oldVal", newVal, "newVal"); - - if (keyCheck) - validateCacheKey(key); + @Override public final boolean replace(final K key, final V oldVal, final V newVal) throws IgniteCheckedException { + A.notNull(oldVal, "oldVal"); - return syncOp(new SyncOp(true) { - @Override public Boolean op(IgniteTxLocalAdapter tx) throws IgniteCheckedException { - // Register before hiding in the filter. - if (ctx.deploymentEnabled()) - ctx.deploy().registerClass(oldVal); - - return tx.putAsync(ctx, null, key, newVal, false, ctx.equalsVal(oldVal)).get() - .success(); - } - - @Override public String toString() { - return "replace [key=" + key + ", oldVal=" + oldVal + ", newVal=" + newVal + ']'; - } - }); + return put(key, newVal, ctx.equalsVal(oldVal)); } /** {@inheritDoc} */ @Override public IgniteInternalFuture replaceAsync(final K key, final V oldVal, final V newVal) { - final boolean statsEnabled = ctx.config().isStatisticsEnabled(); - - final long start = statsEnabled ? System.nanoTime() : 0L; - - A.notNull(key, "key", oldVal, "oldVal", newVal, "newVal"); - - if (keyCheck) - validateCacheKey(key); - - IgniteInternalFuture fut = asyncOp(new AsyncOp() { - @Override public IgniteInternalFuture op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) { - // Register before hiding in the filter. - if (ctx.deploymentEnabled()) { - try { - ctx.deploy().registerClass(oldVal); - } - catch (IgniteCheckedException e) { - return new GridFinishedFuture<>(e); - } - } + A.notNull(oldVal, "oldVal"); - return tx.putAsync(ctx, readyTopVer, key, newVal, false, ctx.equalsVal(oldVal)).chain( - (IgniteClosure, Boolean>)RET2FLAG); - } - - @Override public String toString() { - return "replaceAsync [key=" + key + ", oldVal=" + oldVal + ", newVal=" + newVal + ']'; - } - }); - - if (statsEnabled) - fut.listen(new UpdatePutAndGetTimeStatClosure(metrics0(), start)); - - return fut; + return putAsync(key, newVal, ctx.equalsVal(oldVal)); } /** {@inheritDoc} */ @Override public void putAll(@Nullable final Map m) throws IgniteCheckedException { + if (F.isEmpty(m)) + return; + boolean statsEnabled = ctx.config().isStatisticsEnabled(); long start = statsEnabled ? System.nanoTime() : 0L; - if (F.isEmpty(m)) - return; - if (keyCheck) validateCacheKeys(m.keySet()); + putAll0(m); + + if (statsEnabled) + metrics0().addPutTimeNanos(System.nanoTime() - start); + } + + /** + * @param m Map. + * @throws IgniteCheckedException If failed. + */ + protected void putAll0(final Map m) throws IgniteCheckedException { syncOp(new SyncInOp(m.size() == 1) { @Override public void inOp(IgniteTxLocalAdapter tx) throws IgniteCheckedException { tx.putAllAsync(ctx, null, m, false).get(); @@ -2889,9 +2727,6 @@ public IgniteInternalFuture putAsync0(final K key, final V val, return "putAll [map=" + m + ']'; } }); - - if (statsEnabled) - metrics0().addPutTimeNanos(System.nanoTime() - start); } /** {@inheritDoc} */ @@ -2902,6 +2737,14 @@ public IgniteInternalFuture putAsync0(final K key, final V val, if (keyCheck) validateCacheKeys(m.keySet()); + return putAllAsync0(m); + } + + /** + * @param m Map. + * @return Future. + */ + protected IgniteInternalFuture putAllAsync0(final Map m) { return asyncOp(new AsyncOp(m.keySet()) { @Override public IgniteInternalFuture op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) { return tx.putAllAsync(ctx, @@ -2927,11 +2770,25 @@ public IgniteInternalFuture putAsync0(final K key, final V val, if (keyCheck) validateCacheKey(key); + V prevVal = getAndRemove0(key); + + if (statsEnabled) + metrics0().addRemoveAndGetTimeNanos(System.nanoTime() - start); + + return prevVal; + } + + /** + * @param key Key. + * @return Previous value. + * @throws IgniteCheckedException If failed. + */ + protected V getAndRemove0(final K key) throws IgniteCheckedException { final boolean keepBinary = ctx.keepBinary(); - V prevVal = syncOp(new SyncOp(true) { + return syncOp(new SyncOp(true) { @Override public V op(IgniteTxLocalAdapter tx) throws IgniteCheckedException { - K key0 = keepBinary ? (K)ctx.toCacheKeyObject(key) : key; + K key0 = keepBinary ? (K) ctx.toCacheKeyObject(key) : key; V ret = tx.removeAllAsync(ctx, null, @@ -2941,9 +2798,9 @@ public IgniteInternalFuture putAsync0(final K key, final V val, /*singleRmv*/false).get().value(); if (ctx.config().getInterceptor() != null) { - K key = keepBinary ? (K)ctx.unwrapBinaryIfNeeded(key0, true, false) : key0; + K key = keepBinary ? (K) ctx.unwrapBinaryIfNeeded(key0, true, false) : key0; - return (V)ctx.config().getInterceptor().onBeforeRemove(new CacheEntryImpl(key, ret)).get2(); + return (V) ctx.config().getInterceptor().onBeforeRemove(new CacheEntryImpl(key, ret)).get2(); } return ret; @@ -2953,11 +2810,6 @@ public IgniteInternalFuture putAsync0(final K key, final V val, return "remove [key=" + key + ']'; } }); - - if (statsEnabled) - metrics0().addRemoveAndGetTimeNanos(System.nanoTime() - start); - - return prevVal; } /** {@inheritDoc} */ @@ -2971,7 +2823,20 @@ public IgniteInternalFuture putAsync0(final K key, final V val, if (keyCheck) validateCacheKey(key); - IgniteInternalFuture fut = asyncOp(new AsyncOp() { + IgniteInternalFuture fut = getAndRemoveAsync0(key); + + if (statsEnabled) + fut.listen(new UpdateRemoveTimeStatClosure(metrics0(), start)); + + return fut; + } + + /** + * @param key Key. + * @return Future. + */ + protected IgniteInternalFuture getAndRemoveAsync0(final K key) { + return asyncOp(new AsyncOp() { @Override public IgniteInternalFuture op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) { // TODO should we invoke interceptor here? return tx.removeAllAsync(ctx, @@ -2986,11 +2851,6 @@ public IgniteInternalFuture putAsync0(final K key, final V val, return "removeAsync [key=" + key + ']'; } }); - - if (statsEnabled) - fut.listen(new UpdateRemoveTimeStatClosure(metrics0(), start)); - - return fut; } /** {@inheritDoc} */ @@ -3023,6 +2883,17 @@ public IgniteInternalFuture putAsync0(final K key, final V val, if (keyCheck) validateCacheKeys(keys); + removeAll0(keys); + + if (statsEnabled) + metrics0().addRemoveTimeNanos(System.nanoTime() - start); + } + + /** + * @param keys Keys. + * @throws IgniteCheckedException If failed. + */ + protected void removeAll0(final Collection keys) throws IgniteCheckedException { syncOp(new SyncInOp(keys.size() == 1) { @Override public void inOp(IgniteTxLocalAdapter tx) throws IgniteCheckedException { tx.removeAllAsync(ctx, @@ -3037,24 +2908,34 @@ public IgniteInternalFuture putAsync0(final K key, final V val, return "removeAll [keys=" + keys + ']'; } }); - - if (statsEnabled) - metrics0().addRemoveTimeNanos(System.nanoTime() - start); } /** {@inheritDoc} */ @Override public IgniteInternalFuture removeAllAsync(@Nullable final Collection keys) { + if (F.isEmpty(keys)) + return new GridFinishedFuture(); + final boolean statsEnabled = ctx.config().isStatisticsEnabled(); final long start = statsEnabled ? System.nanoTime() : 0L; - if (F.isEmpty(keys)) - return new GridFinishedFuture(); - if (keyCheck) validateCacheKeys(keys); - IgniteInternalFuture fut = asyncOp(new AsyncOp(keys) { + IgniteInternalFuture fut = removeAllAsync0(keys); + + if (statsEnabled) + fut.listen(new UpdateRemoveTimeStatClosure<>(metrics0(), start)); + + return fut; + } + + /** + * @param keys Keys. + * @return Future. + */ + protected IgniteInternalFuture removeAllAsync0(final Collection keys) { + return asyncOp(new AsyncOp(keys) { @Override public IgniteInternalFuture op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) { return tx.removeAllAsync(ctx, readyTopVer, @@ -3068,15 +2949,20 @@ public IgniteInternalFuture putAsync0(final K key, final V val, return "removeAllAsync [keys=" + keys + ']'; } }); - - if (statsEnabled) - fut.listen(new UpdateRemoveTimeStatClosure<>(metrics0(), start)); - - return fut; } /** {@inheritDoc} */ @Override public boolean remove(final K key) throws IgniteCheckedException { + return remove(key, (CacheEntryPredicate)null); + } + + /** + * @param key Key. + * @param filter Filter. + * @return {@code True} if entry was removed. + * @throws IgniteCheckedException If failed. + */ + public boolean remove(final K key, @Nullable CacheEntryPredicate filter) throws IgniteCheckedException { boolean statsEnabled = ctx.config().isStatisticsEnabled(); long start = statsEnabled ? System.nanoTime() : 0L; @@ -3086,13 +2972,27 @@ public IgniteInternalFuture putAsync0(final K key, final V val, if (keyCheck) validateCacheKey(key); - boolean rmv = syncOp(new SyncOp(true) { + boolean rmv = remove0(key, filter); + + if (statsEnabled && rmv) + metrics0().addRemoveTimeNanos(System.nanoTime() - start); + + return rmv; + } + + /** + * @param key Key. + * @return {@code True} if entry was removed. + * @throws IgniteCheckedException If failed. + */ + protected boolean remove0(final K key, final CacheEntryPredicate filter) throws IgniteCheckedException { + Boolean res = syncOp(new SyncOp(true) { @Override public Boolean op(IgniteTxLocalAdapter tx) throws IgniteCheckedException { return tx.removeAllAsync(ctx, null, Collections.singletonList(key), /*retval*/false, - null, + filter, /*singleRmv*/true).get().success(); } @@ -3101,10 +3001,9 @@ public IgniteInternalFuture putAsync0(final K key, final V val, } }); - if (statsEnabled && rmv) - metrics0().addRemoveTimeNanos(System.nanoTime() - start); + assert res != null; - return rmv; + return res; } /** {@inheritDoc} */ @@ -3129,7 +3028,21 @@ public IgniteInternalFuture removeAsync(final K key, @Nullable final Ca if (keyCheck) validateCacheKey(key); - IgniteInternalFuture fut = asyncOp(new AsyncOp() { + IgniteInternalFuture fut = removeAsync0(key, filter); + + if (statsEnabled) + fut.listen(new UpdateRemoveTimeStatClosure(metrics0(), start)); + + return fut; + } + + /** + * @param key Key. + * @param filter Filter. + * @return Future. + */ + protected IgniteInternalFuture removeAsync0(final K key, @Nullable final CacheEntryPredicate filter) { + return asyncOp(new AsyncOp() { @Override public IgniteInternalFuture op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) { return tx.removeAllAsync(ctx, readyTopVer, @@ -3144,11 +3057,6 @@ public IgniteInternalFuture removeAsync(final K key, @Nullable final Ca return "removeAsync [key=" + key + ", filter=" + filter + ']'; } }); - - if (statsEnabled) - fut.listen(new UpdateRemoveTimeStatClosure(metrics0(), start)); - - return fut; } /** {@inheritDoc} */ @@ -3190,86 +3098,21 @@ public IgniteInternalFuture removeAsync(final K key, @Nullable final Ca } /** {@inheritDoc} */ - @Override public boolean remove(final K key, final V val) throws IgniteCheckedException { - boolean statsEnabled = ctx.config().isStatisticsEnabled(); - - long start = statsEnabled ? System.nanoTime() : 0L; - - A.notNull(key, "key", val, "val"); + @Override public final boolean remove(final K key, final V val) throws IgniteCheckedException { + A.notNull(val, "val"); - if (keyCheck) - validateCacheKey(key); - - boolean rmv = syncOp(new SyncOp(true) { - @Override public Boolean op(IgniteTxLocalAdapter tx) throws IgniteCheckedException { - // Register before hiding in the filter. - if (ctx.deploymentEnabled()) - ctx.deploy().registerClass(val); - - return tx.removeAllAsync(ctx, - null, - Collections.singletonList(key), - /*retval*/false, - ctx.equalsVal(val), - /*singleRmv*/false).get().success(); - } - - @Override public String toString() { - return "remove [key=" + key + ", val=" + val + ']'; - } - }); - - if (statsEnabled && rmv) - metrics0().addRemoveTimeNanos(System.nanoTime() - start); - - return rmv; + return remove(key, ctx.equalsVal(val)); } /** {@inheritDoc} */ - @Override public IgniteInternalFuture removeAsync(final K key, final V val) { - final boolean statsEnabled = ctx.config().isStatisticsEnabled(); - - final long start = statsEnabled ? System.nanoTime() : 0L; - - A.notNull(key, "key", val, "val"); - - if (keyCheck) - validateCacheKey(key); + @Override public final IgniteInternalFuture removeAsync(final K key, final V val) { + A.notNull(key, "val"); - IgniteInternalFuture fut = asyncOp(new AsyncOp() { - @Override public IgniteInternalFuture op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) { - // Register before hiding in the filter. - if (ctx.deploymentEnabled()) { - try { - ctx.deploy().registerClass(val); - } - catch (IgniteCheckedException e) { - return new GridFinishedFuture<>(e); - } - } - - return tx.removeAllAsync(ctx, - readyTopVer, - Collections.singletonList(key), - /*retval*/false, - ctx.equalsVal(val), - /*singleRmv*/false).chain( - (IgniteClosure, Boolean>)RET2FLAG); - } - - @Override public String toString() { - return "removeAsync [key=" + key + ", val=" + val + ']'; - } - }); - - if (statsEnabled) - fut.listen(new UpdateRemoveTimeStatClosure(metrics0(), start)); - - return fut; + return removeAsync(key, ctx.equalsVal(val)); } /** {@inheritDoc} */ - @Override public CacheMetrics clusterMetrics() { + @Override public final CacheMetrics clusterMetrics() { return clusterMetrics(ctx.grid().cluster().forDataNodes(ctx.name())); } @@ -3298,7 +3141,7 @@ public IgniteInternalFuture removeAsync(final K key, @Nullable final Ca /** {@inheritDoc} */ @Override public CacheMetricsMXBean localMxBean() { - return localMxBean; + return locMxBean; } /** {@inheritDoc} */ @@ -4647,7 +4490,7 @@ protected void saveFuture(final FutureHolder holder, IgniteInternalFuture fut /** * Releases asynchronous operations permit, if limited. */ - protected void asyncOpRelease() { + private void asyncOpRelease() { if (asyncOpsSem != null) asyncOpsSem.release(); } @@ -4813,12 +4656,10 @@ public Set> entrySet(@Nullable CacheEntryPredicate... filter) * @return Cached value. * @throws IgniteCheckedException If failed. */ - @Nullable public V get(K key, boolean deserializeBinary, final boolean needVer) throws IgniteCheckedException { - checkJta(); - + @Nullable public final V get(K key, boolean deserializeBinary, final boolean needVer) throws IgniteCheckedException { String taskName = ctx.kernalContext().job().currentTaskName(); - return get(key, taskName, deserializeBinary, needVer); + return get0(key, taskName, deserializeBinary, needVer); } /** @@ -4829,11 +4670,13 @@ public Set> entrySet(@Nullable CacheEntryPredicate... filter) * @return Cached value. * @throws IgniteCheckedException If failed. */ - protected V get( + protected V get0( final K key, String taskName, boolean deserializeBinary, boolean needVer) throws IgniteCheckedException { + checkJta(); + try { return getAsync(key, !ctx.config().isReadFromBackup(), @@ -4887,7 +4730,7 @@ public final IgniteInternalFuture getAsync(final K key, boolean deserializeBi * @return Map of cached values. * @throws IgniteCheckedException If read failed. */ - public Map getAll(Collection keys, boolean deserializeBinary, + protected Map getAll0(Collection keys, boolean deserializeBinary, boolean needVer) throws IgniteCheckedException { checkJta(); @@ -4942,7 +4785,7 @@ public void forceKeyCheck() { * @param key Cache key. * @throws IllegalArgumentException If validation fails. */ - protected void validateCacheKey(Object key) { + protected final void validateCacheKey(Object key) { if (keyCheck) { CU.validateCacheKey(key); @@ -4957,7 +4800,7 @@ protected void validateCacheKey(Object key) { * @param keys Cache keys. * @throws IgniteException If validation fails. */ - protected void validateCacheKeys(Iterable keys) { + protected final void validateCacheKeys(Iterable keys) { if (keys == null) return; @@ -4978,7 +4821,7 @@ protected void validateCacheKeys(Iterable keys) { * @param deserializeBinary Deserialize binary flag. * @return Public API iterator. */ - protected Iterator> iterator(final Iterator it, + protected final Iterator> iterator(final Iterator it, final boolean deserializeBinary) { return new Iterator>() { { @@ -5296,7 +5139,7 @@ private static PeekModes parsePeekModes(CachePeekMode[] peekModes, boolean prima * @param plc Explicitly specified expiry policy for cache operation. * @return Expiry policy wrapper. */ - @Nullable public IgniteCacheExpiryPolicy expiryPolicy(@Nullable ExpiryPolicy plc) { + @Nullable public final IgniteCacheExpiryPolicy expiryPolicy(@Nullable ExpiryPolicy plc) { if (plc == null) plc = ctx.expiry(); @@ -5421,7 +5264,7 @@ public IgniteInternalFuture op(final IgniteTxLocalAdapter tx, CacheOperationC * @param opCtx Operation context. * @return Operation future. */ - protected IgniteInternalFuture waitTopologyFuture(IgniteInternalFuture topFut, + private IgniteInternalFuture waitTopologyFuture(IgniteInternalFuture topFut, final AffinityTopologyVersion topVer, final IgniteTxLocalAdapter tx, final CacheOperationContext opCtx) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java index 72e1bb1cc3080..dc6d3dd8271dd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java @@ -140,7 +140,7 @@ public class GridDhtAtomicCache extends GridDhtCacheAdapter { private CI2 updateReplyClos; /** Pending */ - private GridDeferredAckMessageSender deferredUpdateMessageSender; + private GridDeferredAckMessageSender deferredUpdateMsgSnd; /** */ private GridNearAtomicCache near; @@ -174,6 +174,11 @@ public GridDhtAtomicCache(GridCacheContext ctx, GridCacheConcurrentMap map msgLog = ctx.shared().atomicMessageLogger(); } + /** {@inheritDoc} */ + @Override protected void checkJta() throws IgniteCheckedException { + // No-op. + } + /** {@inheritDoc} */ @Override public boolean isDhtAtomic() { return true; @@ -236,7 +241,7 @@ else if (res.error() != null) { @Override public void start() throws IgniteCheckedException { super.start(); - deferredUpdateMessageSender = new GridDeferredAckMessageSender(ctx.time(), ctx.closures()) { + deferredUpdateMsgSnd = new GridDeferredAckMessageSender(ctx.time(), ctx.closures()) { @Override public int getTimeout() { return DEFERRED_UPDATE_RESPONSE_TIMEOUT; } @@ -448,7 +453,7 @@ else if (res.error() != null) { /** {@inheritDoc} */ @Override public void stop() { - deferredUpdateMessageSender.stop(); + deferredUpdateMsgSnd.stop(); } /** @@ -464,7 +469,7 @@ public void near(GridNearAtomicCache near) { } /** {@inheritDoc} */ - @Override protected V get(K key, String taskName, boolean deserializeBinary, boolean needVer) + @Override protected V get0(K key, String taskName, boolean deserializeBinary, boolean needVer) throws IgniteCheckedException { ctx.checkSecurity(SecurityPermission.CACHE_READ); @@ -540,6 +545,21 @@ public void near(GridNearAtomicCache near) { }); } + /** {@inheritDoc} */ + @Override protected Map getAll0(Collection keys, boolean deserializeBinary, boolean needVer) + throws IgniteCheckedException { + return getAllAsyncInternal(keys, + !ctx.config().isReadFromBackup(), + true, + null, + ctx.kernalContext().job().currentTaskName(), + deserializeBinary, + false, + true, + needVer, + false).get(); + } + /** {@inheritDoc} */ @Override public IgniteInternalFuture> getAllAsync( @Nullable final Collection keys, @@ -551,6 +571,43 @@ public void near(GridNearAtomicCache near) { final boolean skipVals, final boolean canRemap, final boolean needVer + ) { + return getAllAsyncInternal(keys, + forcePrimary, + skipTx, + subjId, + taskName, + deserializeBinary, + skipVals, + canRemap, + needVer, + true); + } + + /** + * @param keys Keys. + * @param forcePrimary Force primary flag. + * @param skipTx Skip tx flag. + * @param subjId Subject ID. + * @param taskName Task name. + * @param deserializeBinary Deserialize binary flag. + * @param skipVals Skip values flag. + * @param canRemap Can remap flag. + * @param needVer Need version flag. + * @param asyncOp Async operation flag. + * @return Future. + */ + private IgniteInternalFuture> getAllAsyncInternal( + @Nullable final Collection keys, + final boolean forcePrimary, + boolean skipTx, + @Nullable UUID subjId, + final String taskName, + final boolean deserializeBinary, + final boolean skipVals, + final boolean canRemap, + final boolean needVer, + boolean asyncOp ) { ctx.checkSecurity(SecurityPermission.CACHE_READ); @@ -562,7 +619,7 @@ public void near(GridNearAtomicCache near) { CacheOperationContext opCtx = ctx.operationContextPerCall(); - subjId = ctx.subjectIdPerCall(null, opCtx); + subjId = ctx.subjectIdPerCall(subjId, opCtx); final UUID subjId0 = subjId; @@ -570,44 +627,77 @@ public void near(GridNearAtomicCache near) { final boolean skipStore = opCtx != null && opCtx.skipStore(); - return asyncOp(new CO>>() { - @Override public IgniteInternalFuture> apply() { - return getAllAsync0(ctx.cacheKeysView(keys), - forcePrimary, - subjId0, - taskName, - deserializeBinary, - expiryPlc, - skipVals, - skipStore, - canRemap, - needVer); - } - }); + if (asyncOp) { + return asyncOp(new CO>>() { + @Override public IgniteInternalFuture> apply() { + return getAllAsync0(ctx.cacheKeysView(keys), + forcePrimary, + subjId0, + taskName, + deserializeBinary, + expiryPlc, + skipVals, + skipStore, + canRemap, + needVer); + } + }); + } + else { + return getAllAsync0(ctx.cacheKeysView(keys), + forcePrimary, + subjId0, + taskName, + deserializeBinary, + expiryPlc, + skipVals, + skipStore, + canRemap, + needVer); + } } /** {@inheritDoc} */ - @Override public V getAndPut(K key, V val, @Nullable CacheEntryPredicate filter) throws IgniteCheckedException { - return getAndPutAsync0(key, val, filter).get(); + @Override protected V getAndPut0(K key, V val, @Nullable CacheEntryPredicate filter) throws IgniteCheckedException { + return (V)update0( + key, + val, + null, + null, + true, + filter, + true, + false).get(); } /** {@inheritDoc} */ - @Override public boolean put(K key, V val, CacheEntryPredicate filter) throws IgniteCheckedException { - return putAsync(key, val, filter).get(); + @Override protected boolean put0(K key, V val, CacheEntryPredicate filter) throws IgniteCheckedException { + Boolean res = (Boolean)update0( + key, + val, + null, + null, + false, + filter, + true, + false).get(); + + assert res != null; + + return res; } /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public IgniteInternalFuture getAndPutAsync0(K key, V val, @Nullable CacheEntryPredicate filter) { - A.notNull(key, "key", val, "val"); - - return updateAsync0( + return update0( key, val, null, null, true, filter, + true, true); } @@ -616,13 +706,14 @@ public void near(GridNearAtomicCache near) { @Override public IgniteInternalFuture putAsync0(K key, V val, @Nullable CacheEntryPredicate filter) { A.notNull(key, "key", val, "val"); - return updateAsync0( + return update0( key, val, null, null, false, filter, + true, true); } @@ -630,84 +721,34 @@ public void near(GridNearAtomicCache near) { @Override public V tryGetAndPut(K key, V val) throws IgniteCheckedException { A.notNull(key, "key", val, "val"); - return (V)updateAsync0( + return (V) update0( key, val, null, null, true, null, + false, false).get(); } /** {@inheritDoc} */ - @Override public V getAndPutIfAbsent(K key, V val) throws IgniteCheckedException { - return getAndPutIfAbsentAsync(key, val).get(); - } - - /** {@inheritDoc} */ - @Override public IgniteInternalFuture getAndPutIfAbsentAsync(K key, V val) { - A.notNull(key, "key", val, "val"); - - return getAndPutAsync(key, val, ctx.noVal()); - } - - /** {@inheritDoc} */ - @Override public boolean putIfAbsent(K key, V val) throws IgniteCheckedException { - return putIfAbsentAsync(key, val).get(); - } - - /** {@inheritDoc} */ - @Override public IgniteInternalFuture putIfAbsentAsync(K key, V val) { - A.notNull(key, "key", val, "val"); - - return putAsync(key, val, ctx.noVal()); - } - - /** {@inheritDoc} */ - @Override public V getAndReplace(K key, V val) throws IgniteCheckedException { - return getAndReplaceAsync(key, val).get(); - } - - /** {@inheritDoc} */ - @Override public IgniteInternalFuture getAndReplaceAsync(K key, V val) { - A.notNull(key, "key", val, "val"); - - return getAndPutAsync(key, val, ctx.hasVal()); - } - - /** {@inheritDoc} */ - @Override public boolean replace(K key, V val) throws IgniteCheckedException { - return replaceAsync(key, val).get(); - } - - /** {@inheritDoc} */ - @Override public IgniteInternalFuture replaceAsync(K key, V val) { - A.notNull(key, "key", val, "val"); - - return putAsync(key, val, ctx.hasVal()); - } - - /** {@inheritDoc} */ - @Override public boolean replace(K key, V oldVal, V newVal) throws IgniteCheckedException { - return replaceAsync(key, oldVal, newVal).get(); - } - - /** {@inheritDoc} */ - @Override public IgniteInternalFuture replaceAsync(K key, V oldVal, V newVal) { - A.notNull(key, "key", oldVal, "oldVal", newVal, "newVal"); - - return putAsync(key, newVal, ctx.equalsVal(oldVal)); - } - - /** {@inheritDoc} */ - @Override public void putAll(Map m) throws IgniteCheckedException { - putAllAsync(m).get(); + @Override protected void putAll0(Map m) throws IgniteCheckedException { + updateAll0(m, + null, + null, + null, + null, + false, + false, + true, + UPDATE, + false).get(); } /** {@inheritDoc} */ - @Override public IgniteInternalFuture putAllAsync(Map m) { - return updateAllAsync0(m, + @Override public IgniteInternalFuture putAllAsync0(Map m) { + return updateAll0(m, null, null, null, @@ -715,7 +756,8 @@ public void near(GridNearAtomicCache near) { false, false, true, - UPDATE).chain(RET2NULL); + UPDATE, + true).chain(RET2NULL); } /** {@inheritDoc} */ @@ -728,7 +770,7 @@ public void near(GridNearAtomicCache near) { @Override public IgniteInternalFuture putAllConflictAsync(Map conflictMap) { ctx.dr().onReceiveCacheEntriesReceived(conflictMap.size()); - return updateAllAsync0(null, + return updateAll0(null, null, null, conflictMap, @@ -736,57 +778,40 @@ public void near(GridNearAtomicCache near) { false, false, true, - UPDATE); + UPDATE, + true); } /** {@inheritDoc} */ - @Override public V getAndRemove(K key) throws IgniteCheckedException { - return getAndRemoveAsync(key).get(); + @Override public V getAndRemove0(K key) throws IgniteCheckedException { + return (V)remove0(key, true, null, false).get(); } /** {@inheritDoc} */ @SuppressWarnings("unchecked") - @Override public IgniteInternalFuture getAndRemoveAsync(K key) { - A.notNull(key, "key"); - - return removeAsync0(key, true, null); + @Override public IgniteInternalFuture getAndRemoveAsync0(K key) { + return remove0(key, true, null, true); } /** {@inheritDoc} */ - @Override public void removeAll(Collection keys) throws IgniteCheckedException { - removeAllAsync(keys).get(); + @Override protected void removeAll0(Collection keys) throws IgniteCheckedException { + removeAllAsync0(keys, null, false, false, false).get(); } /** {@inheritDoc} */ - @Override public IgniteInternalFuture removeAllAsync(Collection keys) { - A.notNull(keys, "keys"); - - return removeAllAsync0(keys, null, false, false).chain(RET2NULL); + @Override public IgniteInternalFuture removeAllAsync0(Collection keys) { + return removeAllAsync0(keys, null, false, false, true).chain(RET2NULL); } /** {@inheritDoc} */ - @Override public boolean remove(K key) throws IgniteCheckedException { - return removeAsync(key, (CacheEntryPredicate)null).get(); + @Override protected boolean remove0(K key, CacheEntryPredicate filter) throws IgniteCheckedException { + return (Boolean)remove0(key, false, filter, false).get(); } /** {@inheritDoc} */ @SuppressWarnings("unchecked") - @Override public IgniteInternalFuture removeAsync(K key, @Nullable CacheEntryPredicate filter) { - A.notNull(key, "key"); - - return removeAsync0(key, false, filter); - } - - /** {@inheritDoc} */ - @Override public boolean remove(K key, V val) throws IgniteCheckedException { - return removeAsync(key, val).get(); - } - - /** {@inheritDoc} */ - @Override public IgniteInternalFuture removeAsync(K key, V val) { - A.notNull(key, "key", val, "val"); - - return removeAsync(key, ctx.equalsVal(val)); + @Override public IgniteInternalFuture removeAsync0(K key, @Nullable CacheEntryPredicate filter) { + return remove0(key, false, filter, true); } /** {@inheritDoc} */ @@ -799,7 +824,7 @@ public void near(GridNearAtomicCache near) { @Override public IgniteInternalFuture removeAllConflictAsync(Map conflictMap) { ctx.dr().onReceiveCacheEntriesReceived(conflictMap.size()); - return removeAllAsync0(null, conflictMap, false, false); + return removeAllAsync0(null, conflictMap, false, false, true); } /** @@ -814,7 +839,7 @@ private boolean writeThrough() { * @return Future. */ @SuppressWarnings("unchecked") - protected IgniteInternalFuture asyncOp(final CO> op) { + private IgniteInternalFuture asyncOp(final CO> op) { IgniteInternalFuture fail = asyncOpAcquire(); if (fail != null) @@ -872,7 +897,7 @@ protected IgniteInternalFuture asyncOp(final CO> /** {@inheritDoc} */ @Override public EntryProcessorResult invoke(K key, EntryProcessor entryProcessor, Object... args) throws IgniteCheckedException { - IgniteInternalFuture> invokeFut = invokeAsync(key, entryProcessor, args); + IgniteInternalFuture> invokeFut = invoke0(false, key, entryProcessor, args); EntryProcessorResult res = invokeFut.get(); @@ -882,14 +907,28 @@ protected IgniteInternalFuture asyncOp(final CO> /** {@inheritDoc} */ @Override public Map> invokeAll(Set keys, EntryProcessor entryProcessor, - Object... args) - throws IgniteCheckedException { - return invokeAllAsync(keys, entryProcessor, args).get(); + Object... args) throws IgniteCheckedException + { + return invokeAll0(false, keys, entryProcessor, args).get(); } /** {@inheritDoc} */ - @SuppressWarnings("unchecked") @Override public IgniteInternalFuture> invokeAsync(K key, + EntryProcessor entryProcessor, + Object... args) { + return invoke0(true, key, entryProcessor, args); + } + + /** + * @param async Async operation flag. + * @param key Key. + * @param entryProcessor Entry processor. + * @param args Entry processor arguments. + * @return Future. + */ + private IgniteInternalFuture> invoke0( + boolean async, + K key, EntryProcessor entryProcessor, Object... args) { A.notNull(key, "key", entryProcessor, "entryProcessor"); @@ -901,14 +940,15 @@ protected IgniteInternalFuture asyncOp(final CO> final boolean keepBinary = opCtx != null && opCtx.isKeepBinary(); - IgniteInternalFuture>> fut = updateAsync0( + IgniteInternalFuture>> fut = update0( key, null, entryProcessor, args, false, null, - true); + true, + async); return fut.chain(new CX1>>, EntryProcessorResult>() { @Override public EntryProcessorResult applyx(IgniteInternalFuture>> fut) @@ -939,6 +979,21 @@ protected IgniteInternalFuture asyncOp(final CO> /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public IgniteInternalFuture>> invokeAllAsync(Set keys, + final EntryProcessor entryProcessor, + Object... args) { + return invokeAll0(true, keys, entryProcessor, args); + } + + /** + * @param async Async operation flag. + * @param keys Keys. + * @param entryProcessor Entry processor. + * @param args Entry processor arguments. + * @return Future. + */ + private IgniteInternalFuture>> invokeAll0( + boolean async, + Set keys, final EntryProcessor entryProcessor, Object... args) { A.notNull(keys, "keys", entryProcessor, "entryProcessor"); @@ -956,7 +1011,7 @@ protected IgniteInternalFuture asyncOp(final CO> final boolean keepBinary = opCtx != null && opCtx.isKeepBinary(); - IgniteInternalFuture>> resFut = updateAllAsync0(null, + IgniteInternalFuture>> resFut = updateAll0(null, invokeMap, args, null, @@ -964,7 +1019,8 @@ protected IgniteInternalFuture asyncOp(final CO> false, false, true, - TRANSFORM); + TRANSFORM, + async); return resFut.chain( new CX1>>, Map>>() { @@ -982,7 +1038,21 @@ protected IgniteInternalFuture asyncOp(final CO> @Override public Map> invokeAll( Map> map, Object... args) throws IgniteCheckedException { - return invokeAllAsync(map, args).get(); + A.notNull(map, "map"); + + if (keyCheck) + validateCacheKeys(map.keySet()); + + return (Map>)updateAll0(null, + map, + args, + null, + null, + false, + false, + true, + TRANSFORM, + false).get(); } /** {@inheritDoc} */ @@ -995,7 +1065,7 @@ protected IgniteInternalFuture asyncOp(final CO> if (keyCheck) validateCacheKeys(map.keySet()); - return updateAllAsync0(null, + return updateAll0(null, map, args, null, @@ -1003,7 +1073,8 @@ protected IgniteInternalFuture asyncOp(final CO> false, false, true, - TRANSFORM); + TRANSFORM, + true); } /** @@ -1017,10 +1088,11 @@ protected IgniteInternalFuture asyncOp(final CO> * @param retval Return value required flag. * @param rawRetval Return {@code GridCacheReturn} instance. * @param waitTopFut Whether to wait for topology future. + * @param async Async operation flag. * @return Completion future. */ @SuppressWarnings("ConstantConditions") - private IgniteInternalFuture updateAllAsync0( + private IgniteInternalFuture updateAll0( @Nullable Map map, @Nullable Map invokeMap, @Nullable Object[] invokeArgs, @@ -1029,7 +1101,8 @@ private IgniteInternalFuture updateAllAsync0( final boolean retval, final boolean rawRetval, final boolean waitTopFut, - final GridCacheOperation op + final GridCacheOperation op, + boolean async ) { assert ctx.updatesAllowed(); @@ -1106,13 +1179,20 @@ else if (op == GridCacheOperation.DELETE) { opCtx != null && opCtx.noRetries() ? 1 : MAX_RETRIES, waitTopFut); - return asyncOp(new CO>() { - @Override public IgniteInternalFuture apply() { - updateFut.map(); + if (async) { + return asyncOp(new CO>() { + @Override public IgniteInternalFuture apply() { + updateFut.map(); - return updateFut; - } - }); + return updateFut; + } + }); + } + else { + updateFut.map(); + + return updateFut; + } } /** @@ -1125,16 +1205,18 @@ else if (op == GridCacheOperation.DELETE) { * @param retval Return value flag. * @param filter Filter. * @param waitTopFut Whether to wait for topology future. + * @param async Async operation flag. * @return Future. */ - private IgniteInternalFuture updateAsync0( + private IgniteInternalFuture update0( K key, @Nullable V val, @Nullable EntryProcessor proc, @Nullable Object[] invokeArgs, final boolean retval, @Nullable final CacheEntryPredicate filter, - final boolean waitTopFut + final boolean waitTopFut, + boolean async ) { assert val == null || proc == null; @@ -1147,13 +1229,20 @@ private IgniteInternalFuture updateAsync0( final GridNearAtomicAbstractUpdateFuture updateFut = createSingleUpdateFuture(key, val, proc, invokeArgs, retval, filter, waitTopFut); - return asyncOp(new CO>() { - @Override public IgniteInternalFuture apply() { - updateFut.map(); + if (async) { + return asyncOp(new CO>() { + @Override public IgniteInternalFuture apply() { + updateFut.map(); - return updateFut; - } - }); + return updateFut; + } + }); + } + else { + updateFut.map(); + + return updateFut; + } } /** @@ -1162,33 +1251,38 @@ private IgniteInternalFuture updateAsync0( * @param key Key. * @param retval Whether to return * @param filter Filter. + * @param async Async operation flag. * @return Future. */ - private IgniteInternalFuture removeAsync0(K key, final boolean retval, - @Nullable CacheEntryPredicate filter) { - final boolean statsEnabled = ctx.config().isStatisticsEnabled(); - - final long start = statsEnabled ? System.nanoTime() : 0L; - + private IgniteInternalFuture remove0(K key, final boolean retval, + @Nullable CacheEntryPredicate filter, + boolean async) { assert ctx.updatesAllowed(); - validateCacheKey(key); - ctx.checkSecurity(SecurityPermission.CACHE_REMOVE); - final GridNearAtomicAbstractUpdateFuture updateFut = - createSingleUpdateFuture(key, null, null, null, retval, filter, true); + final GridNearAtomicAbstractUpdateFuture updateFut = createSingleUpdateFuture(key, + null, + null, + null, + retval, + filter, + true); - if (statsEnabled) - updateFut.listen(new UpdateRemoveTimeStatClosure<>(metrics0(), start)); + if (async) { + return asyncOp(new CO>() { + @Override public IgniteInternalFuture apply() { + updateFut.map(); - return asyncOp(new CO>() { - @Override public IgniteInternalFuture apply() { - updateFut.map(); + return updateFut; + } + }); + } + else { + updateFut.map(); - return updateFut; - } - }); + return updateFut; + } } /** @@ -1327,14 +1421,11 @@ private IgniteInternalFuture removeAllAsync0( @Nullable Collection keys, @Nullable Map conflictMap, final boolean retval, - boolean rawRetval + boolean rawRetval, + boolean async ) { assert ctx.updatesAllowed(); - final boolean statsEnabled = ctx.config().isStatisticsEnabled(); - - final long start = statsEnabled ? System.nanoTime() : 0L; - assert keys != null || conflictMap != null; if (keyCheck) @@ -1381,16 +1472,20 @@ private IgniteInternalFuture removeAllAsync0( opCtx != null && opCtx.noRetries() ? 1 : MAX_RETRIES, true); - if (statsEnabled) - updateFut.listen(new UpdateRemoveTimeStatClosure<>(metrics0(), start)); + if (async) { + return asyncOp(new CO>() { + @Override public IgniteInternalFuture apply() { + updateFut.map(); - return asyncOp(new CO>() { - @Override public IgniteInternalFuture apply() { - updateFut.map(); + return updateFut; + } + }); + } + else { + updateFut.map(); - return updateFut; - } - }); + return updateFut; + } } /** @@ -3252,7 +3347,7 @@ && writeThrough() && !req.skipStore(), * @param ver Version to ack. */ private void sendDeferredUpdateResponse(UUID nodeId, GridCacheVersion ver) { - deferredUpdateMessageSender.sendDeferredAckMessage(nodeId, ver); + deferredUpdateMsgSnd.sendDeferredAckMessage(nodeId, ver); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java index ccdc51d2f6139..c9fc983d4850c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java @@ -328,19 +328,6 @@ public GridDistributedCacheEntry entryExx( needVer); } - /** {@inheritDoc} */ - @Override protected GridCacheEntryEx entryExSafe( - KeyCacheObject key, - AffinityTopologyVersion topVer - ) { - try { - return ctx.affinity().localNode(key, topVer) ? entryEx(key) : null; - } - catch (GridDhtInvalidPartitionException ignored) { - return null; - } - } - /** * @param keys Keys to load. * @param readThrough Read through flag. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java index d1056fdff4284..b843e4e6d3db1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java @@ -449,61 +449,11 @@ public void processDhtAtomicUpdateRequest( return dht.putAsync0(key, val, filter); } - /** {@inheritDoc} */ - @Override public V getAndPutIfAbsent(K key, V val) throws IgniteCheckedException { - return dht.getAndPutIfAbsent(key, val); - } - - /** {@inheritDoc} */ - @Override public IgniteInternalFuture getAndPutIfAbsentAsync(K key, V val) { - return dht.getAndPutIfAbsentAsync(key, val); - } - - /** {@inheritDoc} */ - @Override public boolean putIfAbsent(K key, V val) throws IgniteCheckedException { - return dht.putIfAbsent(key, val); - } - - /** {@inheritDoc} */ - @Override public IgniteInternalFuture putIfAbsentAsync(K key, V val) { - return dht.putIfAbsentAsync(key, val); - } - /** {@inheritDoc} */ @Nullable @Override public V tryGetAndPut(K key, V val) throws IgniteCheckedException { return dht.tryGetAndPut(key, val); } - /** {@inheritDoc} */ - @Override public V getAndReplace(K key, V val) throws IgniteCheckedException { - return dht.getAndReplace(key, val); - } - - /** {@inheritDoc} */ - @Override public IgniteInternalFuture getAndReplaceAsync(K key, V val) { - return dht.getAndReplaceAsync(key, val); - } - - /** {@inheritDoc} */ - @Override public boolean replace(K key, V val) throws IgniteCheckedException { - return dht.replace(key, val); - } - - /** {@inheritDoc} */ - @Override public IgniteInternalFuture replaceAsync(K key, V val) { - return dht.replaceAsync(key, val); - } - - /** {@inheritDoc} */ - @Override public boolean replace(K key, V oldVal, V newVal) throws IgniteCheckedException { - return dht.replace(key, oldVal, newVal); - } - - /** {@inheritDoc} */ - @Override public IgniteInternalFuture replaceAsync(K key, V oldVal, V newVal) { - return dht.replaceAsync(key, oldVal, newVal); - } - /** {@inheritDoc} */ @Override public void putAll(Map m) throws IgniteCheckedException { @@ -568,6 +518,11 @@ public void processDhtAtomicUpdateRequest( return dht.invokeAllAsync(keys, entryProcessor, args); } + /** {@inheritDoc} */ + @Override public boolean remove(K key, @Nullable CacheEntryPredicate filter) throws IgniteCheckedException { + return dht.remove(key, filter); + } + /** {@inheritDoc} */ @Override public V getAndRemove(K key) throws IgniteCheckedException { return dht.getAndRemove(key); @@ -601,16 +556,6 @@ public void processDhtAtomicUpdateRequest( return dht.removeAsync(key, filter); } - /** {@inheritDoc} */ - @Override public boolean remove(K key, V val) throws IgniteCheckedException { - return dht.remove(key, val); - } - - /** {@inheritDoc} */ - @Override public IgniteInternalFuture removeAsync(K key, V val) { - return dht.removeAsync(key, val); - } - /** {@inheritDoc} */ @Override public void removeAll() throws IgniteCheckedException { dht.removeAll(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java index 7da11b6b870be..f86df2f29d684 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java @@ -107,6 +107,11 @@ public GridLocalAtomicCache(GridCacheContext ctx) { preldr = new GridCachePreloaderAdapter(ctx); } + /** {@inheritDoc} */ + @Override protected void checkJta() throws IgniteCheckedException { + // No-op. + } + /** {@inheritDoc} */ @Override public boolean isLocal() { return true; @@ -119,9 +124,7 @@ public GridLocalAtomicCache(GridCacheContext ctx) { /** {@inheritDoc} */ @SuppressWarnings("unchecked") - @Override public V getAndPut(K key, V val, @Nullable CacheEntryPredicate filter) throws IgniteCheckedException { - A.notNull(key, "key", val, "val"); - + @Override protected V getAndPut0(K key, V val, @Nullable CacheEntryPredicate filter) throws IgniteCheckedException { CacheOperationContext opCtx = ctx.operationContextPerCall(); return (V)updateAllInternal(UPDATE, @@ -138,16 +141,10 @@ public GridLocalAtomicCache(GridCacheContext ctx) { } /** {@inheritDoc} */ - @Override public boolean put(K key, V val, CacheEntryPredicate filter) throws IgniteCheckedException { - A.notNull(key, "key", val, "val"); - - boolean statsEnabled = ctx.config().isStatisticsEnabled(); - - long start = statsEnabled ? System.nanoTime() : 0L; - + @Override protected boolean put0(K key, V val, CacheEntryPredicate filter) throws IgniteCheckedException { CacheOperationContext opCtx = ctx.operationContextPerCall(); - boolean res = (Boolean)updateAllInternal(UPDATE, + Boolean res = (Boolean)updateAllInternal(UPDATE, Collections.singleton(key), Collections.singleton(val), null, @@ -159,8 +156,7 @@ public GridLocalAtomicCache(GridCacheContext ctx) { ctx.readThrough(), opCtx != null && opCtx.isKeepBinary()); - if (statsEnabled) - metrics0().addPutTimeNanos(System.nanoTime() - start); + assert res != null; return res; } @@ -168,8 +164,6 @@ public GridLocalAtomicCache(GridCacheContext ctx) { /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public IgniteInternalFuture getAndPutAsync0(K key, V val, @Nullable CacheEntryPredicate filter) { - A.notNull(key, "key", val, "val"); - return updateAllAsync0(F0.asMap(key, val), null, null, @@ -181,8 +175,6 @@ public GridLocalAtomicCache(GridCacheContext ctx) { /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public IgniteInternalFuture putAsync0(K key, V val, @Nullable CacheEntryPredicate filter) { - A.notNull(key, "key", val, "val"); - return updateAllAsync0(F0.asMap(key, val), null, null, @@ -192,65 +184,7 @@ public GridLocalAtomicCache(GridCacheContext ctx) { } /** {@inheritDoc} */ - @SuppressWarnings("unchecked") - @Override public V getAndPutIfAbsent(K key, V val) throws IgniteCheckedException { - return getAndPut(key, val, ctx.noVal()); - } - - /** {@inheritDoc} */ - @Override public IgniteInternalFuture getAndPutIfAbsentAsync(K key, V val) { - return getAndPutAsync(key, val, ctx.noVal()); - } - - /** {@inheritDoc} */ - @Override public boolean putIfAbsent(K key, V val) throws IgniteCheckedException { - return put(key, val, ctx.noVal()); - } - - /** {@inheritDoc} */ - @Override public IgniteInternalFuture putIfAbsentAsync(K key, V val) { - return putAsync(key, val, ctx.noVal()); - } - - /** {@inheritDoc} */ - @SuppressWarnings("unchecked") - @Override public V getAndReplace(K key, V val) throws IgniteCheckedException { - return getAndPut(key, val, ctx.hasVal()); - } - - /** {@inheritDoc} */ - @Override public IgniteInternalFuture getAndReplaceAsync(K key, V val) { - return getAndPutAsync(key, val, ctx.hasVal()); - } - - /** {@inheritDoc} */ - @Override public boolean replace(K key, V val) throws IgniteCheckedException { - return put(key, val, ctx.hasVal()); - } - - /** {@inheritDoc} */ - @Override public IgniteInternalFuture replaceAsync(K key, V val) { - return putAsync(key, val, ctx.hasVal()); - } - - /** {@inheritDoc} */ - @Override public boolean replace(K key, V oldVal, V newVal) throws IgniteCheckedException { - A.notNull(oldVal, "oldVal"); - - return put(key, newVal, ctx.equalsVal(oldVal)); - } - - /** {@inheritDoc} */ - @Override public IgniteInternalFuture replaceAsync(K key, V oldVal, V newVal) { - return putAsync(key, newVal, ctx.equalsVal(oldVal)); - } - - /** {@inheritDoc} */ - @Override public void putAll(Map m) throws IgniteCheckedException { - boolean statsEnabled = ctx.config().isStatisticsEnabled(); - - long start = statsEnabled ? System.nanoTime() : 0L; - + @Override protected void putAll0(Map m) throws IgniteCheckedException { CacheOperationContext opCtx = ctx.operationContextPerCall(); updateAllInternal(UPDATE, @@ -264,13 +198,10 @@ public GridLocalAtomicCache(GridCacheContext ctx) { ctx.writeThrough(), ctx.readThrough(), opCtx != null && opCtx.isKeepBinary()); - - if (statsEnabled) - metrics0().addPutTimeNanos(System.nanoTime() - start); } /** {@inheritDoc} */ - @Override public IgniteInternalFuture putAllAsync(Map m) { + @Override public IgniteInternalFuture putAllAsync0(Map m) { return updateAllAsync0(m, null, null, @@ -280,8 +211,7 @@ public GridLocalAtomicCache(GridCacheContext ctx) { } /** {@inheritDoc} */ - @SuppressWarnings("unchecked") - @Override public V getAndRemove(K key) throws IgniteCheckedException { + @Override protected V getAndRemove0(K key) throws IgniteCheckedException { CacheOperationContext opCtx = ctx.operationContextPerCall(); return (V)updateAllInternal(DELETE, @@ -299,13 +229,13 @@ public GridLocalAtomicCache(GridCacheContext ctx) { /** {@inheritDoc} */ @SuppressWarnings("unchecked") - @Override public IgniteInternalFuture getAndRemoveAsync(K key) { + @Override public IgniteInternalFuture getAndRemoveAsync0(K key) { return removeAllAsync0(Collections.singletonList(key), true, false, null); } /** {@inheritDoc} */ @SuppressWarnings("unchecked") - @Override public void removeAll(Collection keys) throws IgniteCheckedException { + @Override public void removeAll0(Collection keys) throws IgniteCheckedException { CacheOperationContext opCtx = ctx.operationContextPerCall(); updateAllInternal(DELETE, @@ -322,19 +252,13 @@ public GridLocalAtomicCache(GridCacheContext ctx) { } /** {@inheritDoc} */ - @Override public IgniteInternalFuture removeAllAsync(Collection keys) { + @Override public IgniteInternalFuture removeAllAsync0(Collection keys) { return removeAllAsync0(keys, false, false, null).chain(RET2NULL); } /** {@inheritDoc} */ @SuppressWarnings("unchecked") - @Override public boolean remove(K key) throws IgniteCheckedException { - boolean statsEnabled = ctx.config().isStatisticsEnabled(); - - long start = statsEnabled ? System.nanoTime() : 0L; - - A.notNull(key, "key"); - + @Override public boolean remove0(K key, final CacheEntryPredicate filter) throws IgniteCheckedException { CacheOperationContext opCtx = ctx.operationContextPerCall(); Boolean rmv = (Boolean)updateAllInternal(DELETE, @@ -344,49 +268,22 @@ public GridLocalAtomicCache(GridCacheContext ctx) { expiryPerCall(), false, false, - null, + filter, ctx.writeThrough(), ctx.readThrough(), opCtx != null && opCtx.isKeepBinary()); - if (statsEnabled && rmv) - metrics0().addRemoveTimeNanos(System.nanoTime() - start); + assert rmv != null; return rmv; } /** {@inheritDoc} */ @SuppressWarnings("unchecked") - @Override public IgniteInternalFuture removeAsync(K key, @Nullable CacheEntryPredicate filter) { - A.notNull(key, "key"); - + @Override public IgniteInternalFuture removeAsync0(K key, @Nullable CacheEntryPredicate filter) { return removeAllAsync0(Collections.singletonList(key), false, false, filter); } - /** {@inheritDoc} */ - @Override public boolean remove(K key, V val) throws IgniteCheckedException { - A.notNull(key, "key", val, "val"); - - CacheOperationContext opCtx = ctx.operationContextPerCall(); - - return (Boolean)updateAllInternal(DELETE, - Collections.singleton(key), - null, - null, - expiryPerCall(), - false, - false, - ctx.equalsVal(val), - ctx.writeThrough(), - ctx.readThrough(), - opCtx != null && opCtx.isKeepBinary()); - } - - /** {@inheritDoc} */ - @Override public IgniteInternalFuture removeAsync(K key, V val) { - return removeAsync(key, ctx.equalsVal(val)); - } - /** {@inheritDoc} */ @Override public IgniteInternalFuture removeAllAsync() { return ctx.closures().callLocalSafe(new Callable() { @@ -399,11 +296,13 @@ public GridLocalAtomicCache(GridCacheContext ctx) { } /** {@inheritDoc} */ - @SuppressWarnings("unchecked") - @Override @Nullable public V get(K key, boolean deserializeBinary, boolean needVer) throws IgniteCheckedException { - String taskName = ctx.kernalContext().job().currentTaskName(); - + @Override protected V get0( + final K key, + String taskName, + boolean deserializeBinary, + boolean needVer) throws IgniteCheckedException + { Map m = getAllInternal(Collections.singleton(key), ctx.isSwapOrOffheapEnabled(), ctx.readThrough(), @@ -419,7 +318,7 @@ public GridLocalAtomicCache(GridCacheContext ctx) { /** {@inheritDoc} */ @SuppressWarnings("unchecked") - @Override public final Map getAll(Collection keys, boolean deserializeBinary, boolean needVer) + @Override public final Map getAll0(Collection keys, boolean deserializeBinary, boolean needVer) throws IgniteCheckedException { A.notNull(keys, "keys"); @@ -795,7 +694,7 @@ private IgniteInternalFuture updateAllAsync0( final boolean keepBinary = opCtx != null && opCtx.isKeepBinary(); - IgniteInternalFuture fut = asyncOp(new Callable() { + return asyncOp(new Callable() { @Override public Object call() throws Exception { return updateAllInternal(op, keys, @@ -810,11 +709,6 @@ private IgniteInternalFuture updateAllAsync0( keepBinary); } }); - - if (ctx.config().isStatisticsEnabled()) - fut.listen(new UpdatePutTimeStatClosure(metrics0(), System.nanoTime())); - - return fut; } /** @@ -836,17 +730,13 @@ private IgniteInternalFuture removeAllAsync0( final boolean readThrough = ctx.readThrough(); - final boolean statsEnabled = ctx.config().isStatisticsEnabled(); - - final long start = statsEnabled ? System.nanoTime() : 0L; - final ExpiryPolicy expiryPlc = expiryPerCall(); CacheOperationContext opCtx = ctx.operationContextPerCall(); final boolean keepBinary = opCtx != null && opCtx.isKeepBinary(); - IgniteInternalFuture fut = asyncOp(new Callable() { + return asyncOp(new Callable() { @Override public Object call() throws Exception { return updateAllInternal(DELETE, keys, @@ -861,11 +751,6 @@ private IgniteInternalFuture removeAllAsync0( keepBinary); } }); - - if (statsEnabled) - fut.listen(new UpdateRemoveTimeStatClosure<>(metrics0(), start)); - - return fut; } /** @@ -1586,7 +1471,7 @@ private static CachePartialUpdateCheckedException partialUpdateException() { * @return Future. */ @SuppressWarnings("unchecked") - protected IgniteInternalFuture asyncOp(final Callable op) { + private IgniteInternalFuture asyncOp(final Callable op) { IgniteInternalFuture fail = asyncOpAcquire(); if (fail != null) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/CachePartitionedNearEnabledMultiNodeLongTxTimeoutFullApiTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/CachePartitionedNearEnabledMultiNodeLongTxTimeoutFullApiTest.java index 3e3b84e4403bf..648134e498c85 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/CachePartitionedNearEnabledMultiNodeLongTxTimeoutFullApiTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/CachePartitionedNearEnabledMultiNodeLongTxTimeoutFullApiTest.java @@ -34,7 +34,7 @@ public class CachePartitionedNearEnabledMultiNodeLongTxTimeoutFullApiTest extend @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(gridName); - cfg.getTransactionConfiguration().setDefaultTxTimeout(Long.MAX_VALUE); + cfg.getTransactionConfiguration().setDefaultTxTimeout(5 * 60_000); return cfg; } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBlockMessageSystemPoolStarvationSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBlockMessageSystemPoolStarvationSelfTest.java index ec3b8089cae6e..a235d8f15e3f6 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBlockMessageSystemPoolStarvationSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBlockMessageSystemPoolStarvationSelfTest.java @@ -17,6 +17,8 @@ package org.apache.ignite.internal.processors.igfs; +import java.util.concurrent.Callable; +import java.util.concurrent.CountDownLatch; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteFileSystem; @@ -39,14 +41,11 @@ import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; import org.apache.ignite.testframework.GridTestUtils; -import org.apache.ignite.transactions.TransactionConcurrency; -import org.apache.ignite.transactions.TransactionIsolation; - -import java.util.concurrent.Callable; -import java.util.concurrent.CountDownLatch; import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; +import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; +import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ; /** * Test to check for system pool starvation due to {@link IgfsBlocksMessage}. @@ -125,8 +124,7 @@ public void testStarvation() throws Exception { @Override public Void call() throws Exception { GridCacheAdapter dataCache = dataCache(attacker); - try (IgniteInternalTx tx = - dataCache.txStartEx(TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ)) { + try (IgniteInternalTx tx = dataCache.txStartEx(PESSIMISTIC, REPEATABLE_READ)) { dataCache.put(DATA_KEY, 0); txStartLatch.countDown(); @@ -185,6 +183,7 @@ private static boolean awaitFuture(final IgniteInternalFuture fut) throws Except * Create IGFS file asynchronously. * * @param path Path. + * @param writeStartLatch Write start latch. * @return Future. */ private IgniteInternalFuture createFileAsync(final IgfsPath path, final CountDownLatch writeStartLatch) { From 0fc6271d8e39125bf5ee341e50a2665a04fc8b1e Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Wed, 21 Jun 2017 13:42:12 +0300 Subject: [PATCH 33/90] GG-12350: GridDhtAtomicSingleUpdateRequest misses topologyVersion() method override. --- .../dht/atomic/GridDhtAtomicSingleUpdateRequest.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateRequest.java index a03d948017f91..0838cb6dd87b6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateRequest.java @@ -360,6 +360,11 @@ public GridDhtAtomicSingleUpdateRequest() { return isFlag(KEEP_BINARY_FLAG_MASK); } + /** {@inheritDoc} */ + @Override public AffinityTopologyVersion topologyVersion() { + return topVer; + } + /** * */ From 113a1380da34ea804d68757d39926da97dee09b6 Mon Sep 17 00:00:00 2001 From: Alexey Goncharuk Date: Tue, 13 Jun 2017 08:20:22 +0300 Subject: [PATCH 34/90] GG-12355: Backported IO latency test. --- .../org/apache/ignite/internal/GridTopic.java | 5 +- .../apache/ignite/internal/IgniteKernal.java | 16 +- .../managers/communication/GridIoManager.java | 699 +++++++++++++++++- .../communication/GridIoMessageFactory.java | 5 + .../communication/IgniteIoTestMessage.java | 594 +++++++++++++++ .../apache/ignite/mxbean/IgniteMXBean.java | 44 ++ 6 files changed, 1358 insertions(+), 5 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessage.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 248f75b9d2689..dc20be002a6e6 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 @@ -94,7 +94,10 @@ public enum GridTopic { TOPIC_QUERY, /** */ - TOPIC_TX; + TOPIC_TX, + + /** */ + TOPIC_IO_TEST; /** Enum values. */ private static final GridTopic[] VALS = values(); 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 25f7884c889b9..4237aeebe49ce 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 @@ -3545,7 +3545,7 @@ private boolean skipDaemon(GridComponent comp) { } /** {@inheritDoc} */ - public void dumpDebugInfo() { + @Override public void dumpDebugInfo() { try { GridKernalContextImpl ctx = this.ctx; @@ -3615,6 +3615,20 @@ void waitPreviousReconnect() { } } + /** {@inheritDoc} */ + @Override public void runIoTest( + long warmup, + long duration, + int threads, + long maxLatency, + int rangesCnt, + int payLoadSize, + boolean procFromNioThread + ) { + ctx.io().runIoTest(warmup, duration, threads, maxLatency, rangesCnt, payLoadSize, procFromNioThread, + new ArrayList(ctx.cluster().get().forServers().forRemotes().nodes())); + } + /** {@inheritDoc} */ @Override public String toString() { return S.toString(IgniteKernal.class, this); 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 3df29cf8fb4e5..4b68e5bdd0da7 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 @@ -18,23 +18,33 @@ package org.apache.ignite.internal.managers.communication; import java.io.Serializable; +import java.text.SimpleDateFormat; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.Date; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.Queue; import java.util.UUID; +import java.util.concurrent.BrokenBarrierException; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; - import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.cluster.ClusterNode; @@ -44,6 +54,7 @@ import org.apache.ignite.internal.GridTopic; import org.apache.ignite.internal.IgniteComponentType; import org.apache.ignite.internal.IgniteDeploymentCheckedException; +import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.direct.DirectMessageReader; import org.apache.ignite.internal.direct.DirectMessageWriter; import org.apache.ignite.internal.managers.GridManagerAdapter; @@ -55,7 +66,10 @@ import org.apache.ignite.internal.processors.timeout.GridTimeoutObject; import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashSet; 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.lang.GridTuple3; +import org.apache.ignite.internal.util.lang.IgnitePair; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.X; @@ -78,11 +92,13 @@ import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentHashMap8; import org.jsr166.ConcurrentLinkedDeque8; +import org.jsr166.LongAdder8; 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.GridTopic.TOPIC_COMM_USER; +import static org.apache.ignite.internal.GridTopic.TOPIC_IO_TEST; import static org.apache.ignite.internal.managers.communication.GridIoPolicy.AFFINITY_POOL; import static org.apache.ignite.internal.managers.communication.GridIoPolicy.IDX_POOL; import static org.apache.ignite.internal.managers.communication.GridIoPolicy.IGFS_POOL; @@ -176,6 +192,12 @@ public class GridIoManager extends GridManagerAdapter> ioTestMap = new AtomicReference<>(); + + /** */ + private final AtomicLong ioTestId = new AtomicLong(); + /** * @param ctx Grid kernal context. */ @@ -297,6 +319,399 @@ public void resetMetrics() { if (log.isDebugEnabled()) log.debug(startInfo()); + + addMessageListener(TOPIC_IO_TEST, new GridMessageListener() { + @Override public void onMessage(UUID nodeId, Object msg) { + ClusterNode node = ctx.discovery().node(nodeId); + + if (node == null) + return; + + IgniteIoTestMessage msg0 = (IgniteIoTestMessage)msg; + + msg0.senderNodeId(nodeId); + + if (msg0.request()) { + IgniteIoTestMessage res = new IgniteIoTestMessage(msg0.id(), false, null); + + res.flags(msg0.flags()); + res.onRequestProcessed(); + + res.copyDataFromRequest(msg0); + + try { + send(node, TOPIC_IO_TEST, res, GridIoPolicy.SYSTEM_POOL); + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to send IO test response [msg=" + msg0 + "]", e); + } + } + else { + IoTestFuture fut = ioTestMap().get(msg0.id()); + + msg0.onResponseProcessed(); + + if (fut == null) + U.warn(log, "Failed to find IO test future [msg=" + msg0 + ']'); + else + fut.onResponse(msg0); + } + } + }); + } + + /** + * @param nodes Nodes. + * @param payload Payload. + * @param procFromNioThread If {@code true} message is processed from NIO thread. + * @return Response future. + */ + public IgniteInternalFuture sendIoTest(List nodes, byte[] payload, boolean procFromNioThread) { + long id = ioTestId.getAndIncrement(); + + IoTestFuture fut = new IoTestFuture(id, nodes.size()); + + IgniteIoTestMessage msg = new IgniteIoTestMessage(id, true, payload); + + msg.processFromNioThread(procFromNioThread); + + ioTestMap().put(id, fut); + + for (int i = 0; i < nodes.size(); i++) { + ClusterNode node = nodes.get(i); + + try { + send(node, TOPIC_IO_TEST, msg, GridIoPolicy.SYSTEM_POOL); + } + catch (IgniteCheckedException e) { + ioTestMap().remove(msg.id()); + + return new GridFinishedFuture(e); + } + } + + return fut; + } + + /** + * @param node Node. + * @param payload Payload. + * @param procFromNioThread If {@code true} message is processed from NIO thread. + * @return Response future. + */ + public IgniteInternalFuture> sendIoTest( + ClusterNode node, + byte[] payload, + boolean procFromNioThread + ) { + long id = ioTestId.getAndIncrement(); + + IoTestFuture fut = new IoTestFuture(id, 1); + + IgniteIoTestMessage msg = new IgniteIoTestMessage(id, true, payload); + + msg.processFromNioThread(procFromNioThread); + + ioTestMap().put(id, fut); + + try { + send(node, TOPIC_IO_TEST, msg, GridIoPolicy.SYSTEM_POOL); + } + catch (IgniteCheckedException e) { + ioTestMap().remove(msg.id()); + + return new GridFinishedFuture(e); + } + + return fut; + } + + /** + * @return IO test futures map. + */ + private ConcurrentHashMap ioTestMap() { + ConcurrentHashMap map = ioTestMap.get(); + + if (map == null) { + if (!ioTestMap.compareAndSet(null, map = new ConcurrentHashMap<>())) + map = ioTestMap.get(); + } + + return map; + } + + /** + * @param warmup Warmup duration in milliseconds. + * @param duration Test duration in milliseconds. + * @param threads Thread count. + * @param latencyLimit Max latency in nanoseconds. + * @param rangesCnt Ranges count in resulting histogram. + * @param payLoadSize Payload size in bytes. + * @param procFromNioThread {@code True} to process requests in NIO threads. + * @param nodes Nodes participating in test. + */ + public void runIoTest( + final long warmup, + final long duration, + final int threads, + final long latencyLimit, + final int rangesCnt, + final int payLoadSize, + final boolean procFromNioThread, + final List nodes + ) { + final ExecutorService svc = Executors.newFixedThreadPool(threads + 1); + + final AtomicBoolean warmupFinished = new AtomicBoolean(); + final AtomicBoolean done = new AtomicBoolean(); + final CyclicBarrier bar = new CyclicBarrier(threads + 1); + final LongAdder8 cnt = new LongAdder8(); + final long sleepDuration = 5000; + final byte[] payLoad = new byte[payLoadSize]; + final Map[] res = new Map[threads]; + + boolean failed = true; + + try { + svc.execute(new Runnable() { + @Override public void run() { + boolean failed = true; + + try { + bar.await(); + + long start = System.currentTimeMillis(); + + if (log.isInfoEnabled()) + log.info("IO test started " + + "[warmup=" + warmup + + ", duration=" + duration + + ", threads=" + threads + + ", latencyLimit=" + latencyLimit + + ", rangesCnt=" + rangesCnt + + ", payLoadSize=" + payLoadSize + + ", procFromNioThreads=" + procFromNioThread + ']' + ); + + for (;;) { + if (!warmupFinished.get() && System.currentTimeMillis() - start > warmup) { + if (log.isInfoEnabled()) + log.info("IO test warmup finished."); + + warmupFinished.set(true); + + start = System.currentTimeMillis(); + } + + if (warmupFinished.get() && System.currentTimeMillis() - start > duration) { + if (log.isInfoEnabled()) + log.info("IO test finished, will wait for all threads to finish."); + + done.set(true); + + bar.await(); + + failed = false; + + break; + } + + if (log.isInfoEnabled()) + log.info("IO test [opsCnt/sec=" + (cnt.sumThenReset() * 1000 / sleepDuration) + + ", warmup=" + !warmupFinished.get() + + ", elapsed=" + (System.currentTimeMillis() - start) + ']'); + + Thread.sleep(sleepDuration); + } + + // At this point all threads have finished the test and + // stored data to the resulting array of maps. + // Need to iterate it over and sum values for all threads. + printIoTestResults(res); + } + catch (InterruptedException | BrokenBarrierException e) { + U.error(log, "IO test failed.", e); + } + finally { + if (failed) + bar.reset(); + + svc.shutdown(); + } + } + }); + + for (int i = 0; i < threads; i++) { + final int i0 = i; + + res[i] = U.newHashMap(nodes.size()); + + svc.execute(new Runnable() { + @Override public void run() { + boolean failed = true; + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + int size = nodes.size(); + Map res0 = res[i0]; + + try { + boolean warmupFinished0 = false; + + bar.await(); + + for (;;) { + if (done.get()) + break; + + if (!warmupFinished0) + warmupFinished0 = warmupFinished.get(); + + ClusterNode node = nodes.get(rnd.nextInt(size)); + + List msgs = sendIoTest(node, payLoad, procFromNioThread).get(); + + cnt.increment(); + + for (IgniteIoTestMessage msg : msgs) { + UUID nodeId = msg.senderNodeId(); + + assert nodeId != null; + + IoTestThreadLocalNodeResults nodeRes = res0.get(nodeId); + + if (nodeRes == null) + res0.put(nodeId, + nodeRes = new IoTestThreadLocalNodeResults(rangesCnt, latencyLimit)); + + nodeRes.onResult(msg); + } + } + + bar.await(); + + failed = false; + } + catch (Exception e) { + U.error(log, "IO test worker thread failed.", e); + } + finally { + if (failed) + bar.reset(); + } + } + }); + } + + failed = false; + } + finally { + if (failed) + U.shutdownNow(GridIoManager.class, svc, log); + } + } + + /** + * @param rawRes Resulting map. + */ + private void printIoTestResults( + Map[] rawRes + ) { + Map res = new HashMap<>(); + + for (Map r : rawRes) { + for (Entry e : r.entrySet()) { + IoTestNodeResults r0 = res.get(e.getKey()); + + if (r0 == null) + res.put(e.getKey(), r0 = new IoTestNodeResults()); + + r0.add(e.getValue()); + } + } + + SimpleDateFormat dateFmt = new SimpleDateFormat("HH:mm:ss,SSS"); + + StringBuilder b = new StringBuilder(U.nl()) + .append("IO test results (round-trip count per each latency bin).") + .append(U.nl()); + + for (Entry e : res.entrySet()) { + ClusterNode node = ctx.discovery().node(e.getKey()); + + long binLatencyMcs = e.getValue().binLatencyMcs(); + + b.append("Node ID: ").append(e.getKey()).append(" (addrs=") + .append(node != null ? node.addresses().toString() : "n/a") + .append(", binLatency=").append(binLatencyMcs).append("mcs") + .append(')').append(U.nl()); + + b.append("Latency bin, mcs | Count exclusive | Percentage exclusive | " + + "Count inclusive | Percentage inclusive ").append(U.nl()); + + long[] nodeRes = e.getValue().resLatency; + + long sum = 0; + + for (int i = 0; i < nodeRes.length; i++) + sum += nodeRes[i]; + + long curSum = 0; + + for (int i = 0; i < nodeRes.length; i++) { + curSum += nodeRes[i]; + + if (i < nodeRes.length - 1) + b.append(String.format("<%11d mcs | %15d | %19.6f%% | %15d | %19.6f%%\n", + (i + 1) * binLatencyMcs, + nodeRes[i], (100.0 * nodeRes[i]) / sum, + curSum, (100.0 * curSum) / sum)); + else + b.append(String.format(">%11d mcs | %15d | %19.6f%% | %15d | %19.6f%%\n", + i * binLatencyMcs, + nodeRes[i], (100.0 * nodeRes[i]) / sum, + curSum, (100.0 * curSum) / sum)); + } + + b.append(U.nl()).append("Total latency (ns): ").append(U.nl()) + .append(String.format("%15d", e.getValue().totalLatency)).append(U.nl()); + + b.append(U.nl()).append("Max latencies (ns):").append(U.nl()); + format(b, e.getValue().maxLatency, dateFmt); + + b.append(U.nl()).append("Max request send queue times (ns):").append(U.nl()); + format(b, e.getValue().maxReqSendQueueTime, dateFmt); + + b.append(U.nl()).append("Max request receive queue times (ns):").append(U.nl()); + format(b, e.getValue().maxReqRcvQueueTime, dateFmt); + + b.append(U.nl()).append("Max response send queue times (ns):").append(U.nl()); + format(b, e.getValue().maxResSendQueueTime, dateFmt); + + b.append(U.nl()).append("Max response receive queue times (ns):").append(U.nl()); + format(b, e.getValue().maxResRcvQueueTime, dateFmt); + + b.append(U.nl()).append("Max request wire times (millis):").append(U.nl()); + format(b, e.getValue().maxReqWireTimeMillis, dateFmt); + + b.append(U.nl()).append("Max response wire times (millis):").append(U.nl()); + format(b, e.getValue().maxResWireTimeMillis, dateFmt); + + b.append(U.nl()); + } + + if (log.isInfoEnabled()) + log.info(b.toString()); + } + + /** + * @param b Builder. + * @param pairs Pairs to format. + * @param dateFmt Formatter. + */ + private void format(StringBuilder b, Collection> pairs, SimpleDateFormat dateFmt) { + for (IgnitePair p : pairs) { + b.append(String.format("%15d", p.get1())).append(" ") + .append(dateFmt.format(new Date(p.get2()))).append(U.nl()); + } } /** {@inheritDoc} */ @@ -678,8 +1093,22 @@ private void processRegularMessage( msgC.run(); } } + + @Override public String toString() { + return "Message closure [msg=" + msg + ']'; + } }; + if (msg.topicOrdinal() == TOPIC_IO_TEST.ordinal()) { + IgniteIoTestMessage msg0 = (IgniteIoTestMessage)msg.message(); + + if (msg0.processFromNioThread()) { + c.run(); + + return; + } + } + try { pools.poolForPolicy(plc).execute(c); } @@ -1796,8 +2225,7 @@ public boolean removeMessageListener(Object topic, @Nullable GridMessageListener if (rmv && log.isDebugEnabled()) log.debug("Removed message listener [topic=" + topic + ", lsnr=" + lsnr + ']'); - if (lsnr instanceof ArrayListener) - { + if (lsnr instanceof ArrayListener) { for (GridMessageListener childLsnr : ((ArrayListener)lsnr).arr) closeListener(childLsnr); } @@ -2460,4 +2888,269 @@ public UUID nodeId() { return S.toString(DelayedMessage.class, this, super.toString()); } } + + /** + * + */ + private class IoTestFuture extends GridFutureAdapter> { + /** */ + private final long id; + + /** */ + private final int cntr; + + /** */ + private final List ress; + + /** + * @param id ID. + * @param cntr Counter. + */ + IoTestFuture(long id, int cntr) { + assert cntr > 0 : cntr; + + this.id = id; + this.cntr = cntr; + + ress = new ArrayList<>(cntr); + } + + /** + * + */ + void onResponse(IgniteIoTestMessage res) { + boolean complete; + + synchronized (this) { + ress.add(res); + + complete = cntr == ress.size(); + } + + if (complete) + onDone(ress); + } + + /** {@inheritDoc} */ + @Override public boolean onDone(List res, @Nullable Throwable err) { + if (super.onDone(res, err)) { + ioTestMap().remove(id); + + return true; + } + + return false; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(IoTestFuture.class, this); + } + } + + /** + * + */ + private static class IoTestThreadLocalNodeResults { + /** */ + private final long[] resLatency; + + /** */ + private final int rangesCnt; + + /** */ + private long totalLatency; + + /** */ + private long maxLatency; + + /** */ + private long maxLatencyTs; + + /** */ + private long maxReqSendQueueTime; + + /** */ + private long maxReqSendQueueTimeTs; + + /** */ + private long maxReqRcvQueueTime; + + /** */ + private long maxReqRcvQueueTimeTs; + + /** */ + private long maxResSendQueueTime; + + /** */ + private long maxResSendQueueTimeTs; + + /** */ + private long maxResRcvQueueTime; + + /** */ + private long maxResRcvQueueTimeTs; + + /** */ + private long maxReqWireTimeMillis; + + /** */ + private long maxReqWireTimeTs; + + /** */ + private long maxResWireTimeMillis; + + /** */ + private long maxResWireTimeTs; + + /** */ + private final long latencyLimit; + + /** + * @param rangesCnt Ranges count. + * @param latencyLimit + */ + public IoTestThreadLocalNodeResults(int rangesCnt, long latencyLimit) { + this.rangesCnt = rangesCnt; + this.latencyLimit = latencyLimit; + + resLatency = new long[rangesCnt + 1]; + } + + /** + * @param msg + */ + public void onResult(IgniteIoTestMessage msg) { + long now = System.currentTimeMillis(); + + long latency = msg.responseProcessedTs() - msg.requestCreateTs(); + + int idx = latency >= latencyLimit ? + rangesCnt /* Timed out. */ : + (int)Math.floor((1.0 * latency) / ((1.0 * latencyLimit) / rangesCnt)); + + resLatency[idx]++; + + totalLatency += latency; + + if (maxLatency < latency) { + maxLatency = latency; + maxLatencyTs = now; + } + + long reqSndQueueTime = msg.requestSendTs() - msg.requestCreateTs(); + + if (maxReqSendQueueTime < reqSndQueueTime) { + maxReqSendQueueTime = reqSndQueueTime; + maxReqSendQueueTimeTs = now; + } + + long reqRcvQueueTime = msg.requestProcessTs() - msg.requestReceiveTs(); + + if (maxReqRcvQueueTime < reqRcvQueueTime) { + maxReqRcvQueueTime = reqRcvQueueTime; + maxReqRcvQueueTimeTs = now; + } + + long resSndQueueTime = msg.responseSendTs() - msg.requestProcessTs(); + + if (maxResSendQueueTime < resSndQueueTime) { + maxResSendQueueTime = resSndQueueTime; + maxResSendQueueTimeTs = now; + } + + long resRcvQueueTime = msg.responseProcessedTs() - msg.responseReceiveTs(); + + if (maxResRcvQueueTime < resRcvQueueTime) { + maxResRcvQueueTime = resRcvQueueTime; + maxResRcvQueueTimeTs = now; + } + + long reqWireTimeMillis = msg.requestReceivedTsMillis() - msg.requestSendTsMillis(); + + if (maxReqWireTimeMillis < reqWireTimeMillis) { + maxReqWireTimeMillis = reqWireTimeMillis; + maxReqWireTimeTs = now; + } + + long resWireTimeMillis = msg.responseReceivedTsMillis() - msg.requestSendTsMillis(); + + if (maxResWireTimeMillis < resWireTimeMillis) { + maxResWireTimeMillis = resWireTimeMillis; + maxResWireTimeTs = now; + } + } + } + + /** + * + */ + private static class IoTestNodeResults { + /** */ + private long latencyLimit; + + /** */ + private long[] resLatency; + + /** */ + private long totalLatency; + + /** */ + private Collection> maxLatency = new ArrayList<>(); + + /** */ + private Collection> maxReqSendQueueTime = new ArrayList<>(); + + /** */ + private Collection> maxReqRcvQueueTime = new ArrayList<>(); + + /** */ + private Collection> maxResSendQueueTime = new ArrayList<>(); + + /** */ + private Collection> maxResRcvQueueTime = new ArrayList<>(); + + /** */ + private Collection> maxReqWireTimeMillis = new ArrayList<>(); + + /** */ + private Collection> maxResWireTimeMillis = new ArrayList<>(); + + /** + * @param res Node results to add. + */ + public void add(IoTestThreadLocalNodeResults res) { + if (resLatency == null) { + resLatency = res.resLatency.clone(); + latencyLimit = res.latencyLimit; + } + else { + assert latencyLimit == res.latencyLimit; + assert resLatency.length == res.resLatency.length; + + for (int i = 0; i < resLatency.length; i++) + resLatency[i] += res.resLatency[i]; + } + + totalLatency += res.totalLatency; + + maxLatency.add(F.pair(res.maxLatency, res.maxLatencyTs)); + maxReqSendQueueTime.add(F.pair(res.maxReqSendQueueTime, res.maxReqSendQueueTimeTs)); + maxReqRcvQueueTime.add(F.pair(res.maxReqRcvQueueTime, res.maxReqRcvQueueTimeTs)); + maxResSendQueueTime.add(F.pair(res.maxResSendQueueTime, res.maxResSendQueueTimeTs)); + maxResRcvQueueTime.add(F.pair(res.maxResRcvQueueTime, res.maxResRcvQueueTimeTs)); + maxReqWireTimeMillis.add(F.pair(res.maxReqWireTimeMillis, res.maxReqWireTimeTs)); + maxResWireTimeMillis.add(F.pair(res.maxResWireTimeMillis, res.maxResWireTimeTs)); + } + + /** + * @return Bin latency in microseconds. + */ + public long binLatencyMcs() { + if (resLatency == null) + throw new IllegalStateException(); + + return latencyLimit / (1000 * (resLatency.length - 1)); + } + } } 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 f36191cbafcd9..743d5f74d2ec9 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 @@ -165,6 +165,11 @@ public GridIoMessageFactory(MessageFactory[] ext) { Message msg = null; switch (type) { + case -43: + msg = new IgniteIoTestMessage(); + + break; + case -27: msg = new GridDhtTxOnePhaseCommitAckRequest(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessage.java new file mode 100644 index 0000000000000..6145439d87b4f --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessage.java @@ -0,0 +1,594 @@ +/* + * 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.communication; + +import java.nio.ByteBuffer; +import java.util.UUID; +import org.apache.ignite.internal.GridDirectTransient; +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; + +/** + * + */ +public class IgniteIoTestMessage implements Message { + /** */ + private static byte FLAG_PROC_FROM_NIO = 1; + + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private long id; + + /** */ + private byte flags; + + /** */ + private boolean req; + + /** */ + private byte payload[]; + + /** */ + private long reqCreateTs; + + /** */ + private long reqSndTs; + + /** */ + private long reqSndTsMillis; + + /** */ + private long reqRcvTs; + + /** */ + private long reqRcvTsMillis; + + /** */ + private long reqProcTs; + + /** */ + private long resSndTs; + + /** */ + private long resSndTsMillis; + + /** */ + private long resRcvTs; + + /** */ + private long resRcvTsMillis; + + /** */ + private long resProcTs; + + /** */ + @GridDirectTransient + private UUID sndNodeId; + + /** + * + */ + public IgniteIoTestMessage() { + // No-op. + } + + /** + * @param id Message ID. + * @param req Request flag. + * @param payload Payload. + */ + public IgniteIoTestMessage(long id, boolean req, byte[] payload) { + this.id = id; + this.req = req; + this.payload = payload; + + reqCreateTs = System.nanoTime(); + } + + /** + * @return {@code True} if message should be processed from NIO thread + * (otherwise message is submitted to system pool). + */ + public boolean processFromNioThread() { + return isFlag(FLAG_PROC_FROM_NIO); + } + + /** + * @param procFromNioThread {@code True} if message should be processed from NIO thread. + */ + public void processFromNioThread(boolean procFromNioThread) { + setFlag(procFromNioThread, FLAG_PROC_FROM_NIO); + } + + /** + * @param flags Flags. + */ + public void flags(byte flags) { + this.flags = flags; + } + + /** + * @return Flags. + */ + public byte flags() { + return flags; + } + + /** + * Sets flag mask. + * + * @param flag Set or clear. + * @param mask Mask. + */ + private void setFlag(boolean flag, int mask) { + flags = flag ? (byte)(flags | mask) : (byte)(flags & ~mask); + } + + /** + * Reads flag mask. + * + * @param mask Mask to read. + * @return Flag value. + */ + private boolean isFlag(int mask) { + return (flags & mask) != 0; + } + + /** + * @return {@code true} if this is request. + */ + public boolean request() { + return req; + } + + /** + * @return ID. + */ + public long id() { + return id; + } + + /** + * @return Request create timestamp. + */ + public long requestCreateTs() { + return reqCreateTs; + } + + /** + * @return Request send timestamp. + */ + public long requestSendTs() { + return reqSndTs; + } + + /** + * @return Request receive timestamp. + */ + public long requestReceiveTs() { + return reqRcvTs; + } + + /** + * @return Request process started timestamp. + */ + public long requestProcessTs() { + return reqProcTs; + } + + /** + * @return Response send timestamp. + */ + public long responseSendTs() { + return resSndTs; + } + + /** + * @return Response receive timestamp. + */ + public long responseReceiveTs() { + return resRcvTs; + } + + /** + * @return Response process timestamp. + */ + public long responseProcessTs() { + return resProcTs; + } + + /** + * @return Request send timestamp (millis). + */ + public long requestSendTsMillis() { + return reqSndTsMillis; + } + + /** + * @return Request received timestamp (millis). + */ + public long requestReceivedTsMillis() { + return reqRcvTsMillis; + } + + /** + * @return Response send timestamp (millis). + */ + public long responseSendTsMillis() { + return resSndTsMillis; + } + + /** + * @return Response received timestamp (millis). + */ + public long responseReceivedTsMillis() { + return resRcvTsMillis; + } + + /** + * This method is called to initialize tracing variables. + * TODO: introduce direct message lifecycle API? + */ + public void onAfterRead() { + if (req && reqRcvTs == 0) { + reqRcvTs = System.nanoTime(); + + reqRcvTsMillis = System.currentTimeMillis(); + } + + if (!req && resRcvTs == 0) { + resRcvTs = System.nanoTime(); + + resRcvTsMillis = System.currentTimeMillis(); + } + } + + /** + * This method is called to initialize tracing variables. + * TODO: introduce direct message lifecycle API? + */ + public void onBeforeWrite() { + if (req && reqSndTs == 0) { + reqSndTs = System.nanoTime(); + + reqSndTsMillis = System.currentTimeMillis(); + } + + if (!req && resSndTs == 0) { + resSndTs = System.nanoTime(); + + resSndTsMillis = System.currentTimeMillis(); + } + } + + /** + * + */ + public void copyDataFromRequest(IgniteIoTestMessage req) { + reqCreateTs = req.reqCreateTs; + + reqSndTs = req.reqSndTs; + reqSndTsMillis = req.reqSndTsMillis; + + reqRcvTs = req.reqRcvTs; + reqRcvTsMillis = req.reqRcvTsMillis; + } + + /** + * + */ + public void onRequestProcessed() { + reqProcTs = System.nanoTime(); + } + + /** + * + */ + public void onResponseProcessed() { + resProcTs = System.nanoTime(); + } + + /** + * @return Response processed timestamp. + */ + public long responseProcessedTs() { + return resProcTs; + } + + /** + * @return Sender node ID. + */ + public UUID senderNodeId() { + return sndNodeId; + } + + /** + * @param sndNodeId Sender node ID. + */ + public void senderNodeId(UUID sndNodeId) { + this.sndNodeId = sndNodeId; + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + writer.setBuffer(buf); + + onBeforeWrite(); + + if (!writer.isHeaderWritten()) { + if (!writer.writeHeader(directType(), fieldsCount())) + return false; + + writer.onHeaderWritten(); + } + + switch (writer.state()) { + case 0: + if (!writer.writeByte("flags", flags)) + return false; + + writer.incrementState(); + + case 1: + if (!writer.writeLong("id", id)) + return false; + + writer.incrementState(); + + case 2: + if (!writer.writeByteArray("payload", payload)) + return false; + + writer.incrementState(); + + case 3: + if (!writer.writeBoolean("req", req)) + return false; + + writer.incrementState(); + + case 4: + if (!writer.writeLong("reqCreateTs", reqCreateTs)) + return false; + + writer.incrementState(); + + case 5: + if (!writer.writeLong("reqProcTs", reqProcTs)) + return false; + + writer.incrementState(); + + case 6: + if (!writer.writeLong("reqRcvTs", reqRcvTs)) + return false; + + writer.incrementState(); + + case 7: + if (!writer.writeLong("reqRcvTsMillis", reqRcvTsMillis)) + return false; + + writer.incrementState(); + + case 8: + if (!writer.writeLong("reqSndTs", reqSndTs)) + return false; + + writer.incrementState(); + + case 9: + if (!writer.writeLong("reqSndTsMillis", reqSndTsMillis)) + return false; + + writer.incrementState(); + + case 10: + if (!writer.writeLong("resProcTs", resProcTs)) + return false; + + writer.incrementState(); + + case 11: + if (!writer.writeLong("resRcvTs", resRcvTs)) + return false; + + writer.incrementState(); + + case 12: + if (!writer.writeLong("resRcvTsMillis", resRcvTsMillis)) + return false; + + writer.incrementState(); + + case 13: + if (!writer.writeLong("resSndTs", resSndTs)) + return false; + + writer.incrementState(); + + case 14: + if (!writer.writeLong("resSndTsMillis", resSndTsMillis)) + 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: + flags = reader.readByte("flags"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 1: + id = reader.readLong("id"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 2: + payload = reader.readByteArray("payload"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 3: + req = reader.readBoolean("req"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 4: + reqCreateTs = reader.readLong("reqCreateTs"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 5: + reqProcTs = reader.readLong("reqProcTs"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 6: + reqRcvTs = reader.readLong("reqRcvTs"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 7: + reqRcvTsMillis = reader.readLong("reqRcvTsMillis"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 8: + reqSndTs = reader.readLong("reqSndTs"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 9: + reqSndTsMillis = reader.readLong("reqSndTsMillis"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 10: + resProcTs = reader.readLong("resProcTs"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 11: + resRcvTs = reader.readLong("resRcvTs"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 12: + resRcvTsMillis = reader.readLong("resRcvTsMillis"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 13: + resSndTs = reader.readLong("resSndTs"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 14: + resSndTsMillis = reader.readLong("resSndTsMillis"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + } + + onAfterRead(); + + return reader.afterMessageRead(IgniteIoTestMessage.class); + } + + /** {@inheritDoc} */ + @Override public byte directType() { + return -43; + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 15; + } + + /** {@inheritDoc} */ + @Override public void onAckReceived() { + // No-op. + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(IgniteIoTestMessage.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/mxbean/IgniteMXBean.java b/modules/core/src/main/java/org/apache/ignite/mxbean/IgniteMXBean.java index 0754c27ee05bd..0a73d5ed61a0f 100644 --- a/modules/core/src/main/java/org/apache/ignite/mxbean/IgniteMXBean.java +++ b/modules/core/src/main/java/org/apache/ignite/mxbean/IgniteMXBean.java @@ -372,4 +372,48 @@ public interface IgniteMXBean { */ @MXBeanDescription("Dumps debug information for the current node.") public void dumpDebugInfo(); + + /** + * Runs IO latency test against all remote server nodes in cluster. + * + * @param warmup Warmup duration in milliseconds. + * @param duration Test duration in milliseconds. + * @param threads Thread count. + * @param maxLatency Max latency in nanoseconds. + * @param rangesCnt Ranges count in resulting histogram. + * @param payLoadSize Payload size in bytes. + * @param procFromNioThread {@code True} to process requests in NIO threads. + */ + @MXBeanDescription("Runs IO latency test against all remote server nodes in cluster.") + @MXBeanParametersNames( + { + "warmup", + "duration", + "threads", + "maxLatency", + "rangesCnt", + "payLoadSize", + "procFromNioThread" + } + ) + @MXBeanParametersDescriptions( + { + "Warmup duration (millis).", + "Test duration (millis).", + "Threads count.", + "Maximum latency expected (nanos).", + "Ranges count for histogram.", + "Payload size (bytes).", + "Process requests in NIO-threads flag." + } + ) + void runIoTest( + long warmup, + long duration, + int threads, + long maxLatency, + int rangesCnt, + int payLoadSize, + boolean procFromNioThread + ); } From 540ca449f1bd2386d3ba0722afb21dd3a504d044 Mon Sep 17 00:00:00 2001 From: Alexey Goncharuk Date: Tue, 13 Jun 2017 20:55:38 +0300 Subject: [PATCH 35/90] GG-12355: Added discovery ring latency test + made it available from MBean (cherry-picked from master). --- .../ignite/spi/discovery/tcp/ClientImpl.java | 20 +++++ .../ignite/spi/discovery/tcp/ServerImpl.java | 57 +++++++++++++- .../spi/discovery/tcp/TcpDiscoveryImpl.java | 6 ++ .../spi/discovery/tcp/TcpDiscoverySpi.java | 5 ++ .../discovery/tcp/TcpDiscoverySpiMBean.java | 22 ++++++ .../TcpDiscoveryRingLatencyCheckMessage.java | 77 +++++++++++++++++++ 6 files changed, 186 insertions(+), 1 deletion(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryRingLatencyCheckMessage.java diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java index b075dc1fb5289..71550447fa04d 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java @@ -97,6 +97,7 @@ import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeLeftMessage; import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryPingRequest; import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryPingResponse; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryRingLatencyCheckMessage; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentHashMap8; @@ -834,6 +835,16 @@ private NavigableSet allVisibleNodes() { U.closeQuiet(sockStream.socket()); } + /** {@inheritDoc} */ + @Override public void checkRingLatency(int maxHops) { + TcpDiscoveryRingLatencyCheckMessage msg = new TcpDiscoveryRingLatencyCheckMessage(getLocalNodeId(), maxHops); + + if (log.isInfoEnabled()) + log.info("Latency check initiated: " + msg.id()); + + sockWriter.sendMessage(msg); + } + /** {@inheritDoc} */ @Override protected IgniteSpiThread workerThread() { return msgWorker; @@ -1228,6 +1239,12 @@ void ackReceived(TcpDiscoveryClientAckResponse res) { msg, sockTimeout); + IgniteUuid latencyCheckId = msg instanceof TcpDiscoveryRingLatencyCheckMessage ? + msg.id() : null; + + if (latencyCheckId != null && log.isInfoEnabled()) + log.info("Latency check message has been written to socket: " + latencyCheckId); + msg = null; if (ack) { @@ -1255,6 +1272,9 @@ void ackReceived(TcpDiscoveryClientAckResponse res) { throw new IOException("Failed to get acknowledge for message: " + unacked); } + + if (latencyCheckId != null && log.isInfoEnabled()) + log.info("Latency check message has been acked: " + latencyCheckId); } } catch (InterruptedException e) { diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java index 58b362ffd0f15..6bdd80cbe6ff6 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java @@ -129,6 +129,7 @@ import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryPingRequest; import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryPingResponse; import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryRedirectToClient; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryRingLatencyCheckMessage; import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryStatusCheckMessage; import org.apache.ignite.thread.IgniteThreadPoolExecutor; import org.jetbrains.annotations.Nullable; @@ -1583,6 +1584,16 @@ private void clearNodeAddedMessage(TcpDiscoveryAbstractMessage msg) { } } + /** {@inheritDoc} */ + @Override public void checkRingLatency(int maxHops) { + TcpDiscoveryRingLatencyCheckMessage msg = new TcpDiscoveryRingLatencyCheckMessage(getLocalNodeId(), maxHops); + + if (log.isInfoEnabled()) + log.info("Latency check initiated: " + msg.id()); + + msgWorker.addMessage(msg); + } + /** {@inheritDoc} */ @Override void simulateNodeFailure() { U.warn(log, "Simulating node failure: " + getLocalNodeId()); @@ -2647,6 +2658,9 @@ else if (msg instanceof TcpDiscoveryCustomEventMessage) else if (msg instanceof TcpDiscoveryClientPingRequest) processClientPingRequest((TcpDiscoveryClientPingRequest)msg); + else if (msg instanceof TcpDiscoveryRingLatencyCheckMessage) + processRingLatencyCheckMessage((TcpDiscoveryRingLatencyCheckMessage)msg); + else assert false : "Unknown message type: " + msg.getClass().getSimpleName(); @@ -3054,12 +3068,20 @@ else if (!spi.failureDetectionTimeoutEnabled() && (e instanceof } } + boolean latencyCheck = msg instanceof TcpDiscoveryRingLatencyCheckMessage; + + if (latencyCheck && log.isInfoEnabled()) + log.info("Latency check message has been written to socket: " + msg.id()); + spi.writeToSocket(sock, out, msg, timeoutHelper.nextTimeoutChunk(spi.getSocketTimeout())); long tstamp0 = U.currentTimeMillis(); int res = spi.readReceipt(sock, timeoutHelper.nextTimeoutChunk(ackTimeout0)); + if (latencyCheck && log.isInfoEnabled()) + log.info("Latency check message has been acked: " + msg.id()); + spi.stats.onMessageSent(msg, tstamp0 - tstamp, U.currentTimeMillis() - tstamp0); onMessageExchanged(); @@ -4477,6 +4499,33 @@ private void processNodeAddFinishedMessage(TcpDiscoveryNodeAddFinishedMessage ms checkPendingCustomMessages(); } + /** + * Processes latency check message. + * + * @param msg Latency check message. + */ + private void processRingLatencyCheckMessage(TcpDiscoveryRingLatencyCheckMessage msg) { + assert msg != null; + + if (msg.maxHopsReached()) { + if (log.isInfoEnabled()) + log.info("Latency check has been discarded (max hops reached) [id=" + msg.id() + + ", maxHops=" + msg.maxHops() + ']'); + + return; + } + + if (log.isInfoEnabled()) + log.info("Latency check processing: " + msg.id()); + + if (sendMessageToRemotes(msg)) + sendMessageAcrossRing(msg); + else { + if (log.isInfoEnabled()) + log.info("Latency check has been discarded (no remote nodes): " + msg.id()); + } + } + /** * Processes node left message. * @@ -6056,7 +6105,7 @@ else if (msg instanceof TcpDiscoveryLoopbackProblemMessage) { continue; } - if (msg instanceof TcpDiscoveryPingResponse) { + else if (msg instanceof TcpDiscoveryPingResponse) { assert msg.client() : msg; ClientMessageWorker clientWorker = clientMsgWorkers.get(msg.creatorNodeId()); @@ -6066,6 +6115,12 @@ else if (msg instanceof TcpDiscoveryLoopbackProblemMessage) { continue; } + else if (msg instanceof TcpDiscoveryRingLatencyCheckMessage) { + if (log.isInfoEnabled()) + log.info("Latency check message has been read: " + msg.id()); + + ((TcpDiscoveryRingLatencyCheckMessage)msg).onRead(); + } TcpDiscoveryClientHeartbeatMessage heartbeatMsg = null; diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java index 84c2ff28afef0..cb85dc1c44b86 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java @@ -39,6 +39,7 @@ import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage; import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode; import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryAbstractMessage; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryRingLatencyCheckMessage; import org.jetbrains.annotations.Nullable; /** @@ -281,6 +282,11 @@ protected static String threadStatus(Thread t) { */ public abstract void brakeConnection(); + /** + * @param maxHops Maximum hops for {@link TcpDiscoveryRingLatencyCheckMessage}. + */ + public abstract void checkRingLatency(int maxHops); + /** * FOR TEST ONLY!!! * 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 a2a47feb31486..cefbcd675a303 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 @@ -1160,6 +1160,11 @@ LinkedHashSet getNodeAddresses(TcpDiscoveryNode node, boolean return stats.coordinatorSinceTimestamp(); } + /** {@inheritDoc} */ + @Override public void checkRingLatency(int maxHops) { + impl.checkRingLatency(maxHops); + } + /** {@inheritDoc} */ @Override protected void onContextInitialized0(IgniteSpiContext spiCtx) throws IgniteSpiException { super.onContextInitialized0(spiCtx); diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiMBean.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiMBean.java index 1427929a4cb88..e96088dd99d37 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiMBean.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiMBean.java @@ -20,6 +20,8 @@ import java.util.Map; import java.util.UUID; import org.apache.ignite.mxbean.MXBeanDescription; +import org.apache.ignite.mxbean.MXBeanParametersDescriptions; +import org.apache.ignite.mxbean.MXBeanParametersNames; import org.apache.ignite.spi.IgniteSpiManagementMBean; import org.jetbrains.annotations.Nullable; @@ -281,4 +283,24 @@ public interface TcpDiscoverySpiMBean extends IgniteSpiManagementMBean { */ @MXBeanDescription("Client mode.") public boolean isClientMode() throws IllegalStateException; + + /** + * Diagnosis method for determining ring message latency. + * On this method call special message will be sent across the ring + * and stats about the message will appear in the logs of each node. + * + * @param maxHops Maximum hops for the message (3 * TOTAL_NODE_CNT is recommended). + */ + @MXBeanDescription("Check ring latency.") + @MXBeanParametersNames( + { + "maxHops" + } + ) + @MXBeanParametersDescriptions( + { + "Maximum hops for the message (3 * TOTAL_NODE_CNT is recommended)." + } + ) + public void checkRingLatency(int maxHops); } \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryRingLatencyCheckMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryRingLatencyCheckMessage.java new file mode 100644 index 0000000000000..d8c11452e641e --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryRingLatencyCheckMessage.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.spi.discovery.tcp.messages; + +import java.util.UUID; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; + +/** + * + */ +public class TcpDiscoveryRingLatencyCheckMessage extends TcpDiscoveryAbstractMessage { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private int maxHops; + + /** */ + private int curHop; + + /** + * @param creatorNodeId Creator node ID. + * @param maxHops Max hops for this message. + */ + public TcpDiscoveryRingLatencyCheckMessage( + UUID creatorNodeId, + int maxHops + ) { + super(creatorNodeId); + + assert maxHops > 0; + + this.maxHops = maxHops; + } + + /** + * + */ + public void onRead() { + curHop++; + } + + /** + * @return Max hops. + */ + public int maxHops() { + return maxHops; + } + + /** + * @return {@code True} if max hops has been reached. + */ + public boolean maxHopsReached() { + return curHop == maxHops; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(TcpDiscoveryRingLatencyCheckMessage.class, this, "super", super.toString()); + } +} From 8cd9e829380f4c91cc9bb126169863286d1cb323 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Wed, 21 Jun 2017 15:40:14 +0300 Subject: [PATCH 36/90] GG-12353: Added local binary context flag. Backport of IGNITE-5223 with fixes. --- .../apache/ignite/IgniteSystemProperties.java | 10 +- .../internal/binary/BinaryReaderExImpl.java | 2 +- .../BinaryMarshallerLocalMetadataCache.java | 126 ++++++++++++++++++ 3 files changed, 128 insertions(+), 10 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerLocalMetadataCache.java diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java index f35e9e620070a..775aae03a1ef4 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -320,14 +320,6 @@ public final class IgniteSystemProperties { */ public static final String IGNITE_H2_DEBUG_CONSOLE = "IGNITE_H2_DEBUG_CONSOLE"; - /** - * This property allows to specify user defined port which H2 indexing SPI will use - * to start H2 debug console on. If this property is not set or set to 0, H2 debug - * console will use system-provided dynamic port. - * This property is only relevant when {@link #IGNITE_H2_DEBUG_CONSOLE} property is set. - */ - public static final String IGNITE_H2_DEBUG_CONSOLE_PORT = "IGNITE_H2_DEBUG_CONSOLE_PORT"; - /** * If this property is set to {@code true} then shared memory space native debug will be enabled. */ @@ -511,7 +503,7 @@ public final class IgniteSystemProperties { /** * Whether Ignite can access unaligned memory addresses. *

- * Defaults to {@code false}, meaning that unaligned access will be performed only on x86 architecture. + * Defaults to {@code} false, meaning that unaligned access will be performed only on x86 architecture. */ public static final String IGNITE_UNALIGNED_MEMORY_ACCESS = "IGNITE_UNALIGNED_MEMORY_ACCESS"; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryReaderExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryReaderExImpl.java index ad2e736fa5f34..7f71b460e5223 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryReaderExImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryReaderExImpl.java @@ -1723,7 +1723,7 @@ public BinarySchema getOrCreateSchema() { if (fieldIdLen != BinaryUtils.FIELD_ID_LEN) { BinaryTypeImpl type = (BinaryTypeImpl)ctx.metadata(typeId); - if (type == null || type.metadata() == null || type.metadata().schemas().isEmpty()) { + if (type == null || type.metadata() == null || type.metadata().schemas() == null || type.metadata().schemas().isEmpty()) { if (IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_USE_LOCAL_BINARY_MARSHALLER_CACHE, false)) { BinaryClassDescriptor desc = ctx.descriptorForTypeId(true, typeId, getClass().getClassLoader(), false); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerLocalMetadataCache.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerLocalMetadataCache.java new file mode 100644 index 0000000000000..8a2d38b61a48b --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerLocalMetadataCache.java @@ -0,0 +1,126 @@ +/* + * 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.binary; + +import java.io.Externalizable; +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import org.apache.ignite.IgniteSystemProperties; +import org.apache.ignite.binary.BinaryObject; +import org.apache.ignite.cluster.ClusterGroup; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteCallable; +import org.apache.ignite.testframework.junits.IgniteTestResources; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Tests IGNITE_USE_LOCAL_BINARY_MARSHALLER_CACHE property. + */ +public class BinaryMarshallerLocalMetadataCache extends GridCommonAbstractTest { + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName, + IgniteTestResources rsrcs) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName, rsrcs); + + cfg.setMarshaller(new BinaryMarshaller()); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + System.setProperty(IgniteSystemProperties.IGNITE_USE_LOCAL_BINARY_MARSHALLER_CACHE, "true"); + + startGrid(0); + startGrid(1); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + try { + stopAllGrids(); + } + finally { + System.clearProperty(IgniteSystemProperties.IGNITE_USE_LOCAL_BINARY_MARSHALLER_CACHE); + } + } + + /** + * @throws Exception If failed. + */ + public void testLocalMetadata() throws Exception { + final BinaryObject obj = grid(0).binary().toBinary(new OptimizedContainer(new Optimized())); + + ClusterGroup remotes = grid(0).cluster().forRemotes(); + + OptimizedContainer res = grid(0).compute(remotes).call(new IgniteCallable() { + @Override public OptimizedContainer call() throws Exception { + + return obj.deserialize(); + } + }); + + OptimizedContainer res2 = grid(0).compute(remotes).call(new IgniteCallable() { + @Override public OptimizedContainer call() throws Exception { + + return obj.deserialize(); + } + }); + + System.out.println(res); + System.out.println(res2); + } + + /** + * + */ + private static class OptimizedContainer { + /** */ + private Optimized optim; + + /** + * @param optim Val. + */ + public OptimizedContainer(Optimized optim) { + this.optim = optim; + } + } + + /** + * + */ + private static class Optimized implements Externalizable { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + + /** */ + private String fld; + + /** {@inheritDoc} */ + @Override public void writeExternal(ObjectOutput out) throws IOException { + U.writeUTFStringNullable(out, fld); + } + + /** {@inheritDoc} */ + @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { + fld = U.readUTFStringNullable(in); + } + } +} From f8224d13cf9a6432ba65e0016370ba51bbb544e9 Mon Sep 17 00:00:00 2001 From: Alexey Goncharuk Date: Thu, 15 Jun 2017 22:49:45 +0300 Subject: [PATCH 37/90] GG-12299: Make sure concurrent type registrations do not trigger multiple cache updates. --- .../internal/MarshallerContextAdapter.java | 134 ++++++++++++++---- .../BinaryMarshallerLocalMetadataCache.java | 63 +++++--- 2 files changed, 149 insertions(+), 48 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextAdapter.java index ad3439325d446..56007adb34544 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextAdapter.java @@ -32,6 +32,8 @@ import org.apache.ignite.IgniteException; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionFullMap; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionMap2; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.lang.IgniteOutClosureX; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.marshaller.MarshallerContext; import org.apache.ignite.plugin.PluginProvider; @@ -49,7 +51,7 @@ public abstract class MarshallerContextAdapter implements MarshallerContext { private static final String JDK_CLS_NAMES_FILE = "META-INF/classnames-jdk.properties"; /** */ - private final ConcurrentMap map = new ConcurrentHashMap8<>(); + private final ConcurrentMap map = new ConcurrentHashMap8<>(); /** */ private final Set registeredSystemTypes = new HashSet<>(); @@ -134,55 +136,129 @@ private void processResource(URL url) throws IOException { int typeId = clsName.hashCode(); - String oldClsName; + Object oldClsNameOrFuture = map.put(typeId, clsName); - if ((oldClsName = map.put(typeId, clsName)) != null) { - if (!oldClsName.equals(clsName)) - throw new IgniteException("Duplicate type ID [id=" + typeId + ", clsName=" + clsName + - ", oldClsName=" + oldClsName + ']'); - } + try { + String oldClsName = unwrap(oldClsNameOrFuture); + + if (oldClsName != null) { + if (!oldClsName.equals(clsName)) + throw new IgniteException("Duplicate type ID [id=" + typeId + ", clsName=" + clsName + + ", oldClsName=" + oldClsName + ']'); + } - registeredSystemTypes.add(clsName); + registeredSystemTypes.add(clsName); + } + catch (IgniteCheckedException e) { + throw new IllegalStateException("Failed to process type ID [typeId=" + typeId + + ", clsName" + clsName + ']', e); + } } } } /** {@inheritDoc} */ - @Override public boolean registerClass(int id, Class cls) throws IgniteCheckedException { - boolean registered = true; - - String clsName = map.get(id); + @Override public boolean registerClass(final int id, final Class cls) throws IgniteCheckedException { + Object clsNameOrFuture = map.get(id); + + String clsName = clsNameOrFuture != null ? + unwrap(clsNameOrFuture) : + computeIfAbsent(id, new IgniteOutClosureX() { + @Override public String applyx() throws IgniteCheckedException { + return registerClassName(id, cls.getName()) ? cls.getName() : null; + } + }); - if (clsName == null) { - registered = registerClassName(id, cls.getName()); + // The only way we can have clsName eq null here is a failing concurrent thread. + if (clsName == null) + return false; - if (registered) - map.putIfAbsent(id, cls.getName()); - } - else if (!clsName.equals(cls.getName())) + if (!clsName.equals(cls.getName())) throw new IgniteCheckedException("Duplicate ID [id=" + id + ", oldCls=" + clsName + ", newCls=" + cls.getName()); - return registered; + return true; } /** {@inheritDoc} */ - @Override public Class getClass(int id, ClassLoader ldr) throws ClassNotFoundException, IgniteCheckedException { - String clsName = map.get(id); + @Override public Class getClass(final int id, ClassLoader ldr) throws ClassNotFoundException, IgniteCheckedException { + Object clsNameOrFuture = map.get(id); + + String clsName = clsNameOrFuture != null ? + unwrap(clsNameOrFuture) : + computeIfAbsent(id, new IgniteOutClosureX() { + @Override public String applyx() throws IgniteCheckedException { + return className(id); + } + }); - if (clsName == null) { - clsName = className(id); + if (clsName == null) + throw new ClassNotFoundException("Unknown type ID: " + id); - if (clsName == null) - throw new ClassNotFoundException("Unknown type ID: " + id); + return U.forName(clsName, ldr); + } + + /** + * Computes the map value for the given ID. Will make sure that if there are two threads are calling + * {@code computeIfAbsent}, only one of them will invoke the closure. If the closure threw an exeption, + * all threads attempting to compute the value will throw this exception. + * + * @param id Type ID. + * @param clo Close to compute. + * @return Computed value. + * @throws IgniteCheckedException If closure threw an exception. + */ + private String computeIfAbsent(int id, IgniteOutClosureX clo) throws IgniteCheckedException { + Object clsNameOrFuture = map.get(id); - String old = map.putIfAbsent(id, clsName); + if (clsNameOrFuture == null) { + GridFutureAdapter fut = new GridFutureAdapter<>(); - if (old != null) - clsName = old; + Object old = map.putIfAbsent(id, fut); + + if (old == null) { + String clsName = null; + + try { + try { + clsName = clo.applyx(); + + fut.onDone(clsName); + + clsNameOrFuture = clsName; + } + catch (Throwable e) { + fut.onDone(e); + + throw e; + } + } + finally { + if (clsName != null) + map.replace(id, fut, clsName); + else + map.remove(id, fut); + } + } + else + clsNameOrFuture = old; } - return U.forName(clsName, ldr); + // Unwrap the existing object. + return unwrap(clsNameOrFuture); + } + + /** + * Unwraps an object into the String. Expects the object be {@code null}, a String or a GridFutureAdapter. + * + * @param clsNameOrFuture Class name or future to unwrap. + * @return Unwrapped value. + * @throws IgniteCheckedException If future completed with an exception. + */ + private String unwrap(Object clsNameOrFuture) throws IgniteCheckedException { + return clsNameOrFuture == null ? null : + clsNameOrFuture instanceof String ? (String)clsNameOrFuture : + ((GridFutureAdapter)clsNameOrFuture).get(); } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerLocalMetadataCache.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerLocalMetadataCache.java index 8a2d38b61a48b..571091e004567 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerLocalMetadataCache.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerLocalMetadataCache.java @@ -21,12 +21,19 @@ import java.io.IOException; import java.io.ObjectInput; import java.io.ObjectOutput; +import java.util.Collections; +import java.util.concurrent.Callable; +import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteSystemProperties; -import org.apache.ignite.binary.BinaryObject; -import org.apache.ignite.cluster.ClusterGroup; +import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.MarshallerContextAdapter; import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.lang.IgniteCallable; +import org.apache.ignite.marshaller.MarshallerContext; +import org.apache.ignite.plugin.PluginProvider; +import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.IgniteTestResources; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; @@ -39,7 +46,9 @@ public class BinaryMarshallerLocalMetadataCache extends GridCommonAbstractTest { IgniteTestResources rsrcs) throws Exception { IgniteConfiguration cfg = super.getConfiguration(gridName, rsrcs); - cfg.setMarshaller(new BinaryMarshaller()); + cfg.setMarshaller(new BinaryMarshallerWrapper()); + + cfg.setCacheConfiguration(new CacheConfiguration().setName("part").setBackups(1)); return cfg; } @@ -66,26 +75,20 @@ public class BinaryMarshallerLocalMetadataCache extends GridCommonAbstractTest { * @throws Exception If failed. */ public void testLocalMetadata() throws Exception { - final BinaryObject obj = grid(0).binary().toBinary(new OptimizedContainer(new Optimized())); - - ClusterGroup remotes = grid(0).cluster().forRemotes(); + final CyclicBarrier bar = new CyclicBarrier(64); - OptimizedContainer res = grid(0).compute(remotes).call(new IgniteCallable() { - @Override public OptimizedContainer call() throws Exception { - - return obj.deserialize(); - } - }); + awaitPartitionMapExchange(); - OptimizedContainer res2 = grid(0).compute(remotes).call(new IgniteCallable() { - @Override public OptimizedContainer call() throws Exception { + GridTestUtils.runMultiThreaded(new Callable() { + @Override public Object call() throws Exception { + bar.await(); - return obj.deserialize(); + return grid(0).binary().toBinary(new OptimizedContainer(new Optimized())); } - }); + }, 64, "async-runner"); - System.out.println(res); - System.out.println(res2); + // We expect 3 here because Externalizable classes are registered twice with different type IDs. + assertEquals(3, ((BinaryMarshallerWrapper)grid(0).configuration().getMarshaller()).registerClassCalled.get()); } /** @@ -123,4 +126,26 @@ private static class Optimized implements Externalizable { fld = U.readUTFStringNullable(in); } } + + private static class BinaryMarshallerWrapper extends BinaryMarshaller { + private MarshallerContext ctx0 = new MarshallerContextAdapter(Collections.emptyList()) { + @Override protected boolean registerClassName(int id, String clsName) { + U.dumpStack(id + " " + clsName); + registerClassCalled.incrementAndGet(); + + return true; + } + + @Override protected String className(int id) throws IgniteCheckedException { + return null; + } + }; + + private AtomicInteger registerClassCalled = new AtomicInteger(); + + /** {@inheritDoc} */ + @Override public MarshallerContext getContext() { + return ctx0; + } + } } From 4ffc3acfa1bc43bea8c79bfd1864787c15cfc4de Mon Sep 17 00:00:00 2001 From: Alexey Goncharuk Date: Tue, 20 Jun 2017 07:59:09 +0300 Subject: [PATCH 38/90] IGNITE-5528 - IS_EVICT_DISABLED flag is not cleared on cache store exception. --- .../processors/cache/GridCacheAdapter.java | 62 ++++--- .../processors/cache/GridCacheEntryEx.java | 3 +- .../processors/cache/GridCacheMapEntry.java | 2 +- ...tionEvictionDuringReadThroughSelfTest.java | 160 ++++++++++++++++++ .../testsuites/IgniteCacheTestSuite5.java | 3 + 5 files changed, 205 insertions(+), 25 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionEvictionDuringReadThroughSelfTest.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 47c9998239777..220c192f21c46 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 @@ -2061,35 +2061,18 @@ protected final IgniteInternalFuture> getAllAsync0( } }); - if (loaded.size() != loadKeys.size()) { - boolean needTouch = - tx0 == null || (!tx0.implicit() && tx0.isolation() == READ_COMMITTED); - - for (Map.Entry e : loadKeys.entrySet()) { - if (loaded.contains(e.getKey())) - continue; - - if (needTouch || e.getValue().reserved()) { - GridCacheEntryEx entry = peekEx(e.getKey()); - - if (entry != null) { - if (e.getValue().reserved()) - entry.clearReserveForLoad(e.getValue().version()); - - if (needTouch) - ctx.evicts().touch(entry, topVer); - } - } - } - } + clearReservationsIfNeeded(topVer, loadKeys, loaded, tx0); return map; } }), true), new C2, Exception, IgniteInternalFuture>>() { @Override public IgniteInternalFuture> apply(Map map, Exception e) { - if (e != null) + if (e != null) { + clearReservationsIfNeeded(topVer, loadKeys, loaded, tx0); + return new GridFinishedFuture<>(e); + } if (tx0 == null || (!tx0.implicit() && tx0.isolation() == READ_COMMITTED)) { Collection notFound = new HashSet<>(loadKeys.keySet()); @@ -2146,6 +2129,41 @@ protected final IgniteInternalFuture> getAllAsync0( } } + /** + * @param topVer Affinity topology version for which load was performed. + * @param loadKeys Keys to load. + * @param loaded Actually loaded keys. + * @param tx0 Transaction within which the load was run, if any. + */ + private void clearReservationsIfNeeded( + AffinityTopologyVersion topVer, + Map loadKeys, + Collection loaded, + IgniteTxLocalAdapter tx0 + ) { + if (loaded.size() != loadKeys.size()) { + boolean needTouch = + tx0 == null || (!tx0.implicit() && tx0.isolation() == READ_COMMITTED); + + for (Map.Entry e : loadKeys.entrySet()) { + if (loaded.contains(e.getKey())) + continue; + + if (needTouch || e.getValue().reserved()) { + GridCacheEntryEx entry = peekEx(e.getKey()); + + if (entry != null) { + if (e.getValue().reserved()) + entry.clearReserveForLoad(e.getValue().version()); + + if (needTouch) + ctx.evicts().touch(entry, topVer); + } + } + } + } + } + /** {@inheritDoc} */ @Override public final V getAndPut(K key, V val) throws IgniteCheckedException { return getAndPut(key, val, null); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java index ccd22850d9eb7..2574ee2623d0a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java @@ -363,9 +363,8 @@ public EntryGetResult innerGetAndReserveForLoad(boolean readSwap, /** * @param ver Expected entry version. - * @throws IgniteCheckedException If failed. */ - public void clearReserveForLoad(GridCacheVersion ver) throws IgniteCheckedException; + public void clearReserveForLoad(GridCacheVersion ver); /** * Reloads entry from underlying storage. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java index ea01bca4bd50b..1b2d63bc624f1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java @@ -3621,7 +3621,7 @@ private long nextPartCounter(AffinityTopologyVersion topVer) { } /** {@inheritDoc} */ - @Override public synchronized void clearReserveForLoad(GridCacheVersion ver) throws IgniteCheckedException { + @Override public synchronized void clearReserveForLoad(GridCacheVersion ver) { if (obsoleteVersionExtras() != null) return; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionEvictionDuringReadThroughSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionEvictionDuringReadThroughSelfTest.java new file mode 100644 index 0000000000000..d5351f7e37776 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionEvictionDuringReadThroughSelfTest.java @@ -0,0 +1,160 @@ +/* + * 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.cache.distributed; + +import java.util.LinkedHashSet; +import java.util.concurrent.Callable; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import javax.cache.Cache; +import javax.cache.configuration.Factory; +import javax.cache.expiry.CreatedExpiryPolicy; +import javax.cache.expiry.Duration; +import javax.cache.integration.CacheLoaderException; +import javax.cache.integration.CacheWriterException; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.eviction.lru.LruEvictionPolicy; +import org.apache.ignite.cache.store.CacheStore; +import org.apache.ignite.cache.store.CacheStoreAdapter; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.configuration.NearCacheConfiguration; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * + */ +public class GridCachePartitionEvictionDuringReadThroughSelfTest extends GridCommonAbstractTest { + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + CacheConfiguration ccfg = + new CacheConfiguration() + .setName("config") + .setAtomicityMode(CacheAtomicityMode.ATOMIC) + .setBackups(0) // No need for backup, just load from the store if needed + .setCacheStoreFactory(new CacheStoreFactory()) + .setEvictionPolicy(new LruEvictionPolicy(100)) + .setNearConfiguration(new NearCacheConfiguration() + .setNearEvictionPolicy(new LruEvictionPolicy())); + + ccfg.setExpiryPolicyFactory(CreatedExpiryPolicy.factoryOf(new Duration(TimeUnit.MINUTES, 1))) + .setReadThrough(true) + .setWriteThrough(false); + + cfg.setCacheConfiguration(ccfg); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** + * @throws Exception if failed. + */ + public void testPartitionRent() throws Exception { + startGrid(0); + + final AtomicBoolean done = new AtomicBoolean(); + + IgniteInternalFuture fut = GridTestUtils.runMultiThreadedAsync(new Callable() { + @Override + public Integer call() throws Exception { + LinkedHashSet set = new LinkedHashSet<>(); + + set.add(1); + set.add(2); + set.add(3); + set.add(4); + set.add(5); + + while (!done.get()) { + try { + grid(0).cache("config").getAll(set); + } + catch (Throwable ignore) { + // No-op. + } + } + + return null; + } + }, 4, "loader"); + + IgniteInternalFuture startFut = GridTestUtils.runAsync(new Callable() { + @Override public Void call() throws Exception { + for (int i = 1; i < 5; i++) { + startGrid(i); + + awaitPartitionMapExchange(); + } + + return null; + } + }); + + startFut.get(); + + done.set(true); + + fut.get(); + } + + /** + * + */ + private static class CacheStoreFactory implements Factory> { + /** {@inheritDoc} */ + @Override public CacheStore create() { + return new HangingCacheStore(); + } + } + + /** + * + */ + private static class HangingCacheStore extends CacheStoreAdapter { + /** */ + private CountDownLatch releaseLatch = new CountDownLatch(1); + + /** {@inheritDoc} */ + @Override public Integer load(Integer key) throws CacheLoaderException { + if (key == 3) + throw new CacheLoaderException(); + + return key; + } + + /** {@inheritDoc} */ + @Override public void write(Cache.Entry entry) throws CacheWriterException { + + } + + /** {@inheritDoc} */ + @Override public void delete(Object key) throws CacheWriterException { + + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java index 7582f5cc3ee88..ab0007b2f1459 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java @@ -28,6 +28,7 @@ import org.apache.ignite.internal.processors.cache.distributed.CacheLateAffinityAssignmentFairAffinityTest; import org.apache.ignite.internal.processors.cache.distributed.CacheLateAffinityAssignmentNodeJoinValidationTest; import org.apache.ignite.internal.processors.cache.distributed.CacheLateAffinityAssignmentTest; +import org.apache.ignite.internal.processors.cache.distributed.GridCachePartitionEvictionDuringReadThroughSelfTest; import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheSyncRebalanceModeSelfTest; import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheTxIteratorSelfTest; import org.apache.ignite.internal.processors.cache.store.IgniteCacheWriteBehindNoUpdateSelfTest; @@ -59,6 +60,8 @@ public static TestSuite suite() throws Exception { suite.addTest(IgniteCacheReadThroughEvictionsVariationsSuite.suite()); suite.addTestSuite(IgniteCacheTxIteratorSelfTest.class); + suite.addTestSuite(GridCachePartitionEvictionDuringReadThroughSelfTest.class); + return suite; } } From 9036ad239d68eff663bc73a81baab2826b054d9a Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Wed, 21 Jun 2017 18:25:31 +0300 Subject: [PATCH 39/90] Added MBean for system cache executors. --- .../apache/ignite/internal/IgniteKernal.java | 38 +++++++++++++++---- 1 file changed, 30 insertions(+), 8 deletions(-) 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 4237aeebe49ce..2aa05f1b9ab11 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 @@ -61,10 +61,10 @@ import org.apache.ignite.IgniteEvents; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteFileSystem; +import org.apache.ignite.IgniteLock; import org.apache.ignite.IgniteLogger; import org.apache.ignite.IgniteMessaging; import org.apache.ignite.IgniteQueue; -import org.apache.ignite.IgniteLock; import org.apache.ignite.IgniteScheduler; import org.apache.ignite.IgniteSemaphore; import org.apache.ignite.IgniteServices; @@ -115,7 +115,6 @@ import org.apache.ignite.internal.processors.datastreamer.DataStreamProcessor; import org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor; import org.apache.ignite.internal.processors.hadoop.Hadoop; -import org.apache.ignite.internal.processors.hadoop.HadoopClassLoader; import org.apache.ignite.internal.processors.hadoop.HadoopProcessorAdapter; import org.apache.ignite.internal.processors.job.GridJobProcessor; import org.apache.ignite.internal.processors.jobmetrics.GridJobMetricsProcessor; @@ -175,6 +174,7 @@ import org.apache.ignite.thread.IgniteStripedThreadPoolExecutor; import org.jetbrains.annotations.Nullable; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_BINARY_MARSHALLER_USE_STRING_SERIALIZATION_VER_2; import static org.apache.ignite.IgniteSystemProperties.IGNITE_CONFIG_URL; import static org.apache.ignite.IgniteSystemProperties.IGNITE_DAEMON; import static org.apache.ignite.IgniteSystemProperties.IGNITE_NO_ASCII; @@ -183,7 +183,6 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK; import static org.apache.ignite.IgniteSystemProperties.IGNITE_STARVATION_CHECK_INTERVAL; import static org.apache.ignite.IgniteSystemProperties.IGNITE_SUCCESS_FILE; -import static org.apache.ignite.IgniteSystemProperties.IGNITE_BINARY_MARSHALLER_USE_STRING_SERIALIZATION_VER_2; import static org.apache.ignite.IgniteSystemProperties.getBoolean; import static org.apache.ignite.IgniteSystemProperties.snapshot; import static org.apache.ignite.internal.GridKernalState.DISCONNECTED; @@ -200,7 +199,6 @@ import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_CLIENT_MODE; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_CONSISTENCY_CHECK_SKIPPED; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_DAEMON; -import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_LATE_AFFINITY_ASSIGNMENT; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_DEPLOYMENT_MODE; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_GRID_NAME; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_IPS; @@ -209,11 +207,12 @@ import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_JVM_ARGS; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_JVM_PID; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_LANG_RUNTIME; +import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_LATE_AFFINITY_ASSIGNMENT; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MACS; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER_COMPACT_FOOTER; -import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER_USE_DFLT_SUID; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER_USE_BINARY_STRING_SER_VER_2; +import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER_USE_DFLT_SUID; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_NODE_CONSISTENT_ID; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_PEER_CLASSLOADING; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_PHY_RAM; @@ -292,6 +291,14 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable { @GridToStringExclude private ObjectName p2PExecSvcMBean; + /** */ + @GridToStringExclude + private ObjectName utilityExecSvcMBean; + + /** */ + @GridToStringExclude + private ObjectName marshallerExecSvcMBean; + /** */ @GridToStringExclude private ObjectName restExecSvcMBean; @@ -969,7 +976,14 @@ public void start(final IgniteConfiguration cfg, // Register MBeans. registerKernalMBean(); registerLocalNodeMBean(); - registerExecutorMBeans(execSvc, sysExecSvc, p2pExecSvc, mgmtExecSvc, restExecSvc); + registerExecutorMBeans( + execSvc, + sysExecSvc, + p2pExecSvc, + mgmtExecSvc, + restExecSvc, + utilityCachePool, + marshCachePool); // Lifecycle bean notifications. notifyLifecycleBeans(AFTER_NODE_START); @@ -1523,15 +1537,21 @@ private void registerLocalNodeMBean() throws IgniteCheckedException { } /** @throws IgniteCheckedException If registration failed. */ - private void registerExecutorMBeans(ExecutorService execSvc, + private void registerExecutorMBeans( + ExecutorService execSvc, ExecutorService sysExecSvc, ExecutorService p2pExecSvc, ExecutorService mgmtExecSvc, - ExecutorService restExecSvc) throws IgniteCheckedException { + ExecutorService restExecSvc, + ExecutorService utilityExecSvc, + ExecutorService marshallerExecSvc + ) throws IgniteCheckedException { pubExecSvcMBean = registerExecutorMBean(execSvc, "GridExecutionExecutor"); sysExecSvcMBean = registerExecutorMBean(sysExecSvc, "GridSystemExecutor"); mgmtExecSvcMBean = registerExecutorMBean(mgmtExecSvc, "GridManagementExecutor"); p2PExecSvcMBean = registerExecutorMBean(p2pExecSvc, "GridClassLoadingExecutor"); + utilityExecSvcMBean = registerExecutorMBean(utilityExecSvc ,"GridUtilityCacheExecutor"); + marshallerExecSvcMBean = registerExecutorMBean(marshallerExecSvc, "GridMarshallerCacheExecutor"); ConnectorConfiguration clientCfg = cfg.getConnectorConfiguration(); @@ -2027,6 +2047,8 @@ else if (state == STARTING) unregisterMBean(p2PExecSvcMBean) & unregisterMBean(kernalMBean) & unregisterMBean(locNodeMBean) & + unregisterMBean(utilityExecSvcMBean) & + unregisterMBean(marshallerExecSvcMBean) & unregisterMBean(restExecSvcMBean) )) errOnStop = false; From ed34a5dc681ea8f284f4d25c5575ad46569cc600 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Wed, 21 Jun 2017 18:33:55 +0300 Subject: [PATCH 40/90] Partial fix of IGNITE-5562. --- .../spi/discovery/tcp/internal/TcpDiscoveryStatistics.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryStatistics.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryStatistics.java index c79064491bb43..a69dbd9aa89aa 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryStatistics.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryStatistics.java @@ -316,7 +316,9 @@ public synchronized void onMessageProcessingFinished(TcpDiscoveryAbstractMessage */ public synchronized void onMessageSent(TcpDiscoveryAbstractMessage msg, long time, long ackTime) { assert msg != null; - assert time >= 0 : time; + + if (time < 0) + time = 0; if (crdSinceTs.get() > 0 && (msg instanceof TcpDiscoveryCustomEventMessage) || From d427021f329292fb69d348ba949ad1f8f1e9089e Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Wed, 21 Jun 2017 19:30:27 +0300 Subject: [PATCH 41/90] IGNITE-5552: ServiceProcessor recalculates all service assignments even if there is a pending topology change. --- .../service/GridServiceProcessor.java | 20 ++++++++++++++++++- 1 file changed, 19 insertions(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java index 25a8edbfc7c17..9c746401b1479 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java @@ -169,7 +169,7 @@ public class GridServiceProcessor extends GridProcessorAdapter { private IgniteInternalCache cache; /** Topology listener. */ - private GridLocalEventListener topLsnr = new TopologyListener(); + private final GridLocalEventListener topLsnr = new TopologyListener(); static { Set versions = new TreeSet<>(new Comparator() { @@ -1574,6 +1574,9 @@ private void onDeployment(final GridServiceDeployment dep, final AffinityTopolog * Topology listener. */ private class TopologyListener implements GridLocalEventListener { + /** */ + private volatile AffinityTopologyVersion currTopVer = null; + /** {@inheritDoc} */ @Override public void onEvent(Event evt) { if (!busyLock.enterBusy()) @@ -1601,6 +1604,8 @@ else if (msg instanceof DynamicCacheChangeBatch) { else topVer = new AffinityTopologyVersion(((DiscoveryEvent)evt).topologyVersion(), 0); + currTopVer = topVer; + depExe.submit(new DepRunnable() { @Override public void run0() { ClusterNode oldest = ctx.discovery().oldestAliveCacheServerNode(topVer); @@ -1618,6 +1623,19 @@ else if (msg instanceof DynamicCacheChangeBatch) { boolean firstTime = true; while (it.hasNext()) { + // If topology changed again, let next event handle it. + AffinityTopologyVersion currTopVer0 = currTopVer; + + if (currTopVer0 != topVer) { + if (log.isInfoEnabled()) + log.info("Service processor detected a topology change during " + + "assignments calculation (will abort current iteration and " + + "re-calculate on the newer version): " + + "[topVer=" + topVer + ", newTopVer=" + currTopVer + ']'); + + return; + } + Cache.Entry e = it.next(); if (!(e.getKey() instanceof GridServiceDeploymentKey)) From f1b9cdc0716a1b23f54d68ce0fe19eb85107567d Mon Sep 17 00:00:00 2001 From: Alexey Goncharuk Date: Wed, 14 Jun 2017 21:37:54 +0300 Subject: [PATCH 42/90] GG-12354: Partial fix of IGNITE-5473: Introduce troubleshooting logger. --- .../dht/GridDhtPartitionTopologyImpl.java | 2 +- ...NearAtomicAbstractSingleUpdateRequest.java | 8 +++ .../GridNearAtomicAbstractUpdateFuture.java | 8 +++ ...idNearAtomicSingleUpdateInvokeRequest.java | 6 ++ .../atomic/GridNearAtomicUpdateFuture.java | 1 + .../atomic/GridNearAtomicUpdateResponse.java | 2 +- .../service/GridServiceProcessor.java | 49 ++++++++++--- .../service/ServiceContextImpl.java | 2 + .../util/tostring/GridToStringBuilder.java | 2 +- .../tcp/TcpCommunicationSpi.java | 57 +++++++-------- .../ignite/spi/discovery/tcp/ServerImpl.java | 72 +++++++++++++------ .../spi/discovery/tcp/TcpDiscoverySpi.java | 16 +++-- 12 files changed, 157 insertions(+), 68 deletions(-) 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 1b4dcc9293dee..1ad0ff0538fa5 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 @@ -210,7 +210,7 @@ private boolean waitForRent() throws IgniteCheckedException { break; } - catch (IgniteFutureTimeoutCheckedException e) { + catch (IgniteFutureTimeoutCheckedException ignore) { if (dumpCnt++ < GridDhtPartitionsExchangeFuture.DUMP_PENDING_OBJECTS_THRESHOLD) { U.warn(log, "Failed to wait for partition eviction [" + "topVer=" + topVer + diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractSingleUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractSingleUpdateRequest.java index 61deeee1e1042..1e87e251a6680 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractSingleUpdateRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractSingleUpdateRequest.java @@ -29,6 +29,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheOperation; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.MessageReader; import org.apache.ignite.plugin.extensions.communication.MessageWriter; import org.jetbrains.annotations.NotNull; @@ -559,4 +560,11 @@ private boolean isFlag(int mask) { @Override public byte fieldsCount() { return 11; } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(GridNearAtomicAbstractSingleUpdateRequest.class, this, + "nodeId", nodeId, "futVer", futVer, "topVer", topVer, + "parent", super.toString()); + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateFuture.java index c92e0f5ec5b42..936c8a3bd3c87 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateFuture.java @@ -35,6 +35,7 @@ import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.typedef.CI2; 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.IgniteUuid; import org.jetbrains.annotations.Nullable; @@ -342,4 +343,11 @@ protected final GridCacheVersion addAtomicFuture(AffinityTopologyVersion topVer) return futVer; } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(GridNearAtomicAbstractUpdateFuture.class, this, + "topLocked", topLocked, "remapCnt", remapCnt, "resCnt", resCnt, "err", err, + "parent", super.toString()); + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java index df9e38441e79f..bf6bc17f16732 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java @@ -34,6 +34,7 @@ import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; 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.plugin.extensions.communication.MessageCollectionItemType; import org.apache.ignite.plugin.extensions.communication.MessageReader; @@ -303,4 +304,9 @@ public GridNearAtomicSingleUpdateInvokeRequest() { @Override public byte directType() { return 126; } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(GridNearAtomicSingleUpdateRequest.class, this, super.toString()); + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java index 950e5bdd2b9b1..a252d9ae30c2b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java @@ -90,6 +90,7 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu private Map mappings; /** Keys to remap. */ + @GridToStringInclude private Collection remapKeys; /** Not null is operation is mapped to single node. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java index 2e38733e8b900..e48b7265f51c6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java @@ -642,6 +642,6 @@ public synchronized void addFailedKeys(Collection keys, Throwabl /** {@inheritDoc} */ @Override public String toString() { - return S.toString(GridNearAtomicUpdateResponse.class, this, "parent"); + return S.toString(GridNearAtomicUpdateResponse.class, this, super.toString()); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java index 9c746401b1479..4ecf674a48420 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java @@ -974,12 +974,24 @@ private void reassign(GridServiceDeployment dep, AffinityTopologyVersion topVer) int cnt = maxPerNodeCnt == 0 ? totalCnt == 0 ? 1 : totalCnt : maxPerNodeCnt; cnts.put(n.id(), cnt); + + if (log.isInfoEnabled()) + log.info("Assigned service to primary node [svc=" + dep.configuration().getName() + + ", topVer=" + topVer + ", node=" + n.id() + ']'); } } else { if (!nodes.isEmpty()) { int size = nodes.size(); + if (log.isInfoEnabled()) + log.info("Calculating assignments for service " + + "[svc=" + dep.configuration().getName() + + ", topVer=" + topVer + + ", nodes=" + U.nodeIds(nodes) + + ", oldAssignment=" + (oldAssigns == null ? "NA" : oldAssigns.assigns()) + + ", totalCnt=" + totalCnt + ", maxPerNodeCnt=" + maxPerNodeCnt + ']'); + int perNodeCnt = totalCnt != 0 ? totalCnt / size : maxPerNodeCnt; int remainder = totalCnt != 0 ? totalCnt % size : 0; @@ -1055,6 +1067,10 @@ private void reassign(GridServiceDeployment dep, AffinityTopologyVersion topVer) assigns.assigns(cnts); + if (log.isInfoEnabled()) + log.info("Calculated new assignments for service [svc=" + dep.configuration().getName() + + ", assignment=" + assigns + ']'); + cache.put(key, assigns); tx.commit(); @@ -1095,6 +1111,10 @@ private void redeploy(GridServiceAssignments assigns) { Collection toInit = new ArrayList<>(); synchronized (ctxs) { + if (log.isInfoEnabled()) + log.info("Updating service deployment [locNodeId=" + ctx.localNodeId() + + ", svc=" + assigns.name() + ", assigns=" + assigns + ']'); + if (ctxs.size() > assignCnt) { int cancelCnt = ctxs.size() - assignCnt; @@ -1483,6 +1503,10 @@ private void processDeployment(CacheEntryEvent>>").append(U.nl()); b.append(">>>").append("Dumping discovery SPI debug info.").append(U.nl()); b.append(">>>").append(U.nl()); @@ -1735,9 +1742,9 @@ TcpDiscoveryNodesRing ring() { b.append(U.nl()); b.append("Stats: ").append(spi.stats).append(U.nl()); - - U.quietAndInfo(log, b.toString()); } + + U.quietAndInfo(log, b.toString()); } /** @@ -3171,12 +3178,8 @@ else if (e instanceof SocketTimeoutException || // If node existed on connection initialization we should check // whether it has not gone yet. - if (nextNodeExists) - U.warn(log, "Failed to send message to next node [msg=" + msg + ", next=" + next + - ", errMsg=" + (err != null ? err.getMessage() : "N/A") + ']'); - else if (log.isDebugEnabled()) - log.debug("Failed to send message to next node [msg=" + msg + ", next=" + next + - ", errMsg=" + (err != null ? err.getMessage() : "N/A") + ']'); + U.warn(log, "Failed to send message to next node [msg=" + msg + ", next=" + next + + ", errMsg=" + (err != null ? err.getMessage() : "N/A") + ']'); } } @@ -3358,6 +3361,8 @@ private void processJoinRequestMessage(final TcpDiscoveryJoinRequestMessage msg) if (existingNode != null) { if (!node.socketAddresses().equals(existingNode.socketAddresses())) { if (!pingNode(existingNode)) { + U.warn(log, "Sending node failed message for existing node: " + node); + addMessage(new TcpDiscoveryNodeFailedMessage(locNodeId, existingNode.id(), existingNode.internalOrder())); @@ -5421,8 +5426,7 @@ private void checkFailedNodesList() { if (msgs != null) { for (TcpDiscoveryNodeFailedMessage msg : msgs) { - if (log.isDebugEnabled()) - log.debug("Add node failed message for node from failed nodes list: " + msg); + U.warn(log, "Added node failed message for node from failed nodes list: " + msg); addMessage(msg); } @@ -5641,8 +5645,9 @@ private class TcpServer extends IgniteSpiThread { long tstamp = U.currentTimeMillis(); - if (log.isDebugEnabled()) - log.debug("Accepted incoming connection from addr: " + sock.getInetAddress()); + if (log.isInfoEnabled()) + log.info("TCP discovery accepted incoming connection " + + "[rmtAddr=" + sock.getInetAddress() + ", rmtPort=" + sock.getPort() + ']'); SocketReader reader = new SocketReader(sock); @@ -5650,6 +5655,10 @@ private class TcpServer extends IgniteSpiThread { readers.add(reader); } + if (log.isInfoEnabled()) + log.info("TCP discovery spawning a new thread for connection " + + "[rmtAddr=" + sock.getInetAddress() + ", rmtPort=" + sock.getPort() + ']'); + reader.start(); spi.stats.onServerSocketInitialized(U.currentTimeMillis() - tstamp); @@ -5712,6 +5721,12 @@ private class SocketReader extends IgniteSpiThread { ClientMessageWorker clientMsgWrk = null; + SocketAddress rmtAddr = sock.getRemoteSocketAddress(); + + if (log.isInfoEnabled()) + log.info("Started serving remote node connection [rmtAddr=" + rmtAddr + + ", rmtPort=" + sock.getPort() + ']'); + try { InputStream in; @@ -5742,11 +5757,11 @@ private class SocketReader extends IgniteSpiThread { else { if (log.isDebugEnabled()) log.debug("Failed to read magic header (too few bytes received) " + - "[rmtAddr=" + sock.getRemoteSocketAddress() + + "[rmtAddr=" + rmtAddr + ", locAddr=" + sock.getLocalSocketAddress() + ']'); LT.warn(log, "Failed to read magic header (too few bytes received) [rmtAddr=" + - sock.getRemoteSocketAddress() + ", locAddr=" + sock.getLocalSocketAddress() + ']'); + rmtAddr + ", locAddr=" + sock.getLocalSocketAddress() + ']'); return; } @@ -5758,7 +5773,7 @@ private class SocketReader extends IgniteSpiThread { "this Ignite port?" + (!spi.isSslEnabled() ? " missed SSL configuration?" : "" ) + ") " + - "[rmtAddr=" + sock.getRemoteSocketAddress() + + "[rmtAddr=" + rmtAddr + ", locAddr=" + sock.getLocalSocketAddress() + ']'); LT.warn(log, "Unknown connection detected (is some other software connecting to " + @@ -5779,6 +5794,11 @@ private class SocketReader extends IgniteSpiThread { if (!spi.isNodeStopping0()) { TcpDiscoveryPingRequest req = (TcpDiscoveryPingRequest)msg; + if (log.isInfoEnabled()) + log.info("Received ping request from the remote node " + + "[rmtNodeId=" + msg.creatorNodeId() + + ", rmtAddr=" + rmtAddr + ", rmtPort=" + sock.getPort() + "]"); + TcpDiscoveryPingResponse res = new TcpDiscoveryPingResponse(locNodeId); IgniteSpiOperationTimeoutHelper timeoutHelper = @@ -5792,6 +5812,12 @@ private class SocketReader extends IgniteSpiThread { } spi.writeToSocket(sock, res, timeoutHelper.nextTimeoutChunk(spi.getSocketTimeout())); + + sock.shutdownOutput(); + + if (log.isInfoEnabled()) + log.info("Finished writing ping response " + "[rmtNodeId=" + msg.creatorNodeId() + + ", rmtAddr=" + rmtAddr + ", rmtPort=" + sock.getPort() + "]"); } else if (log.isDebugEnabled()) log.debug("Ignore ping request, node is stopping."); @@ -5917,7 +5943,7 @@ else if ((X.hasCause(e, ObjectStreamException.class) || !sock.isClosed()) else if (e.hasCause(ClassNotFoundException.class)) LT.warn(log, "Failed to read message due to ClassNotFoundException " + "(make sure same versions of all classes are available on all nodes) " + - "[rmtAddr=" + sock.getRemoteSocketAddress() + + "[rmtAddr=" + rmtAddr + ", err=" + X.cause(e, ClassNotFoundException.class).getMessage() + ']'); // Always report marshalling problems. @@ -6208,6 +6234,10 @@ else if (msg instanceof TcpDiscoveryRingLatencyCheckMessage) { } U.closeQuiet(sock); + + if (log.isInfoEnabled()) + log.info("Finished serving remote node connection [rmtAddr=" + rmtAddr + + ", rmtPort=" + sock.getPort()); } } 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 cefbcd675a303..3cc4ee53242b2 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 @@ -1482,7 +1482,8 @@ protected T readMessage(Socket sock, @Nullable InputStream in, long timeout) catch (IOException | IgniteCheckedException e) { if (X.hasCause(e, SocketTimeoutException.class)) LT.warn(log, "Timed out waiting for message to be read (most probably, the reason is " + - "in long GC pauses on remote node) [curTimeout=" + timeout + ']'); + "long GC pauses on remote node) [curTimeout=" + timeout + + ", rmtAddr=" + sock.getRemoteSocketAddress() + ", rmtPort=" + sock.getPort() + ']'); throw e; } @@ -1523,8 +1524,9 @@ protected int readReceipt(Socket sock, long timeout) throws IOException { catch (SocketTimeoutException e) { LT.warn(log, "Timed out waiting for message delivery receipt (most probably, the reason is " + "in long GC pauses on remote node; consider tuning GC and increasing 'ackTimeout' " + - "configuration property). Will retry to send message with increased timeout. " + - "Current timeout: " + timeout + '.'); + "configuration property). Will retry to send message with increased timeout " + + "[currentTimeout=" + timeout + ", rmtAddr=" + sock.getRemoteSocketAddress() + + ", rmtPort=" + sock.getPort() + ']'); stats.onAckTimeout(); @@ -2066,9 +2068,11 @@ boolean cancel() { LT.warn(log, "Socket write has timed out (consider increasing " + (failureDetectionTimeoutEnabled() ? - "'IgniteConfiguration.failureDetectionTimeout' configuration property) [" + - "failureDetectionTimeout=" + failureDetectionTimeout() + ']' : - "'sockTimeout' configuration property) [sockTimeout=" + sockTimeout + ']')); + "'IgniteConfiguration.failureDetectionTimeout' configuration property) [" + + "failureDetectionTimeout=" + failureDetectionTimeout() : + "'sockTimeout' configuration property) [sockTimeout=" + sockTimeout) + + ", rmtAddr=" + sock.getRemoteSocketAddress() + ", rmtPort=" + sock.getPort() + + ", sockTimeout=" + sockTimeout + ']'); stats.onSocketTimeout(); } From beb2409cfe2045789443d47de735d879961d371e Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Fri, 23 Jun 2017 12:26:06 +0300 Subject: [PATCH 43/90] GG-12352: Forcible node drop makes cluster instable in some cases. Disable forcible node drop by default. --- .../apache/ignite/IgniteSystemProperties.java | 6 +++ .../service/GridServiceProcessor.java | 2 +- .../spi/IgniteSpiOperationTimeoutHelper.java | 6 ++- .../tcp/TcpCommunicationSpi.java | 46 +++++++++++++------ .../ignite/spi/discovery/tcp/ServerImpl.java | 6 ++- .../tcp/TcpCommunicationSpiDropNodesTest.java | 16 +++++++ .../TcpCommunicationSpiFaultyClientTest.java | 15 ++++++ 7 files changed, 79 insertions(+), 18 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java index 775aae03a1ef4..f473c51e74a93 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -533,6 +533,12 @@ public final class IgniteSystemProperties { */ public static final String IGNITE_SECURITY_COMPATIBILITY_MODE = "IGNITE_SECURITY_COMPATIBILITY_MODE"; + /** + * If this property is set, a node will forcible fail a remote node when it fails to establish a communication + * connection. + */ + public static final String IGNITE_ENABLE_FORCIBLE_NODE_KILL = "IGNITE_ENABLE_FORCIBLE_NODE_KILL"; + /** * Enforces singleton. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java index 4ecf674a48420..75c79a3a31fdc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java @@ -1604,7 +1604,7 @@ private class TopologyListener implements GridLocalEventListener { private volatile AffinityTopologyVersion currTopVer = null; /** {@inheritDoc} */ - @Override public void onEvent(Event evt) { + @Override public void onEvent(final Event evt) { if (!busyLock.enterBusy()) return; diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiOperationTimeoutHelper.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiOperationTimeoutHelper.java index e17b0dd1b196f..1d9fa94f44475 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiOperationTimeoutHelper.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiOperationTimeoutHelper.java @@ -96,7 +96,9 @@ public boolean checkFailureTimeoutReached(Exception e) { if (!failureDetectionTimeoutEnabled) return false; - return e instanceof IgniteSpiOperationTimeoutException || e instanceof SocketTimeoutException || - X.hasCause(e, IgniteSpiOperationTimeoutException.class, SocketException.class); + if (X.hasCause(e, IgniteSpiOperationTimeoutException.class, SocketTimeoutException.class)) + return true; + + return X.hasCause(e, SocketException.class) && (timeout - (U.currentTimeMillis() - lastOperStartTs) <= 0); } } \ No newline at end of file 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 80cb77633b844..98183d8afbd7a 100644 --- 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 @@ -23,6 +23,7 @@ import java.net.ConnectException; import java.net.InetAddress; import java.net.InetSocketAddress; +import java.net.SocketException; import java.net.SocketTimeoutException; import java.nio.ByteBuffer; import java.nio.ByteOrder; @@ -53,6 +54,7 @@ import org.apache.ignite.IgniteClientDisconnectedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; +import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.AddressResolver; import org.apache.ignite.configuration.IgniteConfiguration; @@ -328,6 +330,10 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter /** */ private ConnectGateway connectGate; + /** */ + private boolean enableForcibleNodeKill = IgniteSystemProperties + .getBoolean(IgniteSystemProperties.IGNITE_ENABLE_FORCIBLE_NODE_KILL); + /** Server listener. */ private final GridNioServerListener srvLsnr = new GridNioServerListenerAdapter() { @@ -2073,8 +2079,13 @@ private GridCommunicationClient reserveClient(ClusterNode node) throws IgniteChe } } } - else + else { U.sleep(200); + + if (getSpiContext().node(node.id()) == null) + throw new ClusterTopologyCheckedException("Failed to send message " + + "(node left topology): " + node); + } } fut.onDone(client0); @@ -2470,6 +2481,7 @@ protected GridCommunicationClient createTcpClient(ClusterNode node) throws Ignit } if (failureDetectionTimeoutEnabled() && (e instanceof HandshakeTimeoutException || + X.hasCause(e, SocketException.class) || timeoutHelper.checkFailureTimeoutReached(e))) { String msg = "Handshake timed out (failure detection timeout is reached) " + @@ -2561,8 +2573,10 @@ else if (X.hasCause(e, SocketTimeoutException.class)) errs.addSuppressed(new IgniteCheckedException("Failed to connect to address: " + addr, e)); // Reconnect for the second time, if connection is not established. - if (!failureDetThrReached && connectAttempts < 2 && - (e instanceof ConnectException || X.hasCause(e, ConnectException.class))) { + if (!failureDetThrReached && connectAttempts < 5 && + (X.hasCause(e, ConnectException.class) || X.hasCause(e, SocketTimeoutException.class))) { + U.sleep(200); + connectAttempts++; continue; @@ -2585,19 +2599,22 @@ else if (X.hasCause(e, SocketTimeoutException.class)) "operating system firewall is disabled on local and remote hosts) " + "[addrs=" + addrs + ']'); - if (getSpiContext().node(node.id()) != null && (CU.clientNode(node) || !CU.clientNode(getLocalNode())) && + if (enableForcibleNodeKill) { + if (getSpiContext().node(node.id()) != null && (CU.clientNode(node) || !CU.clientNode(getLocalNode())) && X.hasCause(errs, ConnectException.class, SocketTimeoutException.class, HandshakeTimeoutException.class, IgniteSpiOperationTimeoutException.class)) { U.error(log, "TcpCommunicationSpi failed to establish connection to node, node will be dropped from " + "cluster [" + "rmtNode=" + node + ']', errs); - getSpiContext().failNode(node.id(), "TcpCommunicationSpi failed to establish connection to node [" + - "rmtNode=" + node + - ", errs=" + errs + - ", connectErrs=" + Arrays.toString(errs.getSuppressed()) + ']'); + getSpiContext().failNode(node.id(), "TcpCommunicationSpi failed to establish connection to node [" + + "rmtNode=" + node + + ", errs=" + errs + + ", connectErrs=" + Arrays.toString(errs.getSuppressed()) + ']'); + } } - throw errs; + if (X.hasCause(errs, ConnectException.class)) + throw errs; } return client; @@ -2829,8 +2846,9 @@ else if (log.isDebugEnabled()) // Ignoring whatever happened after timeout - reporting only timeout event. if (!cancelled) - throw new HandshakeTimeoutException("Failed to perform handshake due to timeout (consider increasing " + - "'connectionTimeout' configuration property)."); + throw new HandshakeTimeoutException( + new IgniteSpiOperationTimeoutException("Failed to perform handshake due to timeout " + + "(consider increasing 'connectionTimeout' configuration property).")); } return rcvCnt; @@ -2980,10 +2998,10 @@ private static class HandshakeTimeoutException extends IgniteCheckedException { private static final long serialVersionUID = 0L; /** - * @param msg Message. + * @param cause Exception cause */ - HandshakeTimeoutException(String msg) { - super(msg); + HandshakeTimeoutException(IgniteSpiOperationTimeoutException cause) { + super(cause); } } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java index 2f7e9b470aae6..856370c2a0ee3 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java @@ -1146,6 +1146,8 @@ else if (U.currentTimeMillis() - noResStart > spi.joinTimeout) boolean openSock = false; + boolean wasHandshake = false; + Socket sock = null; try { @@ -1163,6 +1165,8 @@ else if (U.currentTimeMillis() - noResStart > spi.joinTimeout) TcpDiscoveryHandshakeResponse res = spi.readMessage(sock, null, timeoutHelper.nextTimeoutChunk( ackTimeout0)); + wasHandshake = true; + if (msg instanceof TcpDiscoveryJoinRequestMessage) { boolean ignore = false; @@ -1264,7 +1268,7 @@ else if (U.currentTimeMillis() - noResStart > spi.joinTimeout) if (!spi.failureDetectionTimeoutEnabled() && ++reconCnt == spi.getReconnectCount()) break; - if (!openSock) { + if (!openSock || !wasHandshake) { // Reconnect for the second time, if connection is not established. if (connectAttempts < 2) { connectAttempts++; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiDropNodesTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiDropNodesTest.java index d29231e449b0b..7b438d6795a8b 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiDropNodesTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiDropNodesTest.java @@ -25,6 +25,7 @@ import java.util.concurrent.CyclicBarrier; import java.util.concurrent.TimeUnit; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.events.Event; @@ -41,6 +42,7 @@ 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.config.GridTestProperties; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import static org.apache.ignite.events.EventType.EVT_NODE_FAILED; @@ -82,6 +84,20 @@ public class TcpCommunicationSpiDropNodesTest extends GridCommonAbstractTest { return cfg; } + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + System.setProperty(IgniteSystemProperties.IGNITE_ENABLE_FORCIBLE_NODE_KILL,"true"); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + System.clearProperty(IgniteSystemProperties.IGNITE_ENABLE_FORCIBLE_NODE_KILL); + } + /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { super.beforeTest(); diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiFaultyClientTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiFaultyClientTest.java index 6e99487bd46ae..33625e4b9edb4 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiFaultyClientTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiFaultyClientTest.java @@ -30,6 +30,7 @@ import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.events.Event; @@ -95,6 +96,20 @@ public class TcpCommunicationSpiFaultyClientTest extends GridCommonAbstractTest return cfg; } + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + System.setProperty(IgniteSystemProperties.IGNITE_ENABLE_FORCIBLE_NODE_KILL,"true"); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + System.clearProperty(IgniteSystemProperties.IGNITE_ENABLE_FORCIBLE_NODE_KILL); + } + /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { super.beforeTest(); From 802f18fc250cbae8959192c78bb28dc525ed3cf7 Mon Sep 17 00:00:00 2001 From: AMRepo Date: Fri, 23 Jun 2017 00:24:57 +0300 Subject: [PATCH 44/90] Fix compilation --- .../internal/processors/service/GridServiceProcessor.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java index 75c79a3a31fdc..009bf47c57101 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java @@ -1846,7 +1846,7 @@ private void undeploy(String name) { if (ctxs != null) { synchronized (ctxs) { if (log.isInfoEnabled()) - log.info("Undeploying services [svc=" + e.getKey().name() + ", assigns=" + assigns + + log.info("Undeploying services [svc=" + name + ", ctxs=" + ctxs + ']');cancel(ctxs, ctxs.size()); } From bfec212b1ece0e9e791de6dfb642324834fa77ca Mon Sep 17 00:00:00 2001 From: AMRepo Date: Fri, 23 Jun 2017 00:24:57 +0300 Subject: [PATCH 45/90] Partially reverted GG-12352. --- .../spi/IgniteSpiOperationTimeoutHelper.java | 4 +-- .../tcp/TcpCommunicationSpi.java | 1 - .../ignite/spi/discovery/tcp/ServerImpl.java | 6 +---- .../apache/ignite/spi/GridTcpForwarder.java | 26 +++++++++++++++++++ 4 files changed, 29 insertions(+), 8 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiOperationTimeoutHelper.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiOperationTimeoutHelper.java index 1d9fa94f44475..33896361e2c21 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiOperationTimeoutHelper.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiOperationTimeoutHelper.java @@ -96,9 +96,9 @@ public boolean checkFailureTimeoutReached(Exception e) { if (!failureDetectionTimeoutEnabled) return false; - if (X.hasCause(e, IgniteSpiOperationTimeoutException.class, SocketTimeoutException.class)) + if (X.hasCause(e, IgniteSpiOperationTimeoutException.class, SocketTimeoutException.class, SocketException.class)) return true; - return X.hasCause(e, SocketException.class) && (timeout - (U.currentTimeMillis() - lastOperStartTs) <= 0); + return (timeout - (U.currentTimeMillis() - lastOperStartTs) <= 0); } } \ No newline at end of file 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 98183d8afbd7a..a87734e5c73e4 100644 --- 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 @@ -2481,7 +2481,6 @@ protected GridCommunicationClient createTcpClient(ClusterNode node) throws Ignit } if (failureDetectionTimeoutEnabled() && (e instanceof HandshakeTimeoutException || - X.hasCause(e, SocketException.class) || timeoutHelper.checkFailureTimeoutReached(e))) { String msg = "Handshake timed out (failure detection timeout is reached) " + diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java index 856370c2a0ee3..2f7e9b470aae6 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java @@ -1146,8 +1146,6 @@ else if (U.currentTimeMillis() - noResStart > spi.joinTimeout) boolean openSock = false; - boolean wasHandshake = false; - Socket sock = null; try { @@ -1165,8 +1163,6 @@ else if (U.currentTimeMillis() - noResStart > spi.joinTimeout) TcpDiscoveryHandshakeResponse res = spi.readMessage(sock, null, timeoutHelper.nextTimeoutChunk( ackTimeout0)); - wasHandshake = true; - if (msg instanceof TcpDiscoveryJoinRequestMessage) { boolean ignore = false; @@ -1268,7 +1264,7 @@ else if (U.currentTimeMillis() - noResStart > spi.joinTimeout) if (!spi.failureDetectionTimeoutEnabled() && ++reconCnt == spi.getReconnectCount()) break; - if (!openSock || !wasHandshake) { + if (!openSock) { // Reconnect for the second time, if connection is not established. if (connectAttempts < 2) { connectAttempts++; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/GridTcpForwarder.java b/modules/core/src/test/java/org/apache/ignite/spi/GridTcpForwarder.java index d08321e18e8b5..68d97c1a3932b 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/GridTcpForwarder.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/GridTcpForwarder.java @@ -23,6 +23,7 @@ import java.net.InetAddress; import java.net.ServerSocket; import java.net.Socket; +import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.internal.util.typedef.internal.U; @@ -85,6 +86,10 @@ public GridTcpForwarder( outputCon.getInputStream(), inputCon.getOutputStream() ); + //Force closing sibling if one of thread failed. + forwardThread1.setUncaughtExceptionHandler(new ForwarderExceptionHandler(forwardThread2)); + forwardThread2.setUncaughtExceptionHandler(new ForwarderExceptionHandler(forwardThread1)); + forwardThread1.start(); forwardThread2.start(); @@ -127,6 +132,25 @@ public GridTcpForwarder( U.join(mainThread, log); } + /** + * + */ + private static class ForwarderExceptionHandler implements Thread.UncaughtExceptionHandler { + /** */ + private Thread siblingThread; + + /** */ + public ForwarderExceptionHandler(Thread siblingThread) { + + this.siblingThread = siblingThread; + } + + /** */ + @Override public void uncaughtException(Thread t, Throwable e) { + siblingThread.interrupt(); + } + } + /** * Thread reads data from input stream and write to output stream. */ @@ -166,6 +190,8 @@ private ForwardThread(String name, InputStream inputStream, OutputStream outputS } catch (IOException e) { log.error("IOException while forwarding data [threadName=" + getName() + "]", e); + + throw new IgniteException(e); } } } From 1abc14fdc4e39d8245c3e50fb2cf3d183df08021 Mon Sep 17 00:00:00 2001 From: AMRepo Date: Fri, 23 Jun 2017 00:24:57 +0300 Subject: [PATCH 46/90] Partially reverted GG-12352. --- .../service/GridServiceProcessor.java | 2 +- .../spi/IgniteSpiOperationTimeoutHelper.java | 4 +-- .../tcp/TcpCommunicationSpi.java | 2 -- .../ignite/spi/discovery/tcp/ServerImpl.java | 6 +---- .../apache/ignite/spi/GridTcpForwarder.java | 26 +++++++++++++++++++ .../tcp/TcpCommunicationSpiDropNodesTest.java | 1 - .../TcpCommunicationSpiFaultyClientTest.java | 5 ---- 7 files changed, 30 insertions(+), 16 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java index 009bf47c57101..20bcff7c780d3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java @@ -1657,7 +1657,7 @@ else if (msg instanceof DynamicCacheChangeBatch) { log.info("Service processor detected a topology change during " + "assignments calculation (will abort current iteration and " + "re-calculate on the newer version): " + - "[topVer=" + topVer + ", newTopVer=" + currTopVer + ']'); + "[topVer=" + topVer + ", newTopVer=" + currTopVer0 + ']'); return; } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiOperationTimeoutHelper.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiOperationTimeoutHelper.java index 1d9fa94f44475..33896361e2c21 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiOperationTimeoutHelper.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiOperationTimeoutHelper.java @@ -96,9 +96,9 @@ public boolean checkFailureTimeoutReached(Exception e) { if (!failureDetectionTimeoutEnabled) return false; - if (X.hasCause(e, IgniteSpiOperationTimeoutException.class, SocketTimeoutException.class)) + if (X.hasCause(e, IgniteSpiOperationTimeoutException.class, SocketTimeoutException.class, SocketException.class)) return true; - return X.hasCause(e, SocketException.class) && (timeout - (U.currentTimeMillis() - lastOperStartTs) <= 0); + return (timeout - (U.currentTimeMillis() - lastOperStartTs) <= 0); } } \ No newline at end of file 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 98183d8afbd7a..1bfa56accf2e0 100644 --- 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 @@ -23,7 +23,6 @@ import java.net.ConnectException; import java.net.InetAddress; import java.net.InetSocketAddress; -import java.net.SocketException; import java.net.SocketTimeoutException; import java.nio.ByteBuffer; import java.nio.ByteOrder; @@ -2481,7 +2480,6 @@ protected GridCommunicationClient createTcpClient(ClusterNode node) throws Ignit } if (failureDetectionTimeoutEnabled() && (e instanceof HandshakeTimeoutException || - X.hasCause(e, SocketException.class) || timeoutHelper.checkFailureTimeoutReached(e))) { String msg = "Handshake timed out (failure detection timeout is reached) " + diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java index 856370c2a0ee3..2f7e9b470aae6 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java @@ -1146,8 +1146,6 @@ else if (U.currentTimeMillis() - noResStart > spi.joinTimeout) boolean openSock = false; - boolean wasHandshake = false; - Socket sock = null; try { @@ -1165,8 +1163,6 @@ else if (U.currentTimeMillis() - noResStart > spi.joinTimeout) TcpDiscoveryHandshakeResponse res = spi.readMessage(sock, null, timeoutHelper.nextTimeoutChunk( ackTimeout0)); - wasHandshake = true; - if (msg instanceof TcpDiscoveryJoinRequestMessage) { boolean ignore = false; @@ -1268,7 +1264,7 @@ else if (U.currentTimeMillis() - noResStart > spi.joinTimeout) if (!spi.failureDetectionTimeoutEnabled() && ++reconCnt == spi.getReconnectCount()) break; - if (!openSock || !wasHandshake) { + if (!openSock) { // Reconnect for the second time, if connection is not established. if (connectAttempts < 2) { connectAttempts++; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/GridTcpForwarder.java b/modules/core/src/test/java/org/apache/ignite/spi/GridTcpForwarder.java index d08321e18e8b5..68d97c1a3932b 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/GridTcpForwarder.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/GridTcpForwarder.java @@ -23,6 +23,7 @@ import java.net.InetAddress; import java.net.ServerSocket; import java.net.Socket; +import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.internal.util.typedef.internal.U; @@ -85,6 +86,10 @@ public GridTcpForwarder( outputCon.getInputStream(), inputCon.getOutputStream() ); + //Force closing sibling if one of thread failed. + forwardThread1.setUncaughtExceptionHandler(new ForwarderExceptionHandler(forwardThread2)); + forwardThread2.setUncaughtExceptionHandler(new ForwarderExceptionHandler(forwardThread1)); + forwardThread1.start(); forwardThread2.start(); @@ -127,6 +132,25 @@ public GridTcpForwarder( U.join(mainThread, log); } + /** + * + */ + private static class ForwarderExceptionHandler implements Thread.UncaughtExceptionHandler { + /** */ + private Thread siblingThread; + + /** */ + public ForwarderExceptionHandler(Thread siblingThread) { + + this.siblingThread = siblingThread; + } + + /** */ + @Override public void uncaughtException(Thread t, Throwable e) { + siblingThread.interrupt(); + } + } + /** * Thread reads data from input stream and write to output stream. */ @@ -166,6 +190,8 @@ private ForwardThread(String name, InputStream inputStream, OutputStream outputS } catch (IOException e) { log.error("IOException while forwarding data [threadName=" + getName() + "]", e); + + throw new IgniteException(e); } } } diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiDropNodesTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiDropNodesTest.java index 7b438d6795a8b..58a0dc170f9d9 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiDropNodesTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiDropNodesTest.java @@ -42,7 +42,6 @@ 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.config.GridTestProperties; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import static org.apache.ignite.events.EventType.EVT_NODE_FAILED; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiFaultyClientTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiFaultyClientTest.java index 33625e4b9edb4..dcfa91d6431a9 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiFaultyClientTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiFaultyClientTest.java @@ -18,16 +18,13 @@ package org.apache.ignite.spi.communication.tcp; import java.io.IOException; -import java.io.OutputStream; import java.net.InetAddress; import java.net.ServerSocket; -import java.net.Socket; import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; -import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteSystemProperties; @@ -45,8 +42,6 @@ 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.spi.discovery.tcp.messages.TcpDiscoveryAbstractMessage; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeFailedMessage; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; From 897f4c00c4945eda3f9f4a41d064ded2f6f27ccc Mon Sep 17 00:00:00 2001 From: sboikov Date: Tue, 27 Jun 2017 12:55:11 +0300 Subject: [PATCH 47/90] gg-12133 Deadlock for metadata update from GridCacheMapEntry.innerUpdate --- .../checkpoint/GridCheckpointManager.java | 2 +- .../managers/communication/GridIoManager.java | 10 +- .../communication/GridMessageListener.java | 3 +- .../GridDeploymentCommunication.java | 4 +- .../eventstorage/GridEventStorageManager.java | 4 +- .../processors/cache/GridCacheIoManager.java | 37 ++-- .../cache/transactions/IgniteTxManager.java | 2 +- .../clock/GridClockSyncProcessor.java | 2 +- .../continuous/GridContinuousProcessor.java | 4 +- .../datastreamer/DataStreamProcessor.java | 2 +- .../datastreamer/DataStreamerImpl.java | 2 +- .../processors/igfs/IgfsDataManager.java | 2 +- .../igfs/IgfsFragmentizerManager.java | 4 +- .../processors/job/GridJobProcessor.java | 8 +- .../handlers/task/GridTaskCommandHandler.java | 4 +- .../processors/task/GridTaskProcessor.java | 6 +- .../jobstealing/JobStealingCollisionSpi.java | 2 +- .../TestRecordingCommunicationSpi.java | 29 +++ ...CommunicationManagerListenersSelfTest.java | 2 +- .../GridCommunicationSendMessageSelfTest.java | 2 +- .../GridCachePartitionedGetSelfTest.java | 2 +- .../IgniteBinaryMetadataUpdateFromInvoke.java | 187 ++++++++++++++++++ .../communication/GridIoManagerBenchmark.java | 4 +- .../GridIoManagerBenchmark0.java | 12 +- .../GridCacheMessageSelfTest.java | 2 +- .../testframework/GridSpiTestContext.java | 4 +- .../IgniteCacheRestartTestSuite2.java | 2 + .../query/h2/opt/GridH2IndexBase.java | 2 +- .../h2/twostep/GridMapQueryExecutor.java | 2 +- .../h2/twostep/GridReduceQueryExecutor.java | 2 +- 30 files changed, 289 insertions(+), 61 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryMetadataUpdateFromInvoke.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/checkpoint/GridCheckpointManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/checkpoint/GridCheckpointManager.java index 9124cafa79a6b..40c2c636ace93 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/checkpoint/GridCheckpointManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/checkpoint/GridCheckpointManager.java @@ -449,7 +449,7 @@ private class CheckpointRequestListener implements GridMessageListener { * @param msg Received message. */ @SuppressWarnings({"MismatchedQueryAndUpdateOfCollection"}) - @Override public void onMessage(UUID nodeId, Object msg) { + @Override public void onMessage(UUID nodeId, Object msg, byte plc) { GridCheckpointRequest req = (GridCheckpointRequest)msg; if (log.isDebugEnabled()) 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 4b68e5bdd0da7..a4e32980114e2 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 @@ -321,7 +321,7 @@ public void resetMetrics() { log.debug(startInfo()); addMessageListener(TOPIC_IO_TEST, new GridMessageListener() { - @Override public void onMessage(UUID nodeId, Object msg) { + @Override public void onMessage(UUID nodeId, Object msg, byte plc) { ClusterNode node = ctx.discovery().node(nodeId); if (node == null) @@ -1508,7 +1508,7 @@ private void invokeListener(Byte plc, GridMessageListener lsnr, UUID nodeId, Obj CUR_PLC.set(plc); try { - lsnr.onMessage(nodeId, msg); + lsnr.onMessage(nodeId, msg, plc); } finally { if (change) @@ -2333,14 +2333,14 @@ private static class ArrayListener implements GridMessageListener { * @param nodeId Node ID. * @param msg Message. */ - @Override public void onMessage(UUID nodeId, Object msg) { + @Override public void onMessage(UUID nodeId, Object msg, byte plc) { GridMessageListener[] arr0 = arr; if (arr0 == null) return; for (GridMessageListener l : arr0) - l.onMessage(nodeId, msg); + l.onMessage(nodeId, msg, plc); } /** @@ -2440,7 +2440,7 @@ private class GridUserMessageListener implements GridMessageListener { /** {@inheritDoc} */ @SuppressWarnings({"SynchronizationOnLocalVariableOrMethodParameter", "ConstantConditions", "OverlyStrongTypeCast"}) - @Override public void onMessage(UUID nodeId, Object msg) { + @Override public void onMessage(UUID nodeId, Object msg, byte plc) { if (!(msg instanceof GridIoUserMessage)) { U.error(log, "Received unknown message (potentially fatal problem): " + msg); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridMessageListener.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridMessageListener.java index 39935917c3e65..c7de57c959c18 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridMessageListener.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridMessageListener.java @@ -30,6 +30,7 @@ public interface GridMessageListener extends EventListener { * @param nodeId ID of node that sent the message. Note that may have already * left topology by the time this message is received. * @param msg Message received. + * @param plc Message policy (pool). */ - public void onMessage(UUID nodeId, Object msg); + public void onMessage(UUID nodeId, Object msg, byte plc); } \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentCommunication.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentCommunication.java index a571ae445438f..661bf68b61a42 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentCommunication.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentCommunication.java @@ -82,7 +82,7 @@ class GridDeploymentCommunication { this.log = log.getLogger(getClass()); peerLsnr = new GridMessageListener() { - @Override public void onMessage(UUID nodeId, Object msg) { + @Override public void onMessage(UUID nodeId, Object msg, byte plc) { processDeploymentRequest(nodeId, msg); } }; @@ -416,7 +416,7 @@ GridDeploymentResponse sendResourceRequest(final String rsrcName, IgniteUuid cls }; GridMessageListener resLsnr = new GridMessageListener() { - @Override public void onMessage(UUID nodeId, Object msg) { + @Override public void onMessage(UUID nodeId, Object msg, byte plc) { assert nodeId != null; assert msg != null; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java index 607bb9688aaf1..b77ec27496c50 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java @@ -877,7 +877,7 @@ private List query(IgnitePredicate p, Collection>() { @Override public void apply(IgniteInternalFuture t) { - cctx.kernalContext().closure().runLocalSafe(new Runnable() { - @Override public void run() { - IgniteLogger log = cacheMsg.messageLogger(cctx); + try { + cctx.kernalContext().pools().poolForPolicy(plc).execute(new Runnable() { + @Override public void run() { + IgniteLogger log = cacheMsg.messageLogger(cctx); - if (log.isDebugEnabled()) { - StringBuilder msg0 = new StringBuilder("Process cache message after wait for " + - "affinity topology version ["); + if (log.isDebugEnabled()) { + StringBuilder msg0 = new StringBuilder("Process cache message after wait for " + + "affinity topology version ["); - appendMessageInfo(cacheMsg, nodeId, msg0).append(']'); + appendMessageInfo(cacheMsg, nodeId, msg0).append(']'); - log.debug(msg0.toString()); - } + log.debug(msg0.toString()); + } - handleMessage(nodeId, cacheMsg); - } - }); + handleMessage(nodeId, cacheMsg); + } + }); + } + catch (IgniteCheckedException e) { + U.error(cacheMsg.messageLogger(cctx), "Failed to get pool for policy: " + plc, e); + } } }); @@ -1336,7 +1345,7 @@ private class OrderedMessageListener implements GridMessageListener { /** {@inheritDoc} */ @SuppressWarnings({"CatchGenericClass", "unchecked"}) - @Override public void onMessage(final UUID nodeId, Object msg) { + @Override public void onMessage(final UUID nodeId, Object msg, byte plc) { if (log.isDebugEnabled()) log.debug("Received cache ordered message [nodeId=" + nodeId + ", msg=" + msg + ']'); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java index 2c02f967d00be..2f21614352e8c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java @@ -2472,7 +2472,7 @@ private CommitListener(IgniteInternalTx tx) { private class DeadlockDetectionListener implements GridMessageListener { /** {@inheritDoc} */ @SuppressWarnings("unchecked") - @Override public void onMessage(UUID nodeId, Object msg) { + @Override public void onMessage(UUID nodeId, Object msg, byte plc) { GridCacheMessage cacheMsg = (GridCacheMessage)msg; unmarshall(nodeId, cacheMsg); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockSyncProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockSyncProcessor.java index 07643164368ce..3586956da99dd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockSyncProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockSyncProcessor.java @@ -96,7 +96,7 @@ public GridClockSyncProcessor(GridKernalContext ctx) { srv.start(ctx); ctx.io().addMessageListener(TOPIC_TIME_SYNC, new GridMessageListener() { - @Override public void onMessage(UUID nodeId, Object msg) { + @Override public void onMessage(UUID nodeId, Object msg, byte plc) { assert msg instanceof GridClockDeltaSnapshotMessage; GridClockDeltaSnapshotMessage msg0 = (GridClockDeltaSnapshotMessage)msg; 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 9fd9b6d44b89e..f0429bc3a3c91 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 @@ -288,7 +288,7 @@ public GridContinuousProcessor(GridKernalContext ctx) { }); ctx.io().addMessageListener(TOPIC_CONTINUOUS, new GridMessageListener() { - @Override public void onMessage(UUID nodeId, Object obj) { + @Override public void onMessage(UUID nodeId, Object obj, byte plc) { GridContinuousMessage msg = (GridContinuousMessage)obj; if (msg.data() == null && msg.dataBytes() != null) { @@ -721,7 +721,7 @@ public IgniteInternalFuture startRoutine(GridContinuousHandler hnd, private void registerMessageListener(GridContinuousHandler hnd) { if (hnd.orderedTopic() != null) { ctx.io().addMessageListener(hnd.orderedTopic(), new GridMessageListener() { - @Override public void onMessage(UUID nodeId, Object obj) { + @Override public void onMessage(UUID nodeId, Object obj, byte plc) { GridContinuousMessage msg = (GridContinuousMessage)obj; // Only notification can be ordered. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java index fee4dd6616b28..6f35a52d5b673 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java @@ -80,7 +80,7 @@ public DataStreamProcessor(GridKernalContext ctx) { if (!ctx.clientNode()) { ctx.io().addMessageListener(TOPIC_DATASTREAM, new GridMessageListener() { - @Override public void onMessage(UUID nodeId, Object msg) { + @Override public void onMessage(UUID nodeId, Object msg, byte plc) { assert msg instanceof DataStreamerRequest; processRequest(nodeId, (DataStreamerRequest)msg); 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 bb9ffdd5795a2..515314eb08728 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 @@ -321,7 +321,7 @@ public DataStreamerImpl( topic = TOPIC_DATASTREAM.topic(IgniteUuid.fromUuid(ctx.localNodeId())); ctx.io().addMessageListener(topic, new GridMessageListener() { - @Override public void onMessage(UUID nodeId, Object msg) { + @Override public void onMessage(UUID nodeId, Object msg, byte plc) { assert msg instanceof DataStreamerResponse; DataStreamerResponse res = (DataStreamerResponse)msg; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java index e534800efd7a6..59e1b724764e8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java @@ -153,7 +153,7 @@ void awaitInit() { topic = F.isEmpty(igfsName) ? TOPIC_IGFS : TOPIC_IGFS.topic(igfsName); igfsCtx.kernalContext().io().addMessageListener(topic, new GridMessageListener() { - @Override public void onMessage(UUID nodeId, Object msg) { + @Override public void onMessage(UUID nodeId, Object msg, byte plc) { if (msg instanceof IgfsBlocksMessage) processBlocksMessage(nodeId, (IgfsBlocksMessage)msg); else if (msg instanceof IgfsAckMessage) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerManager.java index 2e82f33024b57..f76b877f38443 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerManager.java @@ -453,7 +453,7 @@ protected FragmentizerCoordinator() { } /** {@inheritDoc} */ - @Override public void onMessage(UUID nodeId, Object msg) { + @Override public void onMessage(UUID nodeId, Object msg, byte plc) { if (msg instanceof IgfsFragmentizerResponse) { IgfsFragmentizerResponse res = (IgfsFragmentizerResponse)msg; @@ -673,7 +673,7 @@ protected FragmentizerWorker() { } /** {@inheritDoc} */ - @Override public void onMessage(UUID nodeId, Object msg) { + @Override public void onMessage(UUID nodeId, Object msg, byte plc) { if (msg instanceof IgfsFragmentizerRequest || msg instanceof IgfsSyncMessage) { if (log.isDebugEnabled()) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobProcessor.java index ea9cbd7dbd272..eb3300ce3412c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobProcessor.java @@ -444,7 +444,7 @@ public Collection requestJobSiblings( final Condition cond = lock.newCondition(); GridMessageListener msgLsnr = new GridMessageListener() { - @Override public void onMessage(UUID nodeId, Object msg) { + @Override public void onMessage(UUID nodeId, Object msg, byte plc) { String err = null; GridJobSiblingsResponse res = null; @@ -1842,7 +1842,7 @@ private class JobHoldListener implements GridJobHoldListener { */ private class JobSessionListener implements GridMessageListener { /** {@inheritDoc} */ - @Override public void onMessage(UUID nodeId, Object msg) { + @Override public void onMessage(UUID nodeId, Object msg, byte plc) { assert nodeId != null; assert msg != null; @@ -1858,7 +1858,7 @@ private class JobSessionListener implements GridMessageListener { */ private class JobCancelListener implements GridMessageListener { /** {@inheritDoc} */ - @Override public void onMessage(UUID nodeId, Object msg) { + @Override public void onMessage(UUID nodeId, Object msg, byte plc) { assert nodeId != null; assert msg != null; @@ -1876,7 +1876,7 @@ private class JobCancelListener implements GridMessageListener { */ private class JobExecutionListener implements GridMessageListener { /** {@inheritDoc} */ - @Override public void onMessage(UUID nodeId, Object msg) { + @Override public void onMessage(UUID nodeId, Object msg, byte plc) { assert nodeId != null; assert msg != null; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskCommandHandler.java index 947435cb58d83..88f8d4de9241e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskCommandHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskCommandHandler.java @@ -109,7 +109,7 @@ public GridTaskCommandHandler(final GridKernalContext ctx) { super(ctx); ctx.io().addMessageListener(TOPIC_REST, new GridMessageListener() { - @Override public void onMessage(UUID nodeId, Object msg) { + @Override public void onMessage(UUID nodeId, Object msg, byte plc) { if (!(msg instanceof GridTaskResultRequest)) { U.warn(log, "Received unexpected message instead of task result request: " + msg); @@ -425,7 +425,7 @@ private IgniteBiTuple requestTaskResult(final UU final Condition cond = lock.newCondition(); GridMessageListener msgLsnr = new GridMessageListener() { - @Override public void onMessage(UUID nodeId, Object msg) { + @Override public void onMessage(UUID nodeId, Object msg, byte plc) { String err = null; GridTaskResultResponse res = null; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java index 12213581b3286..a807ad9efe5dc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java @@ -1225,7 +1225,7 @@ private JobMessageListener(boolean jobResOnly) { } /** {@inheritDoc} */ - @Override public void onMessage(UUID nodeId, Object msg) { + @Override public void onMessage(UUID nodeId, Object msg, byte plc) { if (msg instanceof GridJobExecuteResponse) processJobExecuteResponse(nodeId, (GridJobExecuteResponse)msg); else if (jobResOnly) @@ -1269,7 +1269,7 @@ private class TaskDiscoveryListener implements GridLocalEventListener { */ private class JobSiblingsMessageListener implements GridMessageListener { /** {@inheritDoc} */ - @Override public void onMessage(UUID nodeId, Object msg) { + @Override public void onMessage(UUID nodeId, Object msg, byte plc) { if (!(msg instanceof GridJobSiblingsRequest)) { U.warn(log, "Received unexpected message instead of siblings request: " + msg); @@ -1341,7 +1341,7 @@ private class JobSiblingsMessageListener implements GridMessageListener { */ private class TaskCancelMessageListener implements GridMessageListener { /** {@inheritDoc} */ - @Override public void onMessage(UUID nodeId, Object msg) { + @Override public void onMessage(UUID nodeId, Object msg, byte plc) { assert msg != null; if (!(msg instanceof GridTaskCancelRequest)) { diff --git a/modules/core/src/main/java/org/apache/ignite/spi/collision/jobstealing/JobStealingCollisionSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/collision/jobstealing/JobStealingCollisionSpi.java index f778bfcc73ae8..fca249893980a 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/collision/jobstealing/JobStealingCollisionSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/collision/jobstealing/JobStealingCollisionSpi.java @@ -547,7 +547,7 @@ public void setStealingAttributes(Map stealAttrs spiCtx.addMessageListener( msgLsnr = new GridMessageListener() { - @Override public void onMessage(UUID nodeId, Object msg) { + @Override public void onMessage(UUID nodeId, Object msg, byte plc) { MessageInfo info = rcvMsgMap.get(nodeId); if (info == null) { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/TestRecordingCommunicationSpi.java b/modules/core/src/test/java/org/apache/ignite/internal/TestRecordingCommunicationSpi.java index 2aed459fedc59..17ca1a7ffc7c4 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/TestRecordingCommunicationSpi.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/TestRecordingCommunicationSpi.java @@ -88,6 +88,8 @@ public class TestRecordingCommunicationSpi extends TcpCommunicationSpi { blockedMsgs.add(new T2<>(node, ioMsg)); + this.notifyAll(); + return; } } @@ -136,6 +138,33 @@ public boolean hasBlockedMessages() { } } + /** + * @param cls Message class. + * @param nodeName Node name. + * @throws InterruptedException If interrupted. + */ + public void waitForBlocked(Class cls, String nodeName) throws InterruptedException { + synchronized (this) { + while (!hasMessage(cls, nodeName)) + wait(); + } + } + + /** + * @param cls Message class. + * @param nodeName Node name. + * @return {@code True} if has blocked message. + */ + private boolean hasMessage(Class cls, String nodeName) { + for (T2 msg : blockedMsgs) { + if (msg.get2().message().getClass() == cls && + nodeName.equals(msg.get1().attribute(ATTR_GRID_NAME))) + return true; + } + + return false; + } + /** * @param blockP Message block predicate. */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridCommunicationManagerListenersSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridCommunicationManagerListenersSelfTest.java index 7613543335ce5..9289f866c8fb5 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridCommunicationManagerListenersSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridCommunicationManagerListenersSelfTest.java @@ -159,7 +159,7 @@ private static class MessageListeningTask extends ComputeTaskSplitAdapter sFut1 = GridTestUtils.runAsync(new Callable() { + @Override public Void call() throws Exception { + IgniteCache cache = ignite(META_UPDATE_FROM_NODE).cache("cache"); + + List keys = primaryKeys(cache, 1); + + b.await(); + + cache.invoke(keys.get(0), new TestEntryProcessor()); + + return null; + } + }, "async-node-0"); + + IgniteInternalFuture sFut2 = GridTestUtils.runAsync(new Callable() { + @Override public Void call() throws Exception { + IgniteCache cache = ignite(META_PRIMARY_NODE).cache("cache"); + + List keys = primaryKeys(cache, 1); + + b.await(); + + Thread.sleep(2000); + + cache.invoke(keys.get(0), new TestEntryProcessor()); + + return null; + } + }, "async-node-1"); + + IgniteInternalFuture fut2 = GridTestUtils.runAsync(new Callable() { + @Override public Void call() throws Exception { + client = true; + + b.await(); + + for (int i = 0; i < 1; i++) + startGrid(SRVS + i); + + stop.set(true); + + return null; + } + }); + + testSpi.waitForBlocked(GridNearTxPrepareRequest.class, getTestGridName(META_PRIMARY_NODE)); + + U.sleep(5000); + + testSpi.stopBlock(); + + sFut1.get(); + sFut2.get(); + fut2.get(); + + stopAllGrids(); + } + } + + /** + * + */ + static class TestEntryProcessor implements CacheEntryProcessor { + @Override public Object process(MutableEntry e, Object... args) { + e.setValue(TestEnum1.ENUM); + + return null; + } + } + + /** + * + */ + enum TestEnum1 { + /** */ + ENUM + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridIoManagerBenchmark.java b/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridIoManagerBenchmark.java index 723495cfa89c2..293e5789ddf3c 100644 --- a/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridIoManagerBenchmark.java +++ b/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridIoManagerBenchmark.java @@ -240,7 +240,7 @@ private static void receiveMessages(final IgniteKernal g) { GridMessageListener lsnr = new GridMessageListener() { private ClusterNode node; - @Override public void onMessage(UUID nodeId, Object msg) { + @Override public void onMessage(UUID nodeId, Object msg, byte plc) { if (node == null) node = g.context().discovery().node(nodeId); @@ -336,7 +336,7 @@ private ClusterNode awaitOther(final GridDiscoveryManager disc) throws Interrupt */ private static class SenderMessageListener implements GridMessageListener { /** {@inheritDoc} */ - @Override public void onMessage(UUID nodeId, Object msg) { + @Override public void onMessage(UUID nodeId, Object msg, byte plc) { msgCntr.increment(); if (testLatency) diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridIoManagerBenchmark0.java b/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridIoManagerBenchmark0.java index f2c62559397b4..83efd7de42098 100644 --- a/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridIoManagerBenchmark0.java +++ b/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridIoManagerBenchmark0.java @@ -130,7 +130,7 @@ public void testThroughput() throws Exception { rcv.addMessageListener( topic, new GridMessageListener() { - @Override public void onMessage(UUID nodeId, Object msg) { + @Override public void onMessage(UUID nodeId, Object msg, byte plc) { try { rcv.send(sndNode, topic, (Message)msg, PUBLIC_POOL); } @@ -141,7 +141,7 @@ public void testThroughput() throws Exception { }); snd.addMessageListener(topic, new GridMessageListener() { - @Override public void onMessage(UUID nodeId, Object msg) { + @Override public void onMessage(UUID nodeId, Object msg, byte plc) { msgCntr.increment(); sem.release(); @@ -224,7 +224,7 @@ public void testLatency() throws Exception { rcv.addMessageListener( topic, new GridMessageListener() { - @Override public void onMessage(UUID nodeId, Object msg) { + @Override public void onMessage(UUID nodeId, Object msg, byte plc) { try { rcv.send(sndNode, topic, (Message)msg, PUBLIC_POOL); } @@ -235,7 +235,7 @@ public void testLatency() throws Exception { }); snd.addMessageListener(topic, new GridMessageListener() { - @Override public void onMessage(UUID nodeId, Object msg) { + @Override public void onMessage(UUID nodeId, Object msg, byte plc) { map.get(((GridTestMessage)msg).id()).countDown(); } }); @@ -324,7 +324,7 @@ public void testVariableLoad() throws Exception { rcv.addMessageListener( topic, new GridMessageListener() { - @Override public void onMessage(UUID nodeId, Object msg) { + @Override public void onMessage(UUID nodeId, Object msg, byte plc) { try { rcv.send(sndNode, topic, (Message)msg, PUBLIC_POOL); } @@ -335,7 +335,7 @@ public void testVariableLoad() throws Exception { }); snd.addMessageListener(topic, new GridMessageListener() { - @Override public void onMessage(UUID nodeId, Object msg) { + @Override public void onMessage(UUID nodeId, Object msg, byte plc) { msgCntr.increment(); sem.release(); diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/GridCacheMessageSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/GridCacheMessageSelfTest.java index 9c975422a2601..2d04db2fedab9 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/GridCacheMessageSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/GridCacheMessageSelfTest.java @@ -139,7 +139,7 @@ private void doSend() throws Exception { final CountDownLatch latch = new CountDownLatch(SAMPLE_CNT); mgr1.addMessageListener(topic, new GridMessageListener() { - @Override public void onMessage(UUID nodeId, Object msg) { + @Override public void onMessage(UUID nodeId, Object msg, byte plc) { try { latch.countDown(); 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 1c8acbc3960ce..0c04039753beb 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 @@ -328,7 +328,7 @@ public Serializable removeSentMessage(ClusterNode node) { @SuppressWarnings("deprecation") public void triggerMessage(ClusterNode node, Object msg) { for (GridMessageListener lsnr : msgLsnrs) - lsnr.onMessage(node.id(), msg); + lsnr.onMessage(node.id(), msg, (byte)0); } /** {@inheritDoc} */ @@ -667,7 +667,7 @@ private class GridLocalMessageListener implements GridMessageListener { @SuppressWarnings({ "SynchronizationOnLocalVariableOrMethodParameter", "ConstantConditions", "OverlyStrongTypeCast"}) - @Override public void onMessage(UUID nodeId, Object msg) { + @Override public void onMessage(UUID nodeId, Object msg, byte plc) { GridIoUserMessage ioMsg = (GridIoUserMessage)msg; ClusterNode node = locNode; diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheRestartTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheRestartTestSuite2.java index 05137868443c8..fb38c5535b35b 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheRestartTestSuite2.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheRestartTestSuite2.java @@ -19,6 +19,7 @@ import junit.framework.TestSuite; import org.apache.ignite.internal.processors.cache.GridCachePutAllFailoverSelfTest; +import org.apache.ignite.internal.processors.cache.IgniteBinaryMetadataUpdateFromInvoke; import org.apache.ignite.internal.processors.cache.IgniteCacheAtomicPutAllFailoverSelfTest; import org.apache.ignite.internal.processors.cache.IgniteCachePutAllRestartTest; import org.apache.ignite.internal.processors.cache.distributed.IgniteBinaryMetadataUpdateNodeRestartTest; @@ -45,6 +46,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(GridCachePutAllFailoverSelfTest.class); suite.addTestSuite(IgniteBinaryMetadataUpdateNodeRestartTest.class); + suite.addTestSuite(IgniteBinaryMetadataUpdateFromInvoke.class); suite.addTestSuite(IgniteCacheGetRestartTest.class); diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java index c29239f143719..22b94c7ac796b 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java @@ -138,7 +138,7 @@ protected final void initDistributedJoinMessaging(GridH2Table tbl) { msgTopic = new IgniteBiTuple<>(GridTopic.TOPIC_QUERY, tbl.identifier() + '.' + getName()); msgLsnr = new GridMessageListener() { - @Override public void onMessage(UUID nodeId, Object msg) { + @Override public void onMessage(UUID nodeId, Object msg, byte plc) { GridSpinBusyLock l = desc.indexing().busyLock(); if (!l.enterBusy()) diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java index ac1a6a60c0200..0605287ded597 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java @@ -167,7 +167,7 @@ public void start(final GridKernalContext ctx, IgniteH2Indexing h2) throws Ignit }, EventType.EVT_NODE_FAILED, EventType.EVT_NODE_LEFT); ctx.io().addMessageListener(GridTopic.TOPIC_QUERY, new GridMessageListener() { - @Override public void onMessage(UUID nodeId, Object msg) { + @Override public void onMessage(UUID nodeId, Object msg, byte plc) { if (!busyLock.enterBusy()) return; diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java index 3f886ee686d66..71c0e1245b625 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java @@ -190,7 +190,7 @@ public void start(final GridKernalContext ctx, final IgniteH2Indexing h2) throws log = ctx.log(GridReduceQueryExecutor.class); ctx.io().addMessageListener(GridTopic.TOPIC_QUERY, new GridMessageListener() { - @Override public void onMessage(UUID nodeId, Object msg) { + @Override public void onMessage(UUID nodeId, Object msg, byte plc) { if (!busyLock.enterBusy()) return; From 90e96db22dbb7a341bfe1a8130b6dc16d5d8ae81 Mon Sep 17 00:00:00 2001 From: agura Date: Tue, 27 Jun 2017 14:43:06 +0300 Subject: [PATCH 48/90] Compilation is fixed. --- .../java/org/apache/ignite/IgniteSystemProperties.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java index f473c51e74a93..e9bbf5a10df4c 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -320,6 +320,14 @@ public final class IgniteSystemProperties { */ public static final String IGNITE_H2_DEBUG_CONSOLE = "IGNITE_H2_DEBUG_CONSOLE"; + /** + * This property allows to specify user defined port which H2 indexing SPI will use + * to start H2 debug console on. If this property is not set or set to 0, H2 debug + * console will use system-provided dynamic port. + * This property is only relevant when {@link #IGNITE_H2_DEBUG_CONSOLE} property is set. + */ + public static final String IGNITE_H2_DEBUG_CONSOLE_PORT = "IGNITE_H2_DEBUG_CONSOLE_PORT"; + /** * If this property is set to {@code true} then shared memory space native debug will be enabled. */ From 473abcafca568c7c898b0b1ae91fe964084fdf43 Mon Sep 17 00:00:00 2001 From: agura Date: Wed, 28 Jun 2017 14:49:49 +0300 Subject: [PATCH 49/90] Tests fixed --- .../IgniteCachePutRetryAbstractSelfTest.java | 51 +++---------------- 1 file changed, 6 insertions(+), 45 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java index fc9017927d3bc..0c2b868704cd3 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java @@ -57,7 +57,6 @@ import org.apache.ignite.spi.swapspace.inmemory.GridTestSwapSpaceSpi; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; -import org.jsr166.ConcurrentHashMap8; import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.CLOCK; import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC; @@ -631,71 +630,33 @@ public SetEntryProcessor(Integer val) { } } - /** - * - */ private static class TestStoreFactory implements Factory { /** {@inheritDoc} */ - @Override public CacheStore create() { - return new CacheStoreAdapter() { - @Override public Object load(Object key) throws CacheLoaderException { - return null; - } - - @Override public void write(Cache.Entry entry) throws CacheWriterException { - // No-op. - } - - @Override public void delete(Object key) throws CacheWriterException { - // No-op. - } - }; - } - } - - -/* - private static class TestStoreFactory implements Factory { - */ -/** {@inheritDoc} *//* - @Override public CacheStore create() { return new TestCacheStore(); } } - */ -/** + /** * - *//* - + */ private static class TestCacheStore extends CacheStoreAdapter { - */ -/** Store map. *//* - + /** Store map. */ private static Map STORE_MAP = new ConcurrentHashMap(); - */ -/** {@inheritDoc} *//* - + /** {@inheritDoc} */ @Override public Object load(Object key) throws CacheLoaderException { return STORE_MAP.get(key); } - */ -/** {@inheritDoc} *//* - + /** {@inheritDoc} */ @Override public void write(Cache.Entry entry) throws CacheWriterException { STORE_MAP.put(entry.getKey(), entry.getValue()); } - */ -/** {@inheritDoc} *//* - + /** {@inheritDoc} */ @Override public void delete(Object key) throws CacheWriterException { STORE_MAP.remove(key); } } -*/ - } \ No newline at end of file From 65d93e48c264f4bfff0a94856fdfeb83375a3976 Mon Sep 17 00:00:00 2001 From: mcherkasov Date: Fri, 23 Jun 2017 21:00:45 +0300 Subject: [PATCH 50/90] IGNITE-5521: Large near caches lead to cluster instability with metrics enabled --- .../near/GridNearCacheAdapter.java | 2 +- .../near/GridCacheNearEvictionSelfTest.java | 5 ++-- .../near/GridCacheNearMetricsSelfTest.java | 30 ++++++++++++++++++- 3 files changed, 33 insertions(+), 4 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java index 4ddad74157987..8dfe1698991a3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java @@ -308,7 +308,7 @@ protected void processGetResponse(UUID nodeId, GridNearGetResponse res) { /** {@inheritDoc} */ @Override public int size() { - return nearEntries().size() + dht().size(); + return dht().size(); } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearEvictionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearEvictionSelfTest.java index df5943611075c..6181f50962003 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearEvictionSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearEvictionSelfTest.java @@ -165,12 +165,13 @@ public void testNearEnabledThreeNodes() throws Exception { for (int i = 0; i < gridCnt; i++) { final GridCacheAdapter cache = internalCache(i); + final GridCacheAdapter near = near(i); // Repeatedly check cache sizes because of concurrent cache updates. assertTrue(GridTestUtils.waitForCondition(new PA() { @Override public boolean apply() { // Every node contains either near, backup, or primary. - return cnt == cache.size(); + return cnt == cache.size() + near.nearSize(); } }, getTestTimeout())); @@ -183,4 +184,4 @@ public void testNearEnabledThreeNodes() throws Exception { stopAllGrids(); } } -} \ No newline at end of file +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMetricsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMetricsSelfTest.java index 09ff519a67976..de2696f56392a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMetricsSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMetricsSelfTest.java @@ -21,7 +21,10 @@ import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.CacheMode; import org.apache.ignite.cache.CacheWriteSynchronizationMode; +import org.apache.ignite.cluster.ClusterMetrics; +import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.processors.cache.GridCacheAbstractSelfTest; import org.apache.ignite.internal.util.typedef.internal.U; @@ -97,6 +100,31 @@ protected int keyCount() { return cc; } + /** + * @throws Exception If failed. + */ + public void testNearCacheDoesNotAffectCacheSize() throws Exception { + IgniteCache cache0 = grid(0).cache(null); + + for (int i = 0; i < 100 ; i++) + cache0.put(i, i); + + IgniteEx g1 = grid(1); + + IgniteCache cache1 = g1.cache(null ); + + ClusterNode localNode = g1.cluster().localNode(); + + int beforeSize = cache1.localMetrics().getSize(); + + for (int i = 0; i < 100 ; i++) { + if (!affinity(cache1).isPrimaryOrBackup(localNode, i)) + cache1.get(i); // put entry to near cache + } + + assertEquals(beforeSize, cache1.localMetrics().getSize()); + } + /** * @throws Exception If failed. */ @@ -417,4 +445,4 @@ else if (affinity(jcache).isBackup(g.cluster().localNode(), key)){ } } } -} \ No newline at end of file +} From ff1afad1278e939aef71b274e959fd4256904971 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Tue, 27 Jun 2017 10:34:42 +0300 Subject: [PATCH 51/90] Partially reverted GG-12352. (cherry picked from commit 3668b91) --- .../ignite/spi/discovery/tcp/ServerImpl.java | 10 +--------- ...IgniteBinaryMetadataUpdateNodeRestartTest.java | 10 ++++++++++ .../IgniteCacheNearRestartRollbackSelfTest.java | 15 +++++++++++++++ ...riteSynchronizationModesMultithreadedTest.java | 5 +++++ 4 files changed, 31 insertions(+), 9 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java index 2f7e9b470aae6..6e5af2986dcff 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java @@ -1242,7 +1242,7 @@ else if (U.currentTimeMillis() - noResStart > spi.joinTimeout) errs.add(e); - if (X.hasCause(e, SSLException.class)) { + if (X.hasCause(e, SSLException.class, StreamCorruptedException.class)) { if (--sslConnectAttempts == 0) throw new IgniteException("Unable to establish secure connection. " + "Was remote cluster configured with SSL? [rmtAddr=" + addr + ", errMsg=\"" + e.getMessage() + "\"]", e); @@ -1250,14 +1250,6 @@ else if (U.currentTimeMillis() - noResStart > spi.joinTimeout) continue; } - if (X.hasCause(e, StreamCorruptedException.class)) { - if (--sslConnectAttempts == 0) - throw new IgniteException("Unable to establish plain connection. " + - "Was remote cluster configured with SSL? [rmtAddr=" + addr + ", errMsg=\"" + e.getMessage() + "\"]", e); - - continue; - } - if (timeoutHelper.checkFailureTimeoutReached(e)) break; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteBinaryMetadataUpdateNodeRestartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteBinaryMetadataUpdateNodeRestartTest.java index 814fb08eacba5..3d552847fd71d 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteBinaryMetadataUpdateNodeRestartTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteBinaryMetadataUpdateNodeRestartTest.java @@ -26,6 +26,7 @@ import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheEntryProcessor; import org.apache.ignite.cluster.ClusterTopologyException; @@ -88,10 +89,19 @@ public class IgniteBinaryMetadataUpdateNodeRestartTest extends GridCommonAbstrac return cfg; } + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + System.setProperty(IgniteSystemProperties.IGNITE_ENABLE_FORCIBLE_NODE_KILL,"true"); + } + /** {@inheritDoc} */ @Override protected void afterTestsStopped() throws Exception { stopAllGrids(); + System.clearProperty(IgniteSystemProperties.IGNITE_ENABLE_FORCIBLE_NODE_KILL); + super.afterTestsStopped(); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheNearRestartRollbackSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheNearRestartRollbackSelfTest.java index a48497d8f39c4..3f242b5726867 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheNearRestartRollbackSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheNearRestartRollbackSelfTest.java @@ -31,6 +31,7 @@ import javax.cache.processor.MutableEntry; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheRebalanceMode; import org.apache.ignite.cache.CacheWriteSynchronizationMode; @@ -92,6 +93,20 @@ public class IgniteCacheNearRestartRollbackSelfTest extends GridCommonAbstractTe return cfg; } + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + System.setProperty(IgniteSystemProperties.IGNITE_ENABLE_FORCIBLE_NODE_KILL,"true"); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + System.clearProperty(IgniteSystemProperties.IGNITE_ENABLE_FORCIBLE_NODE_KILL); + } + /** * @param gridName Grid name. * @return Cache configuration. diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxCacheWriteSynchronizationModesMultithreadedTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxCacheWriteSynchronizationModesMultithreadedTest.java index 08396daa5efeb..c214a772d422c 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxCacheWriteSynchronizationModesMultithreadedTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxCacheWriteSynchronizationModesMultithreadedTest.java @@ -31,6 +31,7 @@ import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.cache.CacheWriteSynchronizationMode; import org.apache.ignite.cache.store.CacheStore; import org.apache.ignite.cache.store.CacheStoreAdapter; @@ -104,6 +105,8 @@ public class IgniteTxCacheWriteSynchronizationModesMultithreadedTest extends Gri @Override protected void beforeTestsStarted() throws Exception { super.beforeTestsStarted(); + System.setProperty(IgniteSystemProperties.IGNITE_ENABLE_FORCIBLE_NODE_KILL,"true"); + startGrids(SRVS); clientMode = true; @@ -119,6 +122,8 @@ public class IgniteTxCacheWriteSynchronizationModesMultithreadedTest extends Gri @Override protected void afterTestsStopped() throws Exception { stopAllGrids(); + System.clearProperty(IgniteSystemProperties.IGNITE_ENABLE_FORCIBLE_NODE_KILL); + super.afterTestsStopped(); } From 7fbb95d1698c534957c9f0b7c5558f43985cfe68 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Tue, 27 Jun 2017 20:43:07 +0300 Subject: [PATCH 52/90] GG-12370: Fixed massive NODE_FAILED events lead to excessive momentary memory consumption. (cherry picked from commit 0521b77) --- .../GridCachePartitionExchangeManager.java | 5 +- .../CacheLateAffinityAssignmentTest.java | 68 +++++++++++++++++++ 2 files changed, 71 insertions(+), 2 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java index 92142c0731240..3df0bafc06c25 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 @@ -269,8 +269,9 @@ else if (customEvt.customMessage() instanceof CacheAffinityChangeMessage) { exchFut = exchangeFuture(exchId, e, null, msg); } } - else - exchangeFuture(msg.exchangeId(), null, null, null).onAffinityChangeMessage(customEvt.eventNode(), msg); + else if (msg.exchangeId().topologyVersion().topologyVersion() >= affinityTopologyVersion(cctx.discovery().localJoinEvent()).topologyVersion()) + exchangeFuture(msg.exchangeId(), null, null, null) + .onAffinityChangeMessage(customEvt.eventNode(), msg); } } 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 7e37450dd5621..771ab34116030 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 @@ -69,6 +69,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsFullMessage; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsSingleMessage; +import org.apache.ignite.internal.util.lang.GridAbsPredicate; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.G; import org.apache.ignite.internal.util.typedef.PA; @@ -1168,6 +1169,73 @@ public void testDelayAssignmentAffinityChanged() throws Exception { checkAffinity(4, topVer(4, 1), true); } + /** + * Wait for rebalance, send affinity change message, but affinity already changed (new node joined). + * + * @throws Exception If failed. + */ + public void testDelayAssignmentAffinityChanged2() throws Exception { + Ignite ignite0 = startServer(0, 1); + + TestTcpDiscoverySpi discoSpi0 = + (TestTcpDiscoverySpi)ignite0.configuration().getDiscoverySpi(); + TestRecordingCommunicationSpi commSpi0 = + (TestRecordingCommunicationSpi)ignite0.configuration().getCommunicationSpi(); + + startClient(1, 2); + + checkAffinity(2, topVer(2, 0), true); + + startServer(2, 3); + + checkAffinity(3, topVer(3, 1), false); + + discoSpi0.blockCustomEvent(); + + stopNode(2, 4); + + discoSpi0.waitCustomEvent(); + + blockSupplySend(commSpi0, CACHE_NAME1); + + final IgniteInternalFuture startedFuture = multithreadedAsync(new Callable() { + @Override public Void call() throws Exception { + startServer(3, 5); + + return null; + } + }, 1, "server-starter"); + + Thread.sleep(2_000); + + discoSpi0.stopBlock(); + + boolean started = GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + return startedFuture.isDone(); + } + }, 10_000); + + if (!started) + startedFuture.cancel(); + + assertTrue(started); + + checkAffinity(3, topVer(5, 0), false); + + checkNoExchange(3, topVer(5, 1)); + + commSpi0.stopBlock(); + + checkAffinity(3, topVer(5, 1), true); + + List exFutures = grid(3).context().cache().context().exchange().exchangeFutures(); + + for (GridDhtPartitionsExchangeFuture f : exFutures) + //Shouldn't contains staled futures. + assertTrue(f.topologyVersion().topologyVersion() >= 5); + } + /** * Wait for rebalance, cache is destroyed and created again. * From 710db327c027d43c0984b7007447cecca71f3cca Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Wed, 28 Jun 2017 11:47:21 +0300 Subject: [PATCH 53/90] Rethrow handshake exceptions as it done for ConnectionException. (cherry picked from commit 7d52172) --- .../tcp/TcpCommunicationSpi.java | 32 +++++++++++++------ 1 file changed, 23 insertions(+), 9 deletions(-) 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 1bfa56accf2e0..9925b3d0c8736 100644 --- 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 @@ -2572,7 +2572,7 @@ else if (X.hasCause(e, SocketTimeoutException.class)) // Reconnect for the second time, if connection is not established. if (!failureDetThrReached && connectAttempts < 5 && - (X.hasCause(e, ConnectException.class) || X.hasCause(e, SocketTimeoutException.class))) { + (X.hasCause(e, ConnectException.class, HandshakeException.class, SocketTimeoutException.class))) { U.sleep(200); connectAttempts++; @@ -2599,7 +2599,8 @@ else if (X.hasCause(e, SocketTimeoutException.class)) if (enableForcibleNodeKill) { if (getSpiContext().node(node.id()) != null && (CU.clientNode(node) || !CU.clientNode(getLocalNode())) && - X.hasCause(errs, ConnectException.class, SocketTimeoutException.class, HandshakeTimeoutException.class, + X.hasCause(errs, ConnectException.class, HandshakeException.class, + SocketTimeoutException.class, HandshakeTimeoutException.class, IgniteSpiOperationTimeoutException.class)) { U.error(log, "TcpCommunicationSpi failed to establish connection to node, node will be dropped from " + "cluster [" + "rmtNode=" + node + ']', errs); @@ -2611,7 +2612,7 @@ else if (X.hasCause(e, SocketTimeoutException.class)) } } - if (X.hasCause(errs, ConnectException.class)) + if (X.hasCause(errs, ConnectException.class, HandshakeException.class)) throw errs; } @@ -2662,7 +2663,7 @@ private long safeHandshake( sslHnd = new BlockingSslHandler(sslMeta.sslEngine(), ch, directBuf, ByteOrder.nativeOrder(), log); if (!sslHnd.handshake()) - throw new IgniteCheckedException("SSL handshake is not completed."); + throw new HandshakeException("SSL handshake is not completed."); ByteBuffer handBuff = sslHnd.applicationBuffer(); @@ -2672,7 +2673,7 @@ private long safeHandshake( int read = ch.read(buf); if (read == -1) - throw new IgniteCheckedException("Failed to read remote node ID (connection closed)."); + throw new HandshakeException("Failed to read remote node ID (connection closed)."); buf.flip(); @@ -2688,7 +2689,7 @@ private long safeHandshake( int read = ch.read(buf); if (read == -1) - throw new IgniteCheckedException("Failed to read remote node ID (connection closed)."); + throw new HandshakeException("Failed to read remote node ID (connection closed)."); i += read; } @@ -2697,7 +2698,7 @@ private long safeHandshake( UUID rmtNodeId0 = U.bytesToUuid(buf.array(), 1); if (!rmtNodeId.equals(rmtNodeId0)) - throw new IgniteCheckedException("Remote node ID is not as expected [expected=" + rmtNodeId + + throw new HandshakeException("Remote node ID is not as expected [expected=" + rmtNodeId + ", rcvd=" + rmtNodeId0 + ']'); else if (log.isDebugEnabled()) log.debug("Received remote node ID: " + rmtNodeId0); @@ -2768,7 +2769,7 @@ else if (log.isDebugEnabled()) int read = ch.read(buf); if (read == -1) - throw new IgniteCheckedException("Failed to read remote node recovery handshake " + + throw new HandshakeException("Failed to read remote node recovery handshake " + "(connection closed)."); buf.flip(); @@ -2802,7 +2803,7 @@ else if (log.isDebugEnabled()) int read = ch.read(buf); if (read == -1) - throw new IgniteCheckedException("Failed to read remote node recovery handshake " + + throw new HandshakeException("Failed to read remote node recovery handshake " + "(connection closed)."); i += read; @@ -2990,6 +2991,19 @@ private ClientKey(UUID nodeId, long order) { } } + /** Internal exception class for proper timeout handling. */ + private static class HandshakeException extends IgniteCheckedException { + /** */ + private static final long serialVersionUID = 0L; + + /** + * @param msg Error message. + */ + HandshakeException(String msg) { + super(msg); + } + } + /** Internal exception class for proper timeout handling. */ private static class HandshakeTimeoutException extends IgniteCheckedException { /** */ From 7c619f77009cc6876405839f9f6201d332d94b47 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Wed, 28 Jun 2017 15:45:42 +0300 Subject: [PATCH 54/90] Fixed tests. (cherry picked from commit 81cdea4) --- .../IgniteClientReconnectAbstractTest.java | 5 +++++ ...iteClientReconnectContinuousProcessorTest.java | 15 +-------------- .../cache/IgniteDynamicCacheStartSelfTest.java | 15 +++++++++++++++ 3 files changed, 21 insertions(+), 14 deletions(-) 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 4d49366e1ad23..17a1ad41ae448 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 @@ -33,6 +33,7 @@ import org.apache.ignite.IgniteClientDisconnectedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; +import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.events.Event; @@ -152,6 +153,8 @@ protected BlockTcpCommunicationSpi commSpi(Ignite ignite) { @Override protected void beforeTestsStarted() throws Exception { super.beforeTestsStarted(); + System.setProperty(IgniteSystemProperties.IGNITE_ENABLE_FORCIBLE_NODE_KILL, "true"); + int srvs = serverCount(); if (srvs > 0) @@ -172,6 +175,8 @@ protected BlockTcpCommunicationSpi commSpi(Ignite ignite) { @Override protected void afterTestsStopped() throws Exception { super.afterTestsStopped(); + System.clearProperty(IgniteSystemProperties.IGNITE_ENABLE_FORCIBLE_NODE_KILL); + stopAllGrids(); } 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 0ff5883d11b57..c86dcabb3a208 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 @@ -23,6 +23,7 @@ import javax.cache.event.CacheEntryUpdatedListener; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.cache.query.ContinuousQuery; import org.apache.ignite.cache.query.QueryCursor; import org.apache.ignite.configuration.CacheConfiguration; @@ -110,20 +111,6 @@ public void testEventListenerReconnect() throws Exception { assertFalse(lsnr.latch.await(3000, MILLISECONDS)); } - /** - * @throws Exception If failed. - */ - public void testMessageListenerReconnectAndStopFromServer() throws Exception { - testMessageListenerReconnect(false); - } - - /** - * @throws Exception If failed. - */ - public void testMessageListenerReconnectAndStopFromClient() throws Exception { - testMessageListenerReconnect(true); - } - /** * @param stopFromClient If {@code true} stops listener from client node, otherwise from server. * @throws Exception If failed. diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java index 48e06ee7c82a0..49a49c7958e94 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java @@ -31,6 +31,7 @@ import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.Ignition; import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheExistsException; import org.apache.ignite.cache.CacheMode; @@ -144,6 +145,20 @@ public int nodeCount() { stopAllGrids(); } + /** + * {@inheritDoc} + */ + @Override protected void afterTest() throws Exception { + super.afterTest(); + + if(grid(0).cache(DYNAMIC_CACHE_NAME)!=null) + grid(0).destroyCache(DYNAMIC_CACHE_NAME); + + while(Ignition.allGrids().size() > nodeCount()) { + stopGrid(nodeCount()); + } + } + /** * @throws Exception If failed. */ From 5219a35208b95a2d7dea3825dd4cb4edc74b3542 Mon Sep 17 00:00:00 2001 From: Slava Koptilin Date: Mon, 26 Jun 2017 15:37:26 +0300 Subject: [PATCH 55/90] IGNITE-5076: improved multi-threaded start of nodes --- .../ignite/testframework/junits/GridAbstractTest.java | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) mode change 100644 => 100755 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 old mode 100644 new mode 100755 index ffaec4fbc89af..5c43ecbfc9bb2 --- 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 @@ -646,12 +646,14 @@ protected final Ignite startGrids(int cnt) throws Exception { * @throws Exception If failed. */ protected Ignite startGridsMultiThreaded(int cnt) throws Exception { - if (cnt == 1) - return startGrids(1); + assert cnt > 0 : "Number of grids must be a positive number"; - Ignite ignite = startGridsMultiThreaded(0, cnt); + Ignite ignite = startGrids(1); - checkTopology(cnt); + if (cnt > 1) { + startGridsMultiThreaded(1, cnt - 1); + checkTopology(cnt); + } return ignite; } From a4fc555b118d91ec0348154b88764f010dbbae52 Mon Sep 17 00:00:00 2001 From: nikolay_tikhonov Date: Fri, 30 Jun 2017 14:38:54 +0300 Subject: [PATCH 56/90] Fixed "IGNITE-5424 GridServiceProxy does not unwraps exception message from InvocationTargetException." This closes #2168 Signed-off-by: nikolay_tikhonov --- .../processors/service/GridServiceProxy.java | 9 ++- .../GridServiceProcessorProxySelfTest.java | 66 +++++++++++++++++++ 2 files changed, 74 insertions(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java index 2286cff9717aa..6e8fb4e0ae667 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java @@ -23,6 +23,7 @@ import java.io.ObjectOutput; import java.io.Serializable; import java.lang.reflect.InvocationHandler; +import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.lang.reflect.Proxy; import java.util.ArrayList; @@ -403,7 +404,13 @@ private ServiceProxyCallable(String mtdName, String svcName, Class[] argTypes, O if (mtd == null) throw new GridServiceMethodNotFoundException(svcName, mtdName, argTypes); - return mtd.invoke(svcCtx.service(), args); + try { + return mtd.invoke(svcCtx.service(), args); + } + catch (InvocationTargetException e) { + // Get error message. + throw new IgniteCheckedException(e.getCause().getMessage(), e); + } } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorProxySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorProxySelfTest.java index 6fc7e02e42946..850cb5702b5b0 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorProxySelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorProxySelfTest.java @@ -18,12 +18,15 @@ package org.apache.ignite.internal.processors.service; import java.util.Map; +import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteException; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.services.Service; import org.apache.ignite.services.ServiceContext; +import org.apache.ignite.testframework.GridTestUtils; /** * Service proxy test. @@ -63,6 +66,31 @@ public void testNodeSingletonProxy() throws Exception { } } + /** + * Unwraps error message from InvocationTargetException. + * + * @throws Exception If failed. + */ + @SuppressWarnings("ThrowableNotThrown") + public void testException() throws Exception { + String name = "errorService"; + + Ignite ignite = grid(0); + + ignite.services(ignite.cluster().forRemotes()).deployNodeSingleton(name, new ErrorServiceImpl()); + + final ErrorService svc = ignite.services().serviceProxy(name, ErrorService.class, false); + + GridTestUtils.assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + svc.go(); + + return null; + } + }, IgniteException.class, "Test exception"); + + } + /** * @throws Exception If failed. */ @@ -353,6 +381,7 @@ protected static class MapServiceImpl implements MapService, Service map.clear(); } + /** {@inheritDoc} */ @Override public int size() { return map.size(); } @@ -372,4 +401,41 @@ protected static class MapServiceImpl implements MapService, Service X.println("Executing cache service: " + ctx.name()); } } + + /** + * + */ + protected interface ErrorService extends Service { + /** + * + */ + void go() throws Exception; + } + + /** + * + */ + protected class ErrorServiceImpl implements ErrorService { + /** {@inheritDoc} */ + @Override public void cancel(ServiceContext ctx) { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void init(ServiceContext ctx) throws Exception { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void execute(ServiceContext ctx) throws Exception { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void go() throws Exception { + throw new Exception("Test exception"); + } + } + + } \ No newline at end of file From 752b1368adbb5c77b5d5caca3c07a72decff5111 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Fri, 30 Jun 2017 14:45:18 +0300 Subject: [PATCH 57/90] IGNITE-5473 partial fix: Create ignite troubleshooting logger. --- .../apache/ignite/IgniteSystemProperties.java | 7 +++++++ .../tcp/TcpCommunicationSpi.java | 20 ++++++++++++++----- 2 files changed, 22 insertions(+), 5 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java index e9bbf5a10df4c..c4208a76f1a7d 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -135,6 +135,13 @@ public final class IgniteSystemProperties { */ public static final String IGNITE_QUIET = "IGNITE_QUIET"; + /** + * Setting this option to {@code true} will enable troubleshooting logger. + * Troubleshooting logger makes logging more verbose without enabling debug mode + * to provide more detailed logs without performance penalty. + */ + public static final String IGNITE_TROUBLESHOOTING_LOGGER = "IGNITE_TROUBLESHOOTING_LOGGER"; + /** * Setting to {@code true} enables writing sensitive information in {@code toString()} output. */ 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 9925b3d0c8736..b16842e0d1129 100644 --- 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 @@ -333,6 +333,10 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter private boolean enableForcibleNodeKill = IgniteSystemProperties .getBoolean(IgniteSystemProperties.IGNITE_ENABLE_FORCIBLE_NODE_KILL); + /** */ + private boolean enableTroubleshootingLog = IgniteSystemProperties + .getBoolean(IgniteSystemProperties.IGNITE_TROUBLESHOOTING_LOGGER); + /** Server listener. */ private final GridNioServerListener srvLsnr = new GridNioServerListenerAdapter() { @@ -2550,9 +2554,10 @@ protected GridCommunicationClient createTcpClient(ClusterNode node) throws Ignit boolean failureDetThrReached = timeoutHelper.checkFailureTimeoutReached(e); - U.error(log, "Failed to establish connection to a remote node [node=" + node + - ", addr=" + addr + ", connectAttempts=" + connectAttempts + - ", failureDetThrReached" + failureDetThrReached + ']', e); + if (enableTroubleshootingLog) + U.error(log, "Failed to establish connection to a remote node [node=" + node + + ", addr=" + addr + ", connectAttempts=" + connectAttempts + + ", failureDetThrReached=" + failureDetThrReached + ']', e); if (failureDetThrReached) LT.warn(log, "Connect timed out (consider increasing 'failureDetectionTimeout' " + @@ -2602,8 +2607,13 @@ else if (X.hasCause(e, SocketTimeoutException.class)) X.hasCause(errs, ConnectException.class, HandshakeException.class, SocketTimeoutException.class, HandshakeTimeoutException.class, IgniteSpiOperationTimeoutException.class)) { - U.error(log, "TcpCommunicationSpi failed to establish connection to node, node will be dropped from " + - "cluster [" + "rmtNode=" + node + ']', errs); + String msg = "TcpCommunicationSpi failed to establish connection to node, node will be dropped from " + + "cluster [" + "rmtNode=" + node + ']'; + + if(enableTroubleshootingLog) + U.error(log, msg, errs); + else + U.warn(log, msg); getSpiContext().failNode(node.id(), "TcpCommunicationSpi failed to establish connection to node [" + "rmtNode=" + node + From 1525c6cf2cb015289392eb54fec4029e9b53b438 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Fri, 30 Jun 2017 18:36:50 +0300 Subject: [PATCH 58/90] Fixed service deployment tests. --- .../IgniteServiceDynamicCachesSelfTest.java | 18 ++++++++++++++---- 1 file changed, 14 insertions(+), 4 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/IgniteServiceDynamicCachesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/IgniteServiceDynamicCachesSelfTest.java index c41f2f03c460e..acd2b252d8f9d 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/IgniteServiceDynamicCachesSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/IgniteServiceDynamicCachesSelfTest.java @@ -43,8 +43,8 @@ public class IgniteServiceDynamicCachesSelfTest extends GridCommonAbstractTest { private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); /** {@inheritDoc} */ - @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { - IgniteConfiguration cfg = super.getConfiguration(gridName); + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); TcpDiscoverySpi discoSpi = new TcpDiscoverySpi(); @@ -83,7 +83,7 @@ public void testDeployCalledAfterCacheStart() throws Exception { final String svcName = "myService"; - svcs.deployKeyAffinitySingleton(svcName, new TestService(), cacheName, "key"); + svcs.deployKeyAffinitySingleton(svcName, new TestService(), cacheName, primaryKey(ig.cache(cacheName))); boolean res = GridTestUtils.waitForCondition(new PA() { @Override public boolean apply() { @@ -125,7 +125,15 @@ public void testDeployCalledBeforeCacheStart() throws Exception { final String svcName = "myService"; - svcs.deployKeyAffinitySingleton(svcName, new TestService(), cacheName, "key"); + ig.createCache(ccfg); + + Object key = primaryKey(ig.cache(cacheName)); + + ig.destroyCache(cacheName); + + awaitPartitionMapExchange(); + + svcs.deployKeyAffinitySingleton(svcName, new TestService(), cacheName, key); assert svcs.service(svcName) == null; @@ -140,6 +148,8 @@ public void testDeployCalledBeforeCacheStart() throws Exception { assertTrue("Service was not deployed", res); + info("stopping cache: " + cacheName); + ig.destroyCache(cacheName); res = GridTestUtils.waitForCondition(new PA() { From a24ca24dd7aaa34707d74a4e660d769d3d5b0ed8 Mon Sep 17 00:00:00 2001 From: sboikov Date: Wed, 19 Apr 2017 12:46:31 +0300 Subject: [PATCH 59/90] Attempt to fix awaitPartitionMapExchange: wait for last exchange completion to avoid races with cache destroy. (cherry picked from commit d383484) --- .../junits/common/GridCommonAbstractTest.java | 35 +++++++++++++++++++ 1 file changed, 35 insertions(+) diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java index 90fabd95a1b47..6616daf297388 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java @@ -58,6 +58,7 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.events.Event; import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteKernal; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.GridCacheAdapter; @@ -450,6 +451,27 @@ protected void awaitPartitionMapExchange(boolean waitEvicts, Set names = new HashSet<>(); + Ignite crd = null; + + for (Ignite g : G.allGrids()) { + ClusterNode node = g.cluster().localNode(); + + if (crd == null || node.order() < crd.cluster().localNode().order()) { + crd = g; + + if (node.order() == 1) + break; + } + } + + if (crd == null) + return; + + AffinityTopologyVersion waitTopVer = ((IgniteKernal)crd).context().discovery().topologyVersionEx(); + + if (waitTopVer.topologyVersion() <= 0) + waitTopVer = new AffinityTopologyVersion(1, 0); + for (Ignite g : G.allGrids()) { if (nodes != null && !nodes.contains(g.cluster().localNode())) continue; @@ -466,6 +488,19 @@ protected void awaitPartitionMapExchange(boolean waitEvicts, else startTime = g0.context().discovery().gridStartTime(); + IgniteInternalFuture exchFut = + g0.context().cache().context().exchange().affinityReadyFuture(waitTopVer); + + if (exchFut != null && !exchFut.isDone()) { + try { + exchFut.get(timeout); + } + catch (IgniteCheckedException e) { + log.error("Failed to wait for exchange [topVer=" + waitTopVer + + ", node=" + g0.name() + ']', e); + } + } + for (IgniteCacheProxy c : g0.context().cache().jcaches()) { CacheConfiguration cfg = c.context().config(); From ef0a874ceb5c8bfa53e16337f6fd1699afaf2a39 Mon Sep 17 00:00:00 2001 From: nikolay_tikhonov Date: Fri, 30 Jun 2017 20:39:01 +0300 Subject: [PATCH 60/90] Fixed CacheSerializableTransactionsTest#testTxConflictRemoveWithOldValue test. Signed-off-by: nikolay_tikhonov --- .../ignite/internal/processors/cache/GridCacheAdapter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 220c192f21c46..cde6309d3f9ff 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 @@ -3011,7 +3011,7 @@ protected boolean remove0(final K key, final CacheEntryPredicate filter) throws Collections.singletonList(key), /*retval*/false, filter, - /*singleRmv*/true).get().success(); + /*singleRmv*/false).get().success(); } @Override public String toString() { From 4dce965ea86374cba7265cb5d22e975aeac7d480 Mon Sep 17 00:00:00 2001 From: nikolay_tikhonov Date: Fri, 30 Jun 2017 21:36:02 +0300 Subject: [PATCH 61/90] Fixed org.jsr107.tck.PutTest tests. Signed-off-by: nikolay_tikhonov --- .../internal/processors/cache/GridCacheAdapter.java | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) 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 cde6309d3f9ff..7c5a54dfcb852 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 @@ -642,7 +642,7 @@ public void onKernalStop() { return (IgniteInternalFuture)getAsync( key, - /*force primary*/false, + /*force primary*/ !ctx.config().isReadFromBackup(), /*skip tx*/false, /*subj id*/null, /*task name*/null, @@ -669,7 +669,7 @@ public void onKernalStop() { return getAllAsync( keys, - /*force primary*/false, + /*force primary*/ !ctx.config().isReadFromBackup(), /*skip tx*/false, /*subj id*/null, /*task name*/null, @@ -1137,7 +1137,7 @@ public List> splitClearLocally(boolean srv, bool execSvc = Executors.newFixedThreadPool(jobs.size() - 1); for (int i = 1; i < jobs.size(); i++) - execSvc.submit(jobs.get(i)); + execSvc.execute(jobs.get(i)); } jobs.get(0).run(); @@ -2715,6 +2715,8 @@ public IgniteInternalFuture putAsync0(final K key, final V val, /** {@inheritDoc} */ @Override public void putAll(@Nullable final Map m) throws IgniteCheckedException { + A.notNull(m, "map"); + if (F.isEmpty(m)) return; @@ -3000,6 +3002,7 @@ public boolean remove(final K key, @Nullable CacheEntryPredicate filter) throws /** * @param key Key. + * @param filter Filter. * @return {@code True} if entry was removed. * @throws IgniteCheckedException If failed. */ @@ -3011,7 +3014,7 @@ protected boolean remove0(final K key, final CacheEntryPredicate filter) throws Collections.singletonList(key), /*retval*/false, filter, - /*singleRmv*/false).get().success(); + /*singleRmv*/filter == null).get().success(); } @Override public String toString() { From 355a5283559c885f57c4557bba2c6d9170a9b5fc Mon Sep 17 00:00:00 2001 From: mcherkasov Date: Fri, 30 Jun 2017 20:23:55 +0300 Subject: [PATCH 62/90] IGNITE-5554 ServiceProcessor may process failed reassignments in timeout thread --- .../service/GridServiceProcessor.java | 21 +++++++++---------- 1 file changed, 10 insertions(+), 11 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java index 20bcff7c780d3..c452da31086d0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java @@ -1581,16 +1581,11 @@ private void onDeployment(final GridServiceDeployment dep, final AffinityTopolog } @Override public void onTimeout() { - if (!busyLock.enterBusy()) - return; - - try { - // Try again. - onDeployment(dep, topVer); - } - finally { - busyLock.leaveBusy(); - } + depExe.execute(new DepRunnable() { + @Override public void run0() { + onDeployment(dep, topVer); + } + }); } }); } @@ -1777,7 +1772,11 @@ private void onReassignmentFailed(final AffinityTopologyVersion topVer, } @Override public void onTimeout() { - onReassignmentFailed(topVer, retries); + depExe.execute(new Runnable() { + public void run() { + onReassignmentFailed(topVer, retries); + } + }); } }); } From 92aa7c6e3c0d9b5cc68002433861b175d54f9421 Mon Sep 17 00:00:00 2001 From: agura Date: Tue, 4 Jul 2017 16:56:40 +0300 Subject: [PATCH 63/90] ignite-5685 JDBC prepared statement shouldn't clear parameters after execution --- .../jdbc2/JdbcPreparedStatementSelfTest.java | 35 +++++++++++++++++++ .../jdbc/JdbcPreparedStatementSelfTest.java | 35 +++++++++++++++++++ .../internal/jdbc/JdbcPreparedStatement.java | 6 +--- .../internal/jdbc2/JdbcPreparedStatement.java | 6 ++-- 4 files changed, 73 insertions(+), 9 deletions(-) diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcPreparedStatementSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcPreparedStatementSelfTest.java index ea586b297a579..56c248817e12a 100644 --- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcPreparedStatementSelfTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcPreparedStatementSelfTest.java @@ -137,6 +137,41 @@ public class JdbcPreparedStatementSelfTest extends GridCommonAbstractTest { } } + /** + * @throws Exception If failed. + */ + public void testRepeatableUsage() throws Exception { + stmt = conn.prepareStatement("select * from TestObject where id = ?"); + + stmt.setInt(1, 1); + + ResultSet rs = stmt.executeQuery(); + + int cnt = 0; + + while (rs.next()) { + if (cnt == 0) + assertEquals(1, rs.getInt(1)); + + cnt++; + } + + assertEquals(1, cnt); + + cnt = 0; + + rs = stmt.executeQuery(); + + while (rs.next()) { + if (cnt == 0) + assertEquals(1, rs.getInt(1)); + + cnt++; + } + + assertEquals(1, cnt); + } + /** * @throws Exception If failed. */ diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcPreparedStatementSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcPreparedStatementSelfTest.java index 0c701ef3e7ce3..cd8648833c3d1 100644 --- a/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcPreparedStatementSelfTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcPreparedStatementSelfTest.java @@ -155,6 +155,41 @@ public class JdbcPreparedStatementSelfTest extends GridCommonAbstractTest { } } + /** + * @throws Exception If failed. + */ + public void testRepeatableUsage() throws Exception { + stmt = conn.prepareStatement("select * from TestObject where id = ?"); + + stmt.setInt(1, 1); + + ResultSet rs = stmt.executeQuery(); + + int cnt = 0; + + while (rs.next()) { + if (cnt == 0) + assertEquals(1, rs.getInt(1)); + + cnt++; + } + + assertEquals(1, cnt); + + cnt = 0; + + rs = stmt.executeQuery(); + + while (rs.next()) { + if (cnt == 0) + assertEquals(1, rs.getInt(1)); + + cnt++; + } + + assertEquals(1, cnt); + } + /** * @throws Exception If failed. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/JdbcPreparedStatement.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/JdbcPreparedStatement.java index 7e5358becab93..93cda1e4768cd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/JdbcPreparedStatement.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/JdbcPreparedStatement.java @@ -69,11 +69,7 @@ public class JdbcPreparedStatement extends JdbcStatement implements PreparedStat /** {@inheritDoc} */ @Override public ResultSet executeQuery() throws SQLException { - ResultSet rs = executeQuery(sql); - - args = null; - - return rs; + return executeQuery(sql); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcPreparedStatement.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcPreparedStatement.java index a99f24c8633d7..72375cd939a88 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcPreparedStatement.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcPreparedStatement.java @@ -50,11 +50,9 @@ public class JdbcPreparedStatement extends JdbcStatement implements PreparedStat /** {@inheritDoc} */ @Override public ResultSet executeQuery() throws SQLException { - ResultSet rs = executeQuery(sql); - - args = null; + ensureNotClosed(); - return rs; + return executeQuery(sql); } /** {@inheritDoc} */ From d9fc20a61d5ac0a6e63b26faa7fa0af753b2fa06 Mon Sep 17 00:00:00 2001 From: Dmitriy Govorukhin Date: Fri, 7 Apr 2017 14:28:22 +0300 Subject: [PATCH 64/90] IGNITE-4889 - Changed Hibernate integration to use custom keys (cherry picked from commit 6b62a20) --- .../HibernateAccessStrategyAdapter.java | 8 +- .../cache/hibernate/HibernateCacheProxy.java | 818 ++++++++++++++++++ .../hibernate/HibernateCollectionRegion.java | 3 +- .../hibernate/HibernateEntityRegion.java | 3 +- .../hibernate/HibernateGeneralDataRegion.java | 3 +- .../hibernate/HibernateKeyTransformer.java | 29 + .../cache/hibernate/HibernateKeyWrapper.java | 72 ++ .../hibernate/HibernateNaturalIdRegion.java | 3 +- .../HibernateNonStrictAccessStrategy.java | 5 +- .../HibernateQueryResultsRegion.java | 3 +- .../HibernateReadOnlyAccessStrategy.java | 3 +- .../HibernateReadWriteAccessStrategy.java | 3 +- .../cache/hibernate/HibernateRegion.java | 6 +- .../hibernate/HibernateRegionFactory.java | 29 +- .../hibernate/HibernateTimestampsRegion.java | 3 +- .../HibernateTransactionalAccessStrategy.java | 2 +- .../HibernateTransactionalDataRegion.java | 3 +- .../IgniteBinaryHibernateTestSuite.java | 37 + 18 files changed, 1000 insertions(+), 33 deletions(-) create mode 100644 modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateCacheProxy.java create mode 100644 modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateKeyTransformer.java create mode 100644 modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateKeyWrapper.java create mode 100644 modules/hibernate/src/test/java/org/apache/ignite/testsuites/IgniteBinaryHibernateTestSuite.java diff --git a/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateAccessStrategyAdapter.java b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateAccessStrategyAdapter.java index 27734d9ad9ca5..f6c1d0e15eb43 100644 --- a/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateAccessStrategyAdapter.java +++ b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateAccessStrategyAdapter.java @@ -92,7 +92,7 @@ */ public abstract class HibernateAccessStrategyAdapter { /** */ - protected final IgniteInternalCache cache; + protected final HibernateCacheProxy cache; /** Grid. */ protected final Ignite ignite; @@ -104,7 +104,7 @@ public abstract class HibernateAccessStrategyAdapter { * @param ignite Grid. * @param cache Cache. */ - protected HibernateAccessStrategyAdapter(Ignite ignite, IgniteInternalCache cache) { + protected HibernateAccessStrategyAdapter(Ignite ignite, HibernateCacheProxy cache) { this.cache = cache; this.ignite = ignite; @@ -292,8 +292,10 @@ protected final void removeAll() throws CacheException { * @param key Key. * @throws CacheException If failed. */ - static void evict(Ignite ignite, IgniteInternalCache cache, Object key) throws CacheException { + static void evict(Ignite ignite, HibernateCacheProxy cache, Object key) throws CacheException { try { + key = cache.keyTransformer().transform(key); + ignite.compute(ignite.cluster()).call(new ClearKeyCallable(key, cache.name())); } catch (IgniteException e) { diff --git a/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateCacheProxy.java b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateCacheProxy.java new file mode 100644 index 0000000000000..871c4a100ac6f --- /dev/null +++ b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateCacheProxy.java @@ -0,0 +1,818 @@ +/* + * 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.cache.hibernate; + +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import javax.cache.Cache; +import javax.cache.expiry.ExpiryPolicy; +import javax.cache.processor.EntryProcessor; +import javax.cache.processor.EntryProcessorResult; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.cache.CacheEntry; +import org.apache.ignite.cache.CacheMetrics; +import org.apache.ignite.cache.CachePeekMode; +import org.apache.ignite.cache.affinity.Affinity; +import org.apache.ignite.cluster.ClusterGroup; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.CacheEntryPredicate; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.IgniteCacheExpiryPolicy; +import org.apache.ignite.internal.processors.cache.IgniteInternalCache; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.lang.IgniteBiPredicate; +import org.apache.ignite.mxbean.CacheMetricsMXBean; +import org.apache.ignite.transactions.Transaction; +import org.apache.ignite.transactions.TransactionConcurrency; +import org.apache.ignite.transactions.TransactionIsolation; +import org.jetbrains.annotations.Nullable; + +/** + * Hibernate cache proxy. + */ +public class HibernateCacheProxy implements IgniteInternalCache { + /** Delegate. */ + private final IgniteInternalCache delegate; + + /** Transformer. */ + private final HibernateKeyTransformer keyTransformer; + + /** + * @param delegate Delegate. + * @param keyTransformer Key keyTransformer. + */ + HibernateCacheProxy( + IgniteInternalCache delegate, + HibernateKeyTransformer keyTransformer + ) { + assert delegate != null; + assert keyTransformer != null; + + this.delegate = delegate; + this.keyTransformer = keyTransformer; + } + + /** + * @return HibernateKeyTransformer + */ + HibernateKeyTransformer keyTransformer(){ + return keyTransformer; + } + + /** {@inheritDoc} */ + @Override public String name() { + return delegate.name(); + } + + /** {@inheritDoc} */ + @Override public boolean skipStore() { + return delegate.skipStore(); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalCache setSkipStore(boolean skipStore) { + return delegate.setSkipStore(skipStore); + } + + /** {@inheritDoc} */ + @Override public boolean isEmpty() { + return delegate.isEmpty(); + } + + /** {@inheritDoc} */ + @Override public boolean containsKey(Object key) { + return delegate.containsKey(keyTransformer.transform(key)); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture containsKeyAsync(Object key) { + return delegate.containsKeyAsync(keyTransformer.transform(key)); + } + + /** {@inheritDoc} */ + @Override public void promoteAll(@Nullable Collection keys) throws IgniteCheckedException { + delegate.promoteAll(transform(keys)); + } + + /** {@inheritDoc} */ + @Override public long overflowSize() throws IgniteCheckedException { + return delegate.overflowSize(); + } + + /** {@inheritDoc} */ + @Override public long swapSize() throws IgniteCheckedException { + return delegate.swapSize(); + } + + /** {@inheritDoc} */ + @Override public long swapKeys() throws IgniteCheckedException { + return delegate.swapKeys(); + } + + /** {@inheritDoc} */ + @Override public boolean containsKeys(Collection keys) { + return delegate.containsKey(transform(keys)); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture containsKeysAsync(Collection keys) { + return delegate.containsKeysAsync(transform(keys)); + } + + /** {@inheritDoc} */ + @Nullable @Override public Object localPeek( + Object key, + CachePeekMode[] peekModes, + @Nullable IgniteCacheExpiryPolicy plc + ) throws IgniteCheckedException { + return delegate.localPeek(keyTransformer.transform(key), peekModes, plc); + } + + /** {@inheritDoc} */ + @Override public Iterable> localEntries( + CachePeekMode[] peekModes + ) throws IgniteCheckedException { + return delegate.localEntries(peekModes); + } + + /** {@inheritDoc} */ + @Nullable @Override public Object get(Object key) throws IgniteCheckedException { + return delegate.get(keyTransformer.transform(key)); + } + + /** {@inheritDoc} */ + @Nullable @Override public CacheEntry getEntry(Object key) throws IgniteCheckedException { + return delegate.getEntry(keyTransformer.transform(key)); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture getAsync(Object key) { + return delegate.getAsync(keyTransformer.transform(key)); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture> getEntryAsync(Object key) { + return delegate.getEntryAsync(keyTransformer.transform(key)); + } + + /** {@inheritDoc} */ + @Override public Map getAll(@Nullable Collection keys) throws IgniteCheckedException { + return delegate.getAll(transform(keys)); + } + + /** {@inheritDoc} */ + @Override public Collection> getEntries( + @Nullable Collection keys) throws IgniteCheckedException { + return delegate.getEntries(transform(keys)); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture> getAllAsync(@Nullable Collection keys) { + return delegate.getAllAsync(transform(keys)); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture>> getEntriesAsync( + @Nullable Collection keys + ) { + return delegate.getEntriesAsync(transform(keys)); + } + + /** {@inheritDoc} */ + @Nullable @Override public Object getAndPut(Object key, Object val) throws IgniteCheckedException { + return delegate.getAndPut(keyTransformer.transform(key), val); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture getAndPutAsync(Object key, Object val) { + return delegate.getAndPutAsync(keyTransformer.transform(key), val); + } + + /** {@inheritDoc} */ + @Override public boolean put(Object key, Object val) throws IgniteCheckedException { + return delegate.put(keyTransformer.transform(key), val); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture putAsync(Object key, Object val) { + return delegate.putAsync(keyTransformer.transform(key), val); + } + + /** {@inheritDoc} */ + @Nullable @Override public Object getAndPutIfAbsent(Object key, Object val) throws IgniteCheckedException { + return delegate.getAndPutIfAbsent(keyTransformer.transform(key), val); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture getAndPutIfAbsentAsync(Object key, Object val) { + return delegate.getAndPutIfAbsentAsync(keyTransformer.transform(key), val); + } + + /** {@inheritDoc} */ + @Override public boolean putIfAbsent(Object key, Object val) throws IgniteCheckedException { + return delegate.putIfAbsent(keyTransformer.transform(key), val); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture putIfAbsentAsync(Object key, Object val) { + return delegate.putIfAbsentAsync(keyTransformer.transform(key), val); + } + + /** {@inheritDoc} */ + @Nullable @Override public Object getAndReplace(Object key, Object val) throws IgniteCheckedException { + return delegate.getAndReplace(keyTransformer.transform(key), val); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture getAndReplaceAsync(Object key, Object val) { + return delegate.getAndReplaceAsync(keyTransformer.transform(key), val); + } + + /** {@inheritDoc} */ + @Override public boolean replace(Object key, Object val) throws IgniteCheckedException { + return delegate.replace(keyTransformer.transform(key), val); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture replaceAsync(Object key, Object val) { + return delegate.replaceAsync(keyTransformer.transform(key), val); + } + + /** {@inheritDoc} */ + @Override public boolean replace(Object key, Object oldVal, Object newVal) throws IgniteCheckedException { + return delegate.replace(keyTransformer.transform(key), oldVal, newVal); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture replaceAsync(Object key, Object oldVal, Object newVal) { + return delegate.replaceAsync(keyTransformer.transform(key), oldVal, newVal); + } + + /** {@inheritDoc} */ + @Override public void putAll(@Nullable Map m) throws IgniteCheckedException { + delegate.putAll(transform(m)); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture putAllAsync(@Nullable Map m) { + return delegate.putAllAsync(transform(m)); + } + + /** {@inheritDoc} */ + @Override public Set keySet() { + return delegate.keySet(); + } + + /** {@inheritDoc} */ + @Override public Set keySetx() { + return delegate.keySetx(); + } + + /** {@inheritDoc} */ + @Override public Set primaryKeySet() { + return delegate.primaryKeySet(); + } + + /** {@inheritDoc} */ + @Override public Iterable values() { + return delegate.values(); + } + + /** {@inheritDoc} */ + @Override public Set> entrySet() { + return delegate.entrySet(); + } + + /** {@inheritDoc} */ + @Nullable @Override public Set> entrySet(int part) { + return delegate.entrySet(part); + } + + /** {@inheritDoc} */ + @Override public Set> entrySetx(CacheEntryPredicate... filter) { + return delegate.entrySetx(filter); + } + + /** {@inheritDoc} */ + @Override public Transaction txStart( + TransactionConcurrency concurrency, + TransactionIsolation isolation + ) { + return delegate.txStart(concurrency, isolation); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalTx txStartEx( + TransactionConcurrency concurrency, + TransactionIsolation isolation + ) { + return delegate.txStartEx(concurrency, isolation); + } + + /** {@inheritDoc} */ + @Override public Transaction txStart( + TransactionConcurrency concurrency, + TransactionIsolation isolation, + long timeout, + int txSize + ) { + return delegate.txStart(concurrency, isolation, timeout, txSize); + } + + /** {@inheritDoc} */ + @Nullable @Override public Transaction tx() { + return delegate.tx(); + } + + /** {@inheritDoc} */ + @Override public boolean evict(Object key) { + return delegate.evict(keyTransformer.transform(key)); + } + + /** {@inheritDoc} */ + @Override public void evictAll(@Nullable Collection keys) { + delegate.evictAll(transform(keys)); + } + + /** {@inheritDoc} */ + @Override public void clearLocally(boolean srv, boolean near, boolean readers) { + delegate.clearLocally(srv, near, readers); + } + + /** {@inheritDoc} */ + @Override public boolean clearLocally(Object key) { + return delegate.clearLocally(keyTransformer.transform(key)); + } + + /** {@inheritDoc} */ + @Override public void clearLocallyAll(Set keys, boolean srv, boolean near, boolean readers) { + delegate.clearLocallyAll((Set)transform(keys), srv, near, readers); + } + + /** {@inheritDoc} */ + @Override public void clear(Object key) throws IgniteCheckedException { + delegate.clear(keyTransformer.transform(key)); + } + + /** {@inheritDoc} */ + @Override public void clearAll(Set keys) throws IgniteCheckedException { + delegate.clearAll((Set)transform(keys)); + } + + /** {@inheritDoc} */ + @Override public void clear() throws IgniteCheckedException { + delegate.clear(); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture clearAsync() { + return delegate.clearAsync(); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture clearAsync(Object key) { + return delegate.clearAsync(keyTransformer.transform(key)); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture clearAllAsync(Set keys) { + return delegate.clearAllAsync((Set)transform(keys)); + } + + /** {@inheritDoc} */ + @Nullable @Override public Object getAndRemove(Object key) throws IgniteCheckedException { + return delegate.getAndRemove(keyTransformer.transform(key)); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture getAndRemoveAsync(Object key) { + return delegate.getAndRemoveAsync(keyTransformer.transform(key)); + } + + /** {@inheritDoc} */ + @Override public boolean remove(Object key) throws IgniteCheckedException { + return delegate.remove(keyTransformer.transform(key)); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture removeAsync(Object key) { + return delegate.removeAsync(keyTransformer.transform(key)); + } + + /** {@inheritDoc} */ + @Override public boolean remove(Object key, Object val) throws IgniteCheckedException { + return delegate.remove(keyTransformer.transform(key), val); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture removeAsync(Object key, Object val) { + return delegate.removeAsync(keyTransformer.transform(key), val); + } + + /** {@inheritDoc} */ + @Override public void removeAll(@Nullable Collection keys) throws IgniteCheckedException { + delegate.removeAll(transform(keys)); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture removeAllAsync(@Nullable Collection keys) { + return delegate.removeAllAsync(transform(keys)); + } + + /** {@inheritDoc} */ + @Override public void removeAll() throws IgniteCheckedException { + delegate.removeAll(); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture removeAllAsync() { + return delegate.removeAllAsync(); + } + + /** {@inheritDoc} */ + @Override public boolean lock(Object key, long timeout) throws IgniteCheckedException { + return delegate.lock(keyTransformer.transform(key), timeout); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture lockAsync(Object key, long timeout) { + return delegate.lockAsync(keyTransformer.transform(key), timeout); + } + + /** {@inheritDoc} */ + @Override public boolean lockAll(@Nullable Collection keys, long timeout) throws IgniteCheckedException { + return delegate.lockAll(transform(keys), timeout); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture lockAllAsync(@Nullable Collection keys, long timeout) { + return delegate.lockAllAsync(transform(keys), timeout); + } + + /** {@inheritDoc} */ + @Override public void unlock(Object key) throws IgniteCheckedException { + delegate.unlock(keyTransformer.transform(key)); + } + + /** {@inheritDoc} */ + @Override public void unlockAll(@Nullable Collection keys) throws IgniteCheckedException { + delegate.unlockAll(transform(keys)); + } + + /** {@inheritDoc} */ + @Override public boolean isLocked(Object key) { + return delegate.isLocked(keyTransformer.transform(key)); + } + + /** {@inheritDoc} */ + @Override public boolean isLockedByThread(Object key) { + return delegate.isLockedByThread(keyTransformer.transform(key)); + } + + /** {@inheritDoc} */ + @Override public int size() { + return delegate.size(); + } + + /** {@inheritDoc} */ + @Override public long sizeLong() { + return delegate.sizeLong(); + } + + /** {@inheritDoc} */ + @Override public int localSize(CachePeekMode[] peekModes) throws IgniteCheckedException { + return delegate.localSize(peekModes); + } + + /** {@inheritDoc} */ + @Override public long localSizeLong(CachePeekMode[] peekModes) throws IgniteCheckedException { + return delegate.localSizeLong(peekModes); + } + + /** {@inheritDoc} */ + @Override public long localSizeLong(int partition, CachePeekMode[] peekModes) throws IgniteCheckedException { + return delegate.localSizeLong(partition, peekModes); + } + + /** {@inheritDoc} */ + @Override public int size(CachePeekMode[] peekModes) throws IgniteCheckedException { + return delegate.size(peekModes); + } + + /** {@inheritDoc} */ + @Override public long sizeLong(CachePeekMode[] peekModes) throws IgniteCheckedException { + return delegate.sizeLong(peekModes); + } + + /** {@inheritDoc} */ + @Override public long sizeLong(int partition, CachePeekMode[] peekModes) throws IgniteCheckedException { + return delegate.sizeLong(partition, peekModes); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture sizeAsync(CachePeekMode[] peekModes) { + return delegate.sizeAsync(peekModes); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture sizeLongAsync(CachePeekMode[] peekModes) { + return delegate.sizeLongAsync(peekModes); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture sizeLongAsync(int partition, CachePeekMode[] peekModes) { + return delegate.sizeLongAsync(partition, peekModes); + } + + /** {@inheritDoc} */ + @Override public int nearSize() { + return delegate.nearSize(); + } + + /** {@inheritDoc} */ + @Override public int primarySize() { + return delegate.primarySize(); + } + + /** {@inheritDoc} */ + @Override public long primarySizeLong() { + return delegate.primarySizeLong(); + } + + /** {@inheritDoc} */ + @Override public CacheConfiguration configuration() { + return delegate.configuration(); + } + + /** {@inheritDoc} */ + @Override public Affinity affinity() { + return delegate.affinity(); + } + + /** {@inheritDoc} */ + @Override public CacheMetrics clusterMetrics() { + return delegate.clusterMetrics(); + } + + /** {@inheritDoc} */ + @Override public CacheMetrics clusterMetrics(ClusterGroup grp) { + return delegate.clusterMetrics(grp); + } + + /** {@inheritDoc} */ + @Override public CacheMetrics localMetrics() { + return delegate.localMetrics(); + } + + /** {@inheritDoc} */ + @Override public CacheMetricsMXBean clusterMxBean() { + return delegate.clusterMxBean(); + } + + /** {@inheritDoc} */ + @Override public CacheMetricsMXBean localMxBean() { + return delegate.localMxBean(); + } + + /** {@inheritDoc} */ + @Override public long offHeapEntriesCount() { + return delegate.offHeapEntriesCount(); + } + + /** {@inheritDoc} */ + @Override public long offHeapAllocatedSize() { + return delegate.offHeapAllocatedSize(); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture rebalance() { + return delegate.rebalance(); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalCache forSubjectId(UUID subjId) { + return delegate.forSubjectId(subjId); + } + + /** {@inheritDoc} */ + @Nullable @Override public Object getForcePrimary(Object key) throws IgniteCheckedException { + return delegate.getForcePrimary(keyTransformer.transform(key)); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture getForcePrimaryAsync(Object key) { + return delegate.getForcePrimaryAsync(keyTransformer.transform(key)); + } + + /** {@inheritDoc} */ + @Override public Map getAllOutTx(Set keys) throws IgniteCheckedException { + return delegate.getAllOutTx((Set)transform(keys)); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture> getAllOutTxAsync(Set keys) { + return delegate.getAllOutTxAsync((Set)transform(keys)); + } + + /** {@inheritDoc} */ + @Override public boolean isIgfsDataCache() { + return delegate.isIgfsDataCache(); + } + + /** {@inheritDoc} */ + @Override public long igfsDataSpaceUsed() { + return delegate.igfsDataSpaceUsed(); + } + + /** {@inheritDoc} */ + @Override public long igfsDataSpaceMax() { + return delegate.igfsDataSpaceMax(); + } + + /** {@inheritDoc} */ + @Override public boolean isMongoDataCache() { + return delegate.isMongoDataCache(); + } + + /** {@inheritDoc} */ + @Override public boolean isMongoMetaCache() { + return delegate.isMongoMetaCache(); + } + + /** {@inheritDoc} */ + @Nullable @Override public ExpiryPolicy expiry() { + return delegate.expiry(); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalCache withExpiryPolicy(ExpiryPolicy plc) { + return delegate.withExpiryPolicy(plc); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalCache withNoRetries() { + return delegate.withNoRetries(); + } + + /** {@inheritDoc} */ + @Override public GridCacheContext context() { + return delegate.context(); + } + + /** {@inheritDoc} */ + @Override public void localLoadCache( + @Nullable IgniteBiPredicate p, + @Nullable Object... args + ) throws IgniteCheckedException { + delegate.localLoadCache(p, args); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture localLoadCacheAsync( + @Nullable IgniteBiPredicate p, + @Nullable Object... args + ) { + return delegate.localLoadCacheAsync(p, args); + } + + /** {@inheritDoc} */ + @Override public Object getTopologySafe(Object key) throws IgniteCheckedException { + return delegate.getTopologySafe(keyTransformer.transform(key)); + } + + /** {@inheritDoc} */ + @Nullable @Override public Object tryGetAndPut(Object key, Object val) throws IgniteCheckedException { + return delegate.tryGetAndPut(keyTransformer.transform(key), val); + } + + /** {@inheritDoc} */ + @Nullable @Override public EntryProcessorResult invoke( + @Nullable AffinityTopologyVersion topVer, + Object key, + EntryProcessor entryProcessor, + Object... args + ) throws IgniteCheckedException { + return delegate.invoke(topVer, key, entryProcessor, args); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture invokeAllAsync(Map map, Object... args) { + return delegate.invokeAllAsync(map, args); + } + + /** {@inheritDoc} */ + @Override public Map invokeAll(Map map, Object... args) throws IgniteCheckedException { + return delegate.invokeAll(map, args); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture invokeAllAsync(Set keys, EntryProcessor entryProcessor, Object... args) { + return delegate.invokeAllAsync((Set)transform(keys), entryProcessor, args); + } + + /** {@inheritDoc} */ + @Override public Map invokeAll(Set keys, EntryProcessor entryProcessor, Object... args) throws IgniteCheckedException { + return delegate.invokeAll((Set)transform(keys), entryProcessor, args); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture invokeAsync( + Object key, + EntryProcessor entryProcessor, + Object... args + ) { + return delegate.invokeAsync(keyTransformer.transform(key), entryProcessor, args); + } + + /** {@inheritDoc} */ + @Nullable @Override public EntryProcessorResult invoke( + Object key, + EntryProcessor entryProcessor, + Object... args + ) throws IgniteCheckedException { + return delegate.invoke(keyTransformer.transform(key), entryProcessor, args); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture removeAllConflictAsync(Map drMap) throws IgniteCheckedException { + return delegate.removeAllConflictAsync(drMap); + } + + /** {@inheritDoc} */ + @Override public void removeAllConflict(Map drMap) throws IgniteCheckedException { + delegate.removeAllConflictAsync(drMap); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture putAllConflictAsync(Map drMap) throws IgniteCheckedException { + return delegate.putAllConflictAsync(drMap); + } + + /** {@inheritDoc} */ + @Override public void putAllConflict(Map drMap) throws IgniteCheckedException { + delegate.putAllConflict(drMap); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalCache keepBinary() { + return delegate.keepBinary(); + } + + /** {@inheritDoc} */ + @Override public IgniteInternalCache cache() { + return delegate.cache(); + } + + /** {@inheritDoc} */ + @Override public Iterator iterator() { + return delegate.iterator(); + } + + /** + * @param keys Keys. + */ + private Collection transform(Collection keys) { + Collection res = new LinkedList<>(); + + for (Object o : keys) + res.add(keyTransformer.transform(o)); + + return res; + } + + /** + * @param map Map. + */ + private Map transform(Map map) { + Map res = new HashMap<>(); + + Set> ents = map.entrySet(); + + for (Map.Entry e : ents) + res.put(keyTransformer.transform(e.getKey()), e.getValue()); + + return res; + } +} diff --git a/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateCollectionRegion.java b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateCollectionRegion.java index 045f401d92f80..eb35a2c4709d7 100644 --- a/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateCollectionRegion.java +++ b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateCollectionRegion.java @@ -18,7 +18,6 @@ package org.apache.ignite.cache.hibernate; import org.apache.ignite.Ignite; -import org.apache.ignite.internal.processors.cache.IgniteInternalCache; import org.hibernate.cache.CacheException; import org.hibernate.cache.spi.CacheDataDescription; import org.hibernate.cache.spi.CollectionRegion; @@ -72,7 +71,7 @@ public class HibernateCollectionRegion extends HibernateTransactionalDataRegion * @param dataDesc Region data description. */ public HibernateCollectionRegion(HibernateRegionFactory factory, String name, - Ignite ignite, IgniteInternalCache cache, CacheDataDescription dataDesc) { + Ignite ignite, HibernateCacheProxy cache, CacheDataDescription dataDesc) { super(factory, name, ignite, cache, dataDesc); } diff --git a/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateEntityRegion.java b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateEntityRegion.java index 1ceda144a8355..ad5b1919f51b3 100644 --- a/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateEntityRegion.java +++ b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateEntityRegion.java @@ -18,7 +18,6 @@ package org.apache.ignite.cache.hibernate; import org.apache.ignite.Ignite; -import org.apache.ignite.internal.processors.cache.IgniteInternalCache; import org.hibernate.cache.CacheException; import org.hibernate.cache.spi.CacheDataDescription; import org.hibernate.cache.spi.EntityRegion; @@ -62,7 +61,7 @@ public class HibernateEntityRegion extends HibernateTransactionalDataRegion impl * @param dataDesc Region data description. */ public HibernateEntityRegion(HibernateRegionFactory factory, String name, Ignite ignite, - IgniteInternalCache cache, CacheDataDescription dataDesc) { + HibernateCacheProxy cache, CacheDataDescription dataDesc) { super(factory, name, ignite, cache, dataDesc); } diff --git a/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateGeneralDataRegion.java b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateGeneralDataRegion.java index fbac624f66af1..2f1a11dc8baa3 100644 --- a/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateGeneralDataRegion.java +++ b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateGeneralDataRegion.java @@ -19,7 +19,6 @@ import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.internal.processors.cache.IgniteInternalCache; import org.hibernate.cache.CacheException; import org.hibernate.cache.spi.GeneralDataRegion; import org.hibernate.cache.spi.QueryResultsRegion; @@ -38,7 +37,7 @@ public class HibernateGeneralDataRegion extends HibernateRegion implements Gener * @param cache Region cache. */ public HibernateGeneralDataRegion(HibernateRegionFactory factory, String name, - Ignite ignite, IgniteInternalCache cache) { + Ignite ignite, HibernateCacheProxy cache) { super(factory, name, ignite, cache); } diff --git a/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateKeyTransformer.java b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateKeyTransformer.java new file mode 100644 index 0000000000000..97fc0e9eb15e6 --- /dev/null +++ b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateKeyTransformer.java @@ -0,0 +1,29 @@ +/* + * 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.cache.hibernate; + +/** + * An interface for transforming hibernate keys to Ignite keys. + */ +public interface HibernateKeyTransformer { + /** + * @param key Hibernate key. + * @return Transformed key. + */ + public Object transform(Object key); +} diff --git a/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateKeyWrapper.java b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateKeyWrapper.java new file mode 100644 index 0000000000000..7de440ebb032c --- /dev/null +++ b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateKeyWrapper.java @@ -0,0 +1,72 @@ +/* + * 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.cache.hibernate; + +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * Hibernate cache key wrapper. + */ +public class HibernateKeyWrapper { + /** Key. */ + private final Object key; + + /** Entry. */ + private final String entry; + + /** */ + private final String tenantId; + + /** + * @param key Key. + * @param entry Entry. + * @param tenantId Tenant ID. + */ + HibernateKeyWrapper(Object key, String entry, String tenantId) { + this.key = key; + this.entry = entry; + this.tenantId = tenantId; + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) return true; + + if (o == null || getClass() != o.getClass()) + return false; + + HibernateKeyWrapper that = (HibernateKeyWrapper) o; + + return (key != null ? key.equals(that.key) : that.key == null) && + (entry != null ? entry.equals(that.entry) : that.entry == null) && + (tenantId != null ? tenantId.equals(that.tenantId) : that.tenantId == null); + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + int res = key != null ? key.hashCode() : 0; + res = 31 * res + (entry != null ? entry.hashCode() : 0); + res = 31 * res + (tenantId != null ? tenantId.hashCode() : 0); + return res; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(HibernateKeyWrapper.class, this); + } +} diff --git a/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateNaturalIdRegion.java b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateNaturalIdRegion.java index 99d5348dea479..862a4228f2767 100644 --- a/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateNaturalIdRegion.java +++ b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateNaturalIdRegion.java @@ -18,7 +18,6 @@ package org.apache.ignite.cache.hibernate; import org.apache.ignite.Ignite; -import org.apache.ignite.internal.processors.cache.IgniteInternalCache; import org.hibernate.cache.CacheException; import org.hibernate.cache.spi.CacheDataDescription; import org.hibernate.cache.spi.NaturalIdRegion; @@ -52,7 +51,7 @@ public class HibernateNaturalIdRegion extends HibernateTransactionalDataRegion i * @param dataDesc Region data description. */ public HibernateNaturalIdRegion(HibernateRegionFactory factory, String name, - Ignite ignite, IgniteInternalCache cache, CacheDataDescription dataDesc) { + Ignite ignite, HibernateCacheProxy cache, CacheDataDescription dataDesc) { super(factory, name, ignite, cache, dataDesc); } diff --git a/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateNonStrictAccessStrategy.java b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateNonStrictAccessStrategy.java index 1cb8d48e0be2f..a36d7e786d739 100644 --- a/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateNonStrictAccessStrategy.java +++ b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateNonStrictAccessStrategy.java @@ -21,7 +21,6 @@ import java.util.Set; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.internal.processors.cache.IgniteInternalCache; import org.apache.ignite.internal.util.GridLeanMap; import org.apache.ignite.internal.util.GridLeanSet; import org.apache.ignite.internal.util.typedef.F; @@ -67,7 +66,7 @@ public class HibernateNonStrictAccessStrategy extends HibernateAccessStrategyAda * @param cache Cache. * @param writeCtx Thread local instance used to track updates done during one Hibernate transaction. */ - protected HibernateNonStrictAccessStrategy(Ignite ignite, IgniteInternalCache cache, ThreadLocal writeCtx) { + protected HibernateNonStrictAccessStrategy(Ignite ignite, HibernateCacheProxy cache, ThreadLocal writeCtx) { super(ignite, cache); this.writeCtx = (ThreadLocal)writeCtx; @@ -212,7 +211,7 @@ void removed(Object key) { * @param cache Cache. * @throws IgniteCheckedException If failed. */ - void updateCache(IgniteInternalCache cache) throws IgniteCheckedException { + void updateCache(HibernateCacheProxy cache) throws IgniteCheckedException { if (!F.isEmpty(rmvs)) cache.removeAll(rmvs); diff --git a/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateQueryResultsRegion.java b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateQueryResultsRegion.java index e3303a71ff413..0b9a43d6a13d6 100644 --- a/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateQueryResultsRegion.java +++ b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateQueryResultsRegion.java @@ -18,7 +18,6 @@ package org.apache.ignite.cache.hibernate; import org.apache.ignite.Ignite; -import org.apache.ignite.internal.processors.cache.IgniteInternalCache; import org.hibernate.Query; import org.hibernate.cache.spi.QueryResultsRegion; @@ -65,7 +64,7 @@ public class HibernateQueryResultsRegion extends HibernateGeneralDataRegion impl * @param cache Region cache. */ public HibernateQueryResultsRegion(HibernateRegionFactory factory, String name, - Ignite ignite, IgniteInternalCache cache) { + Ignite ignite, HibernateCacheProxy cache) { super(factory, name, ignite, cache); } } \ No newline at end of file diff --git a/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateReadOnlyAccessStrategy.java b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateReadOnlyAccessStrategy.java index 58a2c4b8779d4..cdef80e0a0c0b 100644 --- a/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateReadOnlyAccessStrategy.java +++ b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateReadOnlyAccessStrategy.java @@ -19,7 +19,6 @@ import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.internal.processors.cache.IgniteInternalCache; import org.hibernate.cache.CacheException; import org.hibernate.cache.spi.access.AccessType; import org.hibernate.cache.spi.access.SoftLock; @@ -60,7 +59,7 @@ public class HibernateReadOnlyAccessStrategy extends HibernateAccessStrategyAdap * @param ignite Grid. * @param cache Cache. */ - public HibernateReadOnlyAccessStrategy(Ignite ignite, IgniteInternalCache cache) { + public HibernateReadOnlyAccessStrategy(Ignite ignite, HibernateCacheProxy cache) { super(ignite, cache); } diff --git a/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateReadWriteAccessStrategy.java b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateReadWriteAccessStrategy.java index bbb1d4e5dfdc2..625b05061ff0f 100644 --- a/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateReadWriteAccessStrategy.java +++ b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateReadWriteAccessStrategy.java @@ -21,7 +21,6 @@ import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; -import org.apache.ignite.internal.processors.cache.IgniteInternalCache; import org.apache.ignite.internal.util.GridLeanSet; import org.apache.ignite.transactions.Transaction; import org.hibernate.cache.CacheException; @@ -68,7 +67,7 @@ public class HibernateReadWriteAccessStrategy extends HibernateAccessStrategyAda * @param cache Cache. * @param txCtx Thread local instance used to track updates done during one Hibernate transaction. */ - protected HibernateReadWriteAccessStrategy(Ignite ignite, IgniteInternalCache cache, ThreadLocal txCtx) { + protected HibernateReadWriteAccessStrategy(Ignite ignite, HibernateCacheProxy cache, ThreadLocal txCtx) { super(ignite, cache); this.txCtx = (ThreadLocal)txCtx; diff --git a/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateRegion.java b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateRegion.java index 27479e90d8b7b..11a96d09096fb 100644 --- a/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateRegion.java +++ b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateRegion.java @@ -20,7 +20,6 @@ import java.util.Collections; import java.util.Map; import org.apache.ignite.Ignite; -import org.apache.ignite.internal.processors.cache.IgniteInternalCache; import org.hibernate.cache.CacheException; import org.hibernate.cache.spi.Region; @@ -35,7 +34,7 @@ public class HibernateRegion implements Region { private final String name; /** Cache instance. */ - protected final IgniteInternalCache cache; + protected final HibernateCacheProxy cache; /** Grid instance. */ protected Ignite ignite; @@ -46,8 +45,7 @@ public class HibernateRegion implements Region { * @param ignite Grid. * @param cache Region cache. */ - public HibernateRegion(HibernateRegionFactory factory, String name, Ignite ignite, - IgniteInternalCache cache) { + public HibernateRegion(HibernateRegionFactory factory, String name, Ignite ignite, HibernateCacheProxy cache) { this.factory = factory; this.name = name; this.ignite = ignite; diff --git a/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateRegionFactory.java b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateRegionFactory.java index 825abee7e2590..4e4be36ec7c49 100644 --- a/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateRegionFactory.java +++ b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateRegionFactory.java @@ -29,6 +29,7 @@ import org.apache.ignite.internal.util.typedef.G; import org.hibernate.cache.CacheException; import org.hibernate.cache.spi.CacheDataDescription; +import org.hibernate.cache.spi.CacheKey; import org.hibernate.cache.spi.CollectionRegion; import org.hibernate.cache.spi.EntityRegion; import org.hibernate.cache.spi.NaturalIdRegion; @@ -88,7 +89,7 @@ public class HibernateRegionFactory implements RegionFactory { private Ignite ignite; /** Default cache. */ - private IgniteInternalCache dfltCache; + private HibernateCacheProxy dfltCache; /** Default region access type. */ private AccessType dfltAccessType; @@ -99,6 +100,23 @@ public class HibernateRegionFactory implements RegionFactory { /** Map needed to provide the same transaction context for different regions. */ private final ThreadLocal threadLoc = new ThreadLocal(); + /** Key transformer. */ + private final HibernateKeyTransformer hibernate4transformer = new HibernateKeyTransformer() { + @Override public Object transform(Object key) { + if (key instanceof CacheKey) { + CacheKey cacheKey = (CacheKey)key; + + return new HibernateKeyWrapper( + cacheKey.getKey(), + cacheKey.getEntityOrRoleName(), + cacheKey.getTenantId() + ); + } + + return key; + } + }; + /** {@inheritDoc} */ @Override public void start(Settings settings, Properties props) throws CacheException { String gridCfg = props.getProperty(GRID_CONFIG_PROPERTY); @@ -138,10 +156,12 @@ public class HibernateRegionFactory implements RegionFactory { String dfltCacheName = props.getProperty(DFLT_CACHE_NAME_PROPERTY); if (dfltCacheName != null) { - dfltCache = ((IgniteKernal)ignite).getCache(dfltCacheName); + IgniteInternalCache dfltCache = ((IgniteKernal)ignite).getCache(dfltCacheName); if (dfltCache == null) throw new CacheException("Cache specified as default is not configured: " + dfltCacheName); + + this.dfltCache = new HibernateCacheProxy(dfltCache, hibernate4transformer); } IgniteLogger log = ignite.log().getLogger(HibernateRegionFactory.class); @@ -152,6 +172,7 @@ public class HibernateRegionFactory implements RegionFactory { /** {@inheritDoc} */ @Override public void stop() { + // No-op. } /** {@inheritDoc} */ @@ -213,7 +234,7 @@ ThreadLocal threadLocalForCache(String cacheName) { * @return Cache for given region. * @throws CacheException If cache for given region is not configured. */ - private IgniteInternalCache regionCache(String regionName) throws CacheException { + private HibernateCacheProxy regionCache(String regionName) throws CacheException { String cacheName = regionCaches.get(regionName); if (cacheName == null) { @@ -228,6 +249,6 @@ private IgniteInternalCache regionCache(String regionName) throw if (cache == null) throw new CacheException("Cache '" + cacheName + "' for region '" + regionName + "' is not configured."); - return cache; + return new HibernateCacheProxy(cache, hibernate4transformer); } } \ No newline at end of file diff --git a/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateTimestampsRegion.java b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateTimestampsRegion.java index 4cedae2070e31..8b4c243277ec9 100644 --- a/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateTimestampsRegion.java +++ b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateTimestampsRegion.java @@ -18,7 +18,6 @@ package org.apache.ignite.cache.hibernate; import org.apache.ignite.Ignite; -import org.apache.ignite.internal.processors.cache.IgniteInternalCache; import org.hibernate.cache.spi.TimestampsRegion; /** @@ -34,7 +33,7 @@ public class HibernateTimestampsRegion extends HibernateGeneralDataRegion implem * @param cache Region cache. */ public HibernateTimestampsRegion(HibernateRegionFactory factory, String name, - Ignite ignite, IgniteInternalCache cache) { + Ignite ignite, HibernateCacheProxy cache) { super(factory, name, ignite, cache); } } \ No newline at end of file diff --git a/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateTransactionalAccessStrategy.java b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateTransactionalAccessStrategy.java index 80f75a71afed6..ca5284917aa68 100644 --- a/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateTransactionalAccessStrategy.java +++ b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateTransactionalAccessStrategy.java @@ -61,7 +61,7 @@ public class HibernateTransactionalAccessStrategy extends HibernateAccessStrateg * @param ignite Grid. * @param cache Cache. */ - public HibernateTransactionalAccessStrategy(Ignite ignite, IgniteInternalCache cache) { + public HibernateTransactionalAccessStrategy(Ignite ignite, HibernateCacheProxy cache) { super(ignite, cache); } diff --git a/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateTransactionalDataRegion.java b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateTransactionalDataRegion.java index ed2ee01cf2280..581076a3fdbb6 100644 --- a/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateTransactionalDataRegion.java +++ b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateTransactionalDataRegion.java @@ -19,7 +19,6 @@ import org.apache.ignite.Ignite; import org.apache.ignite.configuration.TransactionConfiguration; -import org.apache.ignite.internal.processors.cache.IgniteInternalCache; import org.hibernate.cache.CacheException; import org.hibernate.cache.spi.CacheDataDescription; import org.hibernate.cache.spi.CollectionRegion; @@ -48,7 +47,7 @@ public class HibernateTransactionalDataRegion extends HibernateRegion implements * @param dataDesc Region data description. */ public HibernateTransactionalDataRegion(HibernateRegionFactory factory, String name, - Ignite ignite, IgniteInternalCache cache, CacheDataDescription dataDesc) { + Ignite ignite, HibernateCacheProxy cache, CacheDataDescription dataDesc) { super(factory, name, ignite, cache); this.dataDesc = dataDesc; diff --git a/modules/hibernate/src/test/java/org/apache/ignite/testsuites/IgniteBinaryHibernateTestSuite.java b/modules/hibernate/src/test/java/org/apache/ignite/testsuites/IgniteBinaryHibernateTestSuite.java new file mode 100644 index 0000000000000..3791baed9c93c --- /dev/null +++ b/modules/hibernate/src/test/java/org/apache/ignite/testsuites/IgniteBinaryHibernateTestSuite.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.testsuites; + +import junit.framework.TestSuite; +import org.apache.ignite.internal.binary.BinaryMarshaller; +import org.apache.ignite.testframework.config.GridTestProperties; + +/** + * + */ +public class IgniteBinaryHibernateTestSuite extends TestSuite { + /** + * @return Test suite. + * @throws Exception If failed. + */ + public static TestSuite suite() throws Exception { + GridTestProperties.setProperty(GridTestProperties.MARSH_CLASS_NAME, BinaryMarshaller.class.getName()); + + return IgniteHibernateTestSuite.suite(); + } +} From a352951d91edde9c0029a8bf435d61b4a7cd8c11 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Tue, 4 Jul 2017 20:24:52 +0300 Subject: [PATCH 65/90] IGNITE-4831: Add an option to disable MBeans. --- .../apache/ignite/IgniteSystemProperties.java | 8 ++ .../org/apache/ignite/cache/CacheManager.java | 11 ++ .../apache/ignite/internal/IgniteKernal.java | 35 +++-- .../apache/ignite/internal/IgnitionEx.java | 10 +- .../client/router/impl/GridTcpRouterImpl.java | 57 ++++++--- .../processors/cache/GridCacheProcessor.java | 10 +- .../ignite/internal/util/IgniteUtils.java | 17 ++- .../apache/ignite/spi/IgniteSpiAdapter.java | 3 + .../util/mbeans/GridMBeanDisableSelfTest.java | 121 ++++++++++++++++++ 9 files changed, 238 insertions(+), 34 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/util/mbeans/GridMBeanDisableSelfTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java index c4208a76f1a7d..e38d236f49964 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -382,6 +382,14 @@ public final class IgniteSystemProperties { */ public static final String IGNITE_MBEAN_APPEND_CLASS_LOADER_ID = "IGNITE_MBEAN_APPEND_CLASS_LOADER_ID"; + /** + * If property is set to {@code true}, then Ignite will disable MBeans registration. + * This may be helpful if MBeans are not allowed e.g. for security reasons. + * + * Default is {@code false} + */ + public static final String IGNITE_MBEANS_DISABLED = "IGNITE_MBEANS_DISABLED"; + /** * Property controlling size of buffer holding last exception. Default value of {@code 1000}. */ diff --git a/modules/core/src/main/java/org/apache/ignite/cache/CacheManager.java b/modules/core/src/main/java/org/apache/ignite/cache/CacheManager.java index 6e09d7276f3ad..a5e612ef986fe 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/CacheManager.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheManager.java @@ -44,7 +44,9 @@ import org.apache.ignite.internal.IgnitionEx; import org.apache.ignite.internal.mxbean.IgniteStandardMXBean; import org.apache.ignite.internal.processors.cache.IgniteCacheProxy; +import org.apache.ignite.internal.util.IgniteUtils; import org.apache.ignite.internal.util.typedef.internal.CU; +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.vm.TcpDiscoveryVmIpFinder; import org.jetbrains.annotations.Nullable; @@ -314,6 +316,9 @@ private ObjectName getObjectName(String cacheName, String objName) { /** {@inheritDoc} */ @Override public void enableManagement(String cacheName, boolean enabled) { + if(IgniteUtils.IGNITE_MBEANS_DISABLED) + return; + kernalGateway.readLock(); try { @@ -336,6 +341,9 @@ private ObjectName getObjectName(String cacheName, String objName) { /** {@inheritDoc} */ @Override public void enableStatistics(String cacheName, boolean enabled) { + if(IgniteUtils.IGNITE_MBEANS_DISABLED) + return; + kernalGateway.readLock(); try { @@ -389,6 +397,9 @@ private void registerCacheObject(Object mxbean, String name, String beanType) { * @param beanType Mxbean name. */ private void unregisterCacheObject(String name, String beanType) { + if(IgniteUtils.IGNITE_MBEANS_DISABLED) + return; + MBeanServer mBeanSrv = ignite.configuration().getMBeanServer(); Set registeredObjNames = mBeanSrv.queryNames(getObjectName(name, beanType), null); 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 2aa05f1b9ab11..9bada6c35ce3b 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 @@ -1494,6 +1494,9 @@ private void addSpiAttributes(IgniteSpi... spiList) throws IgniteCheckedExceptio /** @throws IgniteCheckedException If registration failed. */ private void registerKernalMBean() throws IgniteCheckedException { + if(U.IGNITE_MBEANS_DISABLED) + return; + try { kernalMBean = U.registerMBean( cfg.getMBeanServer(), @@ -1515,6 +1518,9 @@ private void registerKernalMBean() throws IgniteCheckedException { /** @throws IgniteCheckedException If registration failed. */ private void registerLocalNodeMBean() throws IgniteCheckedException { + if(U.IGNITE_MBEANS_DISABLED) + return; + ClusterLocalNodeMetricsMXBean mbean = new ClusterLocalNodeMetricsMXBeanImpl(ctx.discovery().localNode()); try { @@ -1546,6 +1552,8 @@ private void registerExecutorMBeans( ExecutorService utilityExecSvc, ExecutorService marshallerExecSvc ) throws IgniteCheckedException { + if(U.IGNITE_MBEANS_DISABLED) + return; pubExecSvcMBean = registerExecutorMBean(execSvc, "GridExecutionExecutor"); sysExecSvcMBean = registerExecutorMBean(sysExecSvc, "GridSystemExecutor"); mgmtExecSvcMBean = registerExecutorMBean(mgmtExecSvc, "GridManagementExecutor"); @@ -1567,6 +1575,7 @@ private void registerExecutorMBeans( */ private ObjectName registerExecutorMBean(ExecutorService exec, String name) throws IgniteCheckedException { assert exec != null; + assert !U.IGNITE_MBEANS_DISABLED; try { ObjectName res = U.registerMBean( @@ -1595,22 +1604,24 @@ private ObjectName registerExecutorMBean(ExecutorService exec, String name) thro * @return {@code True} if successfully unregistered, {@code false} otherwise. */ private boolean unregisterMBean(@Nullable ObjectName mbean) { - if (mbean != null) - try { - cfg.getMBeanServer().unregisterMBean(mbean); + if (mbean == null) + return true; - if (log.isDebugEnabled()) - log.debug("Unregistered MBean: " + mbean); + assert !U.IGNITE_MBEANS_DISABLED; - return true; - } - catch (JMException e) { - U.error(log, "Failed to unregister MBean.", e); + try { + cfg.getMBeanServer().unregisterMBean(mbean); - return false; - } + if (log.isDebugEnabled()) + log.debug("Unregistered MBean: " + mbean); + + return true; + } + catch (JMException e) { + U.error(log, "Failed to unregister MBean.", e); - return true; + return false; + } } /** 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 5b2c3fc53faea..8d54ba57b8762 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 @@ -1950,7 +1950,7 @@ private IgniteConfiguration initializeConfiguration(IgniteConfiguration cfg) if (myCfg.getUserAttributes() == null) myCfg.setUserAttributes(Collections.emptyMap()); - if (myCfg.getMBeanServer() == null) + if (myCfg.getMBeanServer() == null && !U.IGNITE_MBEANS_DISABLED) myCfg.setMBeanServer(ManagementFactory.getPlatformMBeanServer()); Marshaller marsh = myCfg.getMarshaller(); @@ -2430,6 +2430,11 @@ private void stopExecutors0(IgniteLogger log) { * @throws IgniteCheckedException If registration failed. */ private void registerFactoryMbean(MBeanServer srv) throws IgniteCheckedException { + if(U.IGNITE_MBEANS_DISABLED) + return; + + assert srv != null; + synchronized (mbeans) { GridMBeanServerData data = mbeans.get(srv); @@ -2480,6 +2485,9 @@ private void registerFactoryMbean(MBeanServer srv) throws IgniteCheckedException * Unregister delegate Mbean instance for {@link Ignition}. */ private void unregisterFactoryMBean() { + if(U.IGNITE_MBEANS_DISABLED) + return; + synchronized (mbeans) { Iterator> iter = mbeans.entrySet().iterator(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterImpl.java index 06a492924f408..6210bb06b765b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterImpl.java @@ -169,6 +169,32 @@ public GridTcpRouterImpl(GridTcpRouterConfiguration cfg) { "are in use) [firstPort=" + cfg.getPort() + ", lastPort=" + (cfg.getPort() + cfg.getPortRange()) + ", addr=" + hostAddr + ']'); + registerMBean(); + } + + /** + * Stops this router. + */ + @Override public void stop() { + if (srv != null) + srv.stop(); + + if (client != null) + client.stop(true); + + unregisterMBean(); + + if (log.isInfoEnabled()) + log.info("TCP router successfully stopped."); + } + + /** + * Try to register MBean. + */ + private void registerMBean() { + if (U.IGNITE_MBEANS_DISABLED) + return; + try { ObjectName objName = U.registerMBean( ManagementFactory.getPlatformMBeanServer(), @@ -189,28 +215,23 @@ public GridTcpRouterImpl(GridTcpRouterConfiguration cfg) { } /** - * Stops this router. + * Unregister MBean. */ - @Override public void stop() { - if (srv != null) - srv.stop(); - - if (client != null) - client.stop(true); + private void unregisterMBean() { + if (mbeanName == null) + return; - if (mbeanName != null) - try { - ManagementFactory.getPlatformMBeanServer().unregisterMBean(mbeanName); + assert !U.IGNITE_MBEANS_DISABLED; - if (log.isDebugEnabled()) - log.debug("Unregistered MBean: " + mbeanName); - } - catch (JMException e) { - U.error(log, "Failed to unregister MBean.", e); - } + try { + ManagementFactory.getPlatformMBeanServer().unregisterMBean(mbeanName); - if (log.isInfoEnabled()) - log.info("TCP router successfully stopped."); + if (log.isDebugEnabled()) + log.debug("Unregistered MBean: " + mbeanName); + } + catch (JMException e) { + U.error(log, "Failed to unregister MBean.", e); + } } /** 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 ccd7ae0bf500d..341b2c19d38e4 100755 --- 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 @@ -3497,6 +3497,9 @@ public void createMissingCaches() throws IgniteCheckedException { @SuppressWarnings("unchecked") private void registerMbean(Object o, @Nullable String cacheName, boolean near) throws IgniteCheckedException { + if(U.IGNITE_MBEANS_DISABLED) + return; + assert o != null; MBeanServer srvr = ctx.config().getMBeanServer(); @@ -3513,7 +3516,7 @@ private void registerMbean(Object o, @Nullable String cacheName, boolean near) U.registerCacheMBean(srvr, ctx.gridName(), cacheName, o.getClass().getName(), o, (Class)itf); } - catch (JMException e) { + catch (Throwable e) { throw new IgniteCheckedException("Failed to register MBean for component: " + o, e); } @@ -3530,6 +3533,9 @@ private void registerMbean(Object o, @Nullable String cacheName, boolean near) * @param near Near flag. */ private void unregisterMbean(Object o, @Nullable String cacheName, boolean near) { + if(U.IGNITE_MBEANS_DISABLED) + return; + assert o != null; MBeanServer srvr = ctx.config().getMBeanServer(); @@ -3545,7 +3551,7 @@ private void unregisterMbean(Object o, @Nullable String cacheName, boolean near) try { srvr.unregisterMBean(U.makeCacheMBeanName(ctx.gridName(), cacheName, o.getClass().getName())); } - catch (JMException e) { + catch (Throwable e) { U.error(log, "Failed to unregister MBean for component: " + o, e); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java index c2efb953c9135..84566872e5abf 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java @@ -134,6 +134,7 @@ import java.util.zip.ZipOutputStream; import javax.management.DynamicMBean; import javax.management.JMException; +import javax.management.MBeanRegistrationException; import javax.management.MBeanServer; import javax.management.MalformedObjectNameException; import javax.management.ObjectName; @@ -150,10 +151,10 @@ import org.apache.ignite.IgniteClientDisconnectedException; import org.apache.ignite.IgniteDeploymentException; import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteIllegalStateException; import org.apache.ignite.IgniteInterruptedException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.IgniteSystemProperties; -import org.apache.ignite.binary.BinaryObject; import org.apache.ignite.binary.BinaryRawReader; import org.apache.ignite.binary.BinaryRawWriter; import org.apache.ignite.cluster.ClusterGroupEmptyException; @@ -509,6 +510,9 @@ public abstract class IgniteUtils { } }; + /** Ignite MBeans disabled flag. */ + public static boolean IGNITE_MBEANS_DISABLED = IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_MBEANS_DISABLED); + /** * Initializes enterprise check. */ @@ -4423,10 +4427,13 @@ public static ObjectName makeCacheMBeanName(@Nullable String gridName, @Nullable * @param impl MBean implementation. * @param itf MBean interface. * @return JMX object name. + * @throws MBeanRegistrationException if MBeans are disabled. * @throws JMException If MBean creation failed. */ public static ObjectName registerMBean(MBeanServer mbeanSrv, @Nullable String gridName, @Nullable String grp, String name, T impl, @Nullable Class itf) throws JMException { + if(IGNITE_MBEANS_DISABLED) + throw new MBeanRegistrationException(new IgniteIllegalStateException("No MBeans are allowed.")); assert mbeanSrv != null; assert name != null; assert itf != null; @@ -4447,10 +4454,14 @@ public static ObjectName registerMBean(MBeanServer mbeanSrv, @Nullable Strin * @param impl MBean implementation. * @param itf MBean interface. * @return JMX object name. + * @throws MBeanRegistrationException if MBeans are disabled. * @throws JMException If MBean creation failed. */ public static ObjectName registerMBean(MBeanServer mbeanSrv, ObjectName name, T impl, Class itf) throws JMException { + if(IGNITE_MBEANS_DISABLED) + throw new MBeanRegistrationException(new IgniteIllegalStateException("MBeans are disabled.")); + assert mbeanSrv != null; assert name != null; assert itf != null; @@ -4473,10 +4484,14 @@ public static ObjectName registerMBean(MBeanServer mbeanSrv, ObjectName name * @param impl MBean implementation. * @param itf MBean interface. * @return JMX object name. + * @throws MBeanRegistrationException if MBeans are disabled. * @throws JMException If MBean creation failed. */ public static ObjectName registerCacheMBean(MBeanServer mbeanSrv, @Nullable String gridName, @Nullable String cacheName, String name, T impl, Class itf) throws JMException { + if(IGNITE_MBEANS_DISABLED) + throw new MBeanRegistrationException(new IgniteIllegalStateException("MBeans are disabled.")); + assert mbeanSrv != null; assert name != null; assert itf != null; 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 8879364dca5dd..e2350f0a50a3d 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 @@ -410,6 +410,8 @@ private static String format(String msg, Object locVal, Object rmtVal) { */ protected final void registerMBean(String gridName, T impl, Class mbeanItf) throws IgniteSpiException { + if(U.IGNITE_MBEANS_DISABLED) + return; MBeanServer jmx = ignite.configuration().getMBeanServer(); assert mbeanItf == null || mbeanItf.isInterface(); @@ -434,6 +436,7 @@ protected final void registerMBean(String g protected final void unregisterMBean() throws IgniteSpiException { // Unregister SPI MBean. if (spiMBean != null) { + assert !U.IGNITE_MBEANS_DISABLED; MBeanServer jmx = ignite.configuration().getMBeanServer(); assert jmx != null; diff --git a/modules/core/src/test/java/org/apache/ignite/util/mbeans/GridMBeanDisableSelfTest.java b/modules/core/src/test/java/org/apache/ignite/util/mbeans/GridMBeanDisableSelfTest.java new file mode 100644 index 0000000000000..f08f58bd9258a --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/util/mbeans/GridMBeanDisableSelfTest.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.util.mbeans; + +import java.util.concurrent.Callable; +import javax.management.MBeanRegistrationException; +import javax.management.MBeanServer; +import javax.management.ObjectName; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.util.IgniteUtils; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Disabling MBeans test. + */ +public class GridMBeanDisableSelfTest extends GridCommonAbstractTest { + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + IgniteUtils.IGNITE_MBEANS_DISABLED = true; + + super.beforeTestsStarted(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + IgniteUtils.IGNITE_MBEANS_DISABLED = false; + } + + /** + * Test MBean registration. + * + * @throws Exception Thrown if test fails. + */ + public void testCorrectMBeanInfo() throws Exception { + // Node should start and stopped with no errors. + try (final Ignite ignite = startGrid(0)) { + + // Cache should be created and closed with no errors. + try (IgniteCache cache = ignite.getOrCreateCache(new CacheConfiguration("MyCache"))) { + + final MBeanServer server = ignite.configuration().getMBeanServer(); + + GridTestUtils.assertThrowsWithCause( + new Callable() { + @Override public Void call() throws Exception { + U.registerMBean(server, ignite.name(), "dummy", "DummyMbean1", new DummyMBeanImpl(), DummyMBean.class); + + return null; + + } + }, MBeanRegistrationException.class); + + GridTestUtils.assertThrowsWithCause( + new Callable() { + @Override public Void call() throws Exception { + ObjectName objName = U.makeMBeanName( + ignite.name(), + "dummy", + "DummyMbean2" + ); + + U.registerMBean(server, objName, new DummyMBeanImpl(), DummyMBean.class); + + return null; + + } + }, MBeanRegistrationException.class); + + GridTestUtils.assertThrowsWithCause( + new Callable() { + @Override public Void call() throws Exception { + U.registerCacheMBean(server, ignite.name(), "MyCache", "DummyMbean3", + new DummyMBeanImpl(), DummyMBean.class); + + return null; + + } + }, MBeanRegistrationException.class); + } + } + } + + /** + * MBean dummy interface. + */ + interface DummyMBean { + /** */ + void noop(); + } + + /** + * MBean stub. + */ + static class DummyMBeanImpl implements DummyMBean { + /** {@inheritDoc} */ + @Override public void noop() { + // No op. + } + } +} \ No newline at end of file From e4d141e97ab4ec34b5fe6a7bc599413223944438 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Fri, 14 Jul 2017 14:40:02 +0300 Subject: [PATCH 66/90] IGNITE-5103 - Server drops client node from cluster when no heartbeat messages received in interval heartBeatsFrequency * maxMissedClientHeartBeats. --- .../ignite/spi/discovery/tcp/ServerImpl.java | 46 +++++- ...entDiscoverySpiFailureTimeoutSelfTest.java | 14 ++ .../tcp/TcpClientDiscoverySpiSelfTest.java | 2 +- .../TcpDiscoveryClientSuspensionSelfTest.java | 133 ++++++++++++++++++ 4 files changed, 193 insertions(+), 2 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryClientSuspensionSelfTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java index 6e5af2986dcff..75d8f83344d25 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java @@ -6386,6 +6386,12 @@ private class ClientMessageWorker extends MessageWorkerAdapter> pingFut = new AtomicReference<>(); @@ -6397,10 +6403,13 @@ private class ClientMessageWorker extends MessageWorkerAdapter maxHbInterval) { + TcpDiscoveryNode clientNode = ring.node(clientNodeId); + + if (clientNode != null) { + boolean failedNode; + + synchronized (mux) { + failedNode = failedNodes.containsKey(clientNode); + } + + if (!failedNode) { + String msg = "Client node considered as unreachable " + + "and will be dropped from cluster, " + + "because no heartbeat messages received in interval: " + + "TcpDiscoverySpi.getHeartbeatFrequency() * TcpDiscoverySpi.getMaxMissedClientHeartbeats() ms. " + + "It maybe caused by network problems or long GC pause on client node, try to increase mentioned " + + "parameters. " + + "[nodeId=" + clientNodeId + + ", heartBeatFrequency=" + spi.hbFreq + + ", maxMissedClientHeartbeats=" + spi.maxMissedClientHbs + + ']'; + + failNode(clientNodeId, msg); + + U.warn(log, msg); + } + } + } + } + /** {@inheritDoc} */ @Override protected void cleanup() { super.cleanup(); diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiFailureTimeoutSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiFailureTimeoutSelfTest.java index 35aa934e912a2..cc26c1cda6288 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiFailureTimeoutSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiFailureTimeoutSelfTest.java @@ -185,6 +185,20 @@ public void testClientReconnectOnCoordinatorRouterFail2() throws Exception { clientReconnectOnCoordinatorRouterFail(2); } + /** {@inheritDoc} */ + @Override public void testPingFailedClientNode() throws Exception { + int hb = maxMissedClientHbs; + + maxMissedClientHbs = Integer.MAX_VALUE; + + try { + super.testPingFailedClientNode(); + } + finally { + maxMissedClientHbs = hb; + } + } + /** * Test tries to provoke scenario when client sends reconnect message before router failure detected. * diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java index 0483a1ce8fbc6..419497753bbc2 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java @@ -142,7 +142,7 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest { private boolean longSockTimeouts; /** */ - private int maxMissedClientHbs = TcpDiscoverySpi.DFLT_MAX_MISSED_CLIENT_HEARTBEATS; + protected int maxMissedClientHbs = TcpDiscoverySpi.DFLT_MAX_MISSED_CLIENT_HEARTBEATS; /** */ private IgniteInClosure2X afterWrite; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryClientSuspensionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryClientSuspensionSelfTest.java new file mode 100644 index 0000000000000..7a1dd1ba23e5c --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryClientSuspensionSelfTest.java @@ -0,0 +1,133 @@ +/* + * 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; + +import java.util.Timer; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteSystemProperties; +import org.apache.ignite.Ignition; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.util.typedef.internal.U; +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; + +/** + * Test for missed client heartbeats. + */ +public class TcpDiscoveryClientSuspensionSelfTest extends GridCommonAbstractTest { + /** */ + private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + TcpDiscoverySpi disco = new TcpDiscoverySpi(); + + disco.setIpFinder(IP_FINDER); + disco.setHeartbeatFrequency(200); + disco.setMaxMissedClientHeartbeats(10); + + cfg.setDiscoverySpi(disco); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + System.setProperty(IgniteSystemProperties.IGNITE_DISCO_FAILED_CLIENT_RECONNECT_DELAY, "10000"); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + System.clearProperty(IgniteSystemProperties.IGNITE_DISCO_FAILED_CLIENT_RECONNECT_DELAY); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** + * @throws Exception If failed. + */ + public void testOneServer() throws Exception { + doTestClientSuspension(1); + } + + /** + * @throws Exception If failed. + */ + public void testTwoServers() throws Exception { + doTestClientSuspension(2); + } + + /** + * @throws Exception If failed. + */ + public void testThreeServers() throws Exception { + doTestClientSuspension(3); + } + + /** + * @param serverCnt Servers count. + * @throws Exception If failed. + */ + private void doTestClientSuspension(int serverCnt) throws Exception { + startGrids(serverCnt); + + Ignition.setClientMode(true); + + Ignite client = startGrid("client"); + + for (int i = 0; i < serverCnt; i++) + assertEquals(1, grid(i).cluster().forClients().nodes().size()); + + Thread.sleep(3000); + + for (int i = 0; i < serverCnt; i++) + assertEquals(1, grid(i).cluster().forClients().nodes().size()); + + suspendClientHeartbeats(client); + + Thread.sleep(3000); + + for (int i = 0; i < serverCnt; i++) + assertEquals(0, grid(i).cluster().forClients().nodes().size()); + } + + /** + * @param client Client. + */ + private void suspendClientHeartbeats(Ignite client) { + assert client.cluster().localNode().isClient(); + + ClientImpl impl = U.field(client.configuration().getDiscoverySpi(), "impl"); + + Timer timer = U.field(impl, "timer"); + + timer.cancel(); + + System.out.println("Heartbeats suspended"); + } +} From 45573945066113fd29548699f23c2bc9f22cef36 Mon Sep 17 00:00:00 2001 From: Tikhonov Nikolay Date: Wed, 21 Jun 2017 17:55:05 +0300 Subject: [PATCH 67/90] ignite-5489 Fixed possible connection leaks when loadPreviousValue set to true --- .../store/GridCacheStoreManagerAdapter.java | 9 +- .../cache/CacheConnectionLeakStoreTxTest.java | 287 ++++++++++++++++++ .../testsuites/IgniteCacheTestSuite4.java | 2 + 3 files changed, 295 insertions(+), 3 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheConnectionLeakStoreTxTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java index 11d9816ff1e6d..c35b8fb9d2c01 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java @@ -338,7 +338,12 @@ private CacheStore cacheStoreWrapper(GridKernalContext ctx, throw new IgniteCheckedException(new CacheLoaderException(e)); } finally { - sessionEnd0(tx, threwEx); + IgniteInternalTx tx0 = tx; + + if (tx0 != null && (tx0.dht() && tx0.local())) + tx0 = null; + + sessionEnd0(tx0, threwEx); } if (log.isDebugEnabled()) @@ -867,8 +872,6 @@ private void sessionEnd0(@Nullable IgniteInternalTx tx, boolean threwEx) throws lsnr.onSessionEnd(locSes, !threwEx); } - assert !sesHolder.get().ended(store); - store.sessionEnd(!threwEx); } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheConnectionLeakStoreTxTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheConnectionLeakStoreTxTest.java new file mode 100644 index 0000000000000..f6b735077e1d8 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheConnectionLeakStoreTxTest.java @@ -0,0 +1,287 @@ +/* + * 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.cache; + +import java.io.Serializable; +import java.util.concurrent.ConcurrentHashMap; +import javax.cache.Cache; +import javax.cache.configuration.Factory; +import javax.cache.integration.CacheLoaderException; +import javax.cache.integration.CacheWriterException; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.store.CacheStoreAdapter; +import org.apache.ignite.cache.store.CacheStoreSession; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.resources.CacheStoreSessionResource; +import org.apache.ignite.resources.IgniteInstanceResource; +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.junits.cache.TestCacheSession; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.apache.ignite.transactions.Transaction; +import org.apache.ignite.transactions.TransactionConcurrency; +import org.apache.ignite.transactions.TransactionIsolation; + +import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC; +import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; +import static org.apache.ignite.transactions.TransactionIsolation.READ_COMMITTED; +import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ; + +/** + * + */ +public class CacheConnectionLeakStoreTxTest extends GridCommonAbstractTest { + /** */ + private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + + /** */ + private static final int CLIENT_NODE = 1; + + /** */ + private static boolean client; + + /** */ + private static volatile boolean isLoadFromStore; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + TcpDiscoverySpi disco = new TcpDiscoverySpi(); + + disco.setIpFinder(IP_FINDER); + + cfg.setDiscoverySpi(disco); + cfg.setClientMode(client); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + startGrid(0); + + client = true; + + startGrid(CLIENT_NODE); + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + isLoadFromStore = false; + TestStore.sessions.clear(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + } + + /** + * @throws Exception If failed. + */ + public void testConnectionLeakOneBackupAtomic() throws Exception { + checkConnectionLeak(CacheAtomicityMode.ATOMIC, null, null); + } + + /** + * @throws Exception If failed. + */ + public void testConnectionLeakOneBackupAtomicLoadFromStore() throws Exception { + isLoadFromStore = true; + + checkConnectionLeak(CacheAtomicityMode.ATOMIC, null, null); + } + + /** + * @throws Exception If failed. + */ + public void testConnectionLeakOneBackupOptimisticRepeatableRead() throws Exception { + checkConnectionLeak(CacheAtomicityMode.TRANSACTIONAL, OPTIMISTIC, REPEATABLE_READ); + } + + /** + * @throws Exception If failed. + */ + public void testConnectionLeakOneBackupOptimisticRepeatableReadLoadFromStore() throws Exception { + isLoadFromStore = true; + + checkConnectionLeak(CacheAtomicityMode.TRANSACTIONAL, OPTIMISTIC, REPEATABLE_READ); + } + + /** + * @throws Exception If failed. + */ + public void testConnectionLeakOneBackupOptimisticReadCommitted() throws Exception { + checkConnectionLeak(CacheAtomicityMode.TRANSACTIONAL, OPTIMISTIC, READ_COMMITTED); + } + + /** + * @throws Exception If failed. + */ + public void testConnectionLeakOneBackupOptimisticReadCommittedLoadFromStore() throws Exception { + isLoadFromStore = true; + + checkConnectionLeak(CacheAtomicityMode.TRANSACTIONAL, OPTIMISTIC, READ_COMMITTED); + } + + /** + * @throws Exception If failed. + */ + public void testConnectionLeakOneBackupPessimisticRepeatableRead() throws Exception { + checkConnectionLeak(CacheAtomicityMode.TRANSACTIONAL, PESSIMISTIC, REPEATABLE_READ); + } + + /** + * @throws Exception If failed. + */ + public void testConnectionLeakOneBackupPessimisticReadCommitted() throws Exception { + checkConnectionLeak(CacheAtomicityMode.TRANSACTIONAL, PESSIMISTIC, READ_COMMITTED); + } + + /** + * @throws Exception If failed. + */ + public void testConnectionLeakOneBackupPessimisticReadCommittedLoadFromStore() throws Exception { + isLoadFromStore = true; + + checkConnectionLeak(CacheAtomicityMode.TRANSACTIONAL, PESSIMISTIC, READ_COMMITTED); + } + + /** + * @param atomicityMode Atomicity mode. + * @param txConcurrency Transaction concurrency. + * @param txIsolation Transaction isolation. + * + * @throws Exception If failed. + */ + private void checkConnectionLeak( + CacheAtomicityMode atomicityMode, + TransactionConcurrency txConcurrency, + TransactionIsolation txIsolation + ) throws Exception { + CacheConfiguration cacheCfg = new CacheConfiguration<>(); + + cacheCfg.setCacheMode(CacheMode.PARTITIONED); + cacheCfg.setAtomicityMode(atomicityMode); + cacheCfg.setCacheStoreFactory(new TestStoreFactory()); + cacheCfg.setReadThrough(true); + cacheCfg.setWriteThrough(false); + cacheCfg.setLoadPreviousValue(true); + + Ignite ignite = ignite(CLIENT_NODE); + IgniteCache cache = ignite.createCache(cacheCfg); + + try { + assertEquals(0, cache.size()); + + if (atomicityMode == CacheAtomicityMode.TRANSACTIONAL) { + try (Transaction tx = ignite.transactions().txStart(txConcurrency, txIsolation)) { + cacheOp(cache); + + tx.commit(); + } + } + else { + cacheOp(cache); + } + + assertTrue("Session was leak on nodes: " + TestStore.sessions, TestStore.sessions.isEmpty()); + } + finally { + cache.destroy(); + } + } + + /** + * @param cache Cache. + */ + private void cacheOp(IgniteCache cache) { + boolean b = cache.putIfAbsent(42, 42); + + log.info("PutIfAbsent: " + b); + + Integer val = cache.get(42); + + log.info("Get: " + val); + } + + /** + * + */ + private static class TestStoreFactory implements Factory> { + /** {@inheritDoc} */ + @Override public CacheStoreAdapter create() { + return new TestStore(); + } + } + + /** + * + */ + private static class TestStore extends CacheStoreAdapter implements Serializable { + /** */ + @IgniteInstanceResource + private Ignite ignite; + + /** */ + @CacheStoreSessionResource + private CacheStoreSession ses; + + /** */ + private CacheStoreSession NULL = new TestCacheSession(); + + /** */ + public static ConcurrentHashMap sessions = new ConcurrentHashMap<>(); + + /** {@inheritDoc} */ + @Override public Integer load(Integer key) throws CacheLoaderException { + addSession(); + + return isLoadFromStore ? key : null; + } + + /** {@inheritDoc} */ + @Override public void write(Cache.Entry e) throws CacheWriterException { + addSession(); + } + + /** {@inheritDoc} */ + @Override public void delete(Object key) throws CacheWriterException { + addSession(); + } + + /** */ + private void addSession() { + sessions.put(ses == null ? NULL : ses, ignite.cluster().localNode()); + } + + /** {@inheritDoc} */ + @Override public void sessionEnd(boolean commit) { + sessions.remove(ses == null ? NULL : ses); + } + } +} \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java index 2b446bbea8e6d..c1905af9db331 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java @@ -21,6 +21,7 @@ import org.apache.ignite.cache.store.jdbc.CacheJdbcStoreSessionListenerSelfTest; import org.apache.ignite.internal.processors.GridCacheTxLoadFromStoreOnLockSelfTest; import org.apache.ignite.internal.processors.cache.CacheClientStoreSelfTest; +import org.apache.ignite.internal.processors.cache.CacheConnectionLeakStoreTxTest; import org.apache.ignite.internal.processors.cache.CacheGetEntryOptimisticReadCommittedSeltTest; import org.apache.ignite.internal.processors.cache.CacheGetEntryOptimisticRepeatableReadSeltTest; import org.apache.ignite.internal.processors.cache.CacheGetEntryOptimisticSerializableSeltTest; @@ -279,6 +280,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(CacheStoreUsageMultinodeStaticStartTxTest.class); suite.addTestSuite(CacheStoreUsageMultinodeDynamicStartAtomicTest.class); suite.addTestSuite(CacheStoreUsageMultinodeDynamicStartTxTest.class); + suite.addTestSuite(CacheConnectionLeakStoreTxTest.class); suite.addTestSuite(GridCacheStoreManagerDeserializationTest.class); suite.addTestSuite(GridLocalCacheStoreManagerDeserializationTest.class); From 37535634ef3325aaf9923fd17d24038dfd5cee38 Mon Sep 17 00:00:00 2001 From: agura Date: Tue, 11 Jul 2017 16:24:54 +0300 Subject: [PATCH 68/90] ignite-5722 Cache entries stay in onheap after scan query execution for OFFHEAP_TIRED cache with expiry policy --- .../cache/query/GridCacheQueryManager.java | 7 +- .../ScanQueryOffheapExpiryPolicySelfTest.java | 112 ++++++++++++++++++ .../IgniteCacheQuerySelfTestSuite2.java | 3 + 3 files changed, 121 insertions(+), 1 deletion(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/ScanQueryOffheapExpiryPolicySelfTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java index 14b1106eb5e1c..7efb746d4428e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java @@ -3678,7 +3678,12 @@ private CacheObject value(GridCacheEntryEx entry, KeyCacheObject key) throws Ign if (expiryPlc != null) entry.unswap(); - return entry.peek(true, true, true, topVer, expiryPlc); + CacheObject cacheObj = entry.peek(true, true, true, topVer, expiryPlc); + + if (expiryPlc != null) + cctx.evicts().touch(entry, topVer); + + return cacheObj; } catch (GridCacheEntryRemovedException ignore) { entry = null; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/ScanQueryOffheapExpiryPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/ScanQueryOffheapExpiryPolicySelfTest.java new file mode 100644 index 0000000000000..e59e458b2371c --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/ScanQueryOffheapExpiryPolicySelfTest.java @@ -0,0 +1,112 @@ +/* + * 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.cache.query; + +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheMemoryMode; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.query.ScanQuery; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +import javax.cache.expiry.CreatedExpiryPolicy; +import javax.cache.expiry.Duration; +import java.util.concurrent.TimeUnit; + +import static org.apache.ignite.cache.CachePeekMode.OFFHEAP; +import static org.apache.ignite.cache.CachePeekMode.ONHEAP; + +/** + * + */ +public class ScanQueryOffheapExpiryPolicySelfTest extends GridCommonAbstractTest { + + /** Nodes count. */ + private static final int NODES_CNT = 2; + + /** Entries count */ + private static final int ENTRIES_CNT = 1024; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + CacheConfiguration ccfg = defaultCacheConfiguration(); + + ccfg.setAtomicityMode(CacheAtomicityMode.ATOMIC); + ccfg.setCacheMode(CacheMode.PARTITIONED); + ccfg.setBackups(1); + ccfg.setMemoryMode(CacheMemoryMode.OFFHEAP_TIERED); + ccfg.setExpiryPolicyFactory(CreatedExpiryPolicy.factoryOf(new Duration(TimeUnit.MINUTES, 10))); + + cfg.setCacheConfiguration(ccfg); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + startGridsMultiThreaded(NODES_CNT); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + stopAllGrids(); + } + + /** + * @throws Exception If failed. + */ + public void testEntriesMovedFromOnHeap() throws Exception { + Ignite ignite0 = grid(0); + Ignite ignite1 = grid(1); + + IgniteCache cache0 = ignite0.cache(null); + IgniteCache cache1 = ignite1.cache(null); + + populateCache(cache0); + + assertEquals(0, cache0.localSize(ONHEAP)); + assertEquals(0, cache1.localSize(ONHEAP)); + + assertEquals(ENTRIES_CNT, cache0.localSize(OFFHEAP) + cache1.localSize(OFFHEAP)); + + cache0.query(new ScanQuery<>()).getAll(); + cache1.query(new ScanQuery<>()).getAll(); + + assertEquals(0, cache0.localSize(ONHEAP)); + assertEquals(0, cache1.localSize(ONHEAP)); + + assertEquals(ENTRIES_CNT, cache0.localSize(OFFHEAP) + cache1.localSize(OFFHEAP)); + } + + /** + * @param cache Cache instance. + */ + private static void populateCache(IgniteCache cache) { + for (int i = 0; i < ENTRIES_CNT; i++) + cache.put(i, i); + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java index 8ac219f89f199..0241f86198420 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java +++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java @@ -50,6 +50,7 @@ import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalFieldsQuerySelfTest; import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalQueryCancelOrTimeoutSelfTest; import org.apache.ignite.internal.processors.cache.query.GridCacheSwapScanQuerySelfTest; +import org.apache.ignite.internal.processors.cache.query.ScanQueryOffheapExpiryPolicySelfTest; import org.apache.ignite.internal.processors.query.h2.sql.BaseH2CompareQueryTest; import org.apache.ignite.internal.processors.query.h2.sql.H2CompareBigQueryTest; import org.apache.ignite.spi.communication.tcp.GridOrderedMessageCancelSelfTest; @@ -88,6 +89,8 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(CacheQueryOffheapEvictDataLostTest.class); + suite.addTestSuite(ScanQueryOffheapExpiryPolicySelfTest.class); + // Ignite cache and H2 comparison. suite.addTestSuite(BaseH2CompareQueryTest.class); suite.addTestSuite(H2CompareBigQueryTest.class); From 97d3f42c1c95a6aafce1d0c300ccfe6708398c17 Mon Sep 17 00:00:00 2001 From: shtykh_roman Date: Wed, 7 Sep 2016 08:35:31 +0300 Subject: [PATCH 69/90] IGNITE-3809: Fix for ArrayIndexOutOfBoundsException in GridUnsafeLru. (cherry picked from commit 31b9bb8) --- .../util/offheap/unsafe/GridUnsafeLru.java | 30 ++++++++++++++++--- 1 file changed, 26 insertions(+), 4 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeLru.java b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeLru.java index aaff4f93a94f7..ea652171d8b2c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeLru.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeLru.java @@ -28,8 +28,7 @@ /** * Striped LRU queue. */ -@SuppressWarnings("ForLoopReplaceableByForEach") -class GridUnsafeLru { +@SuppressWarnings("ForLoopReplaceableByForEach") class GridUnsafeLru { /** Number of stripes. */ private final short cnt; @@ -47,6 +46,9 @@ class GridUnsafeLru { /** Current round-robin remove stripe index. */ private final AtomicInteger rmvIdx; + /** Max stripe index count. */ + private final int maxIdxCnt; + /** Released flag. */ private AtomicBoolean released = new AtomicBoolean(false); @@ -68,6 +70,8 @@ class GridUnsafeLru { addIdx = new AtomicInteger(); rmvIdx = new AtomicInteger(cnt / 2); + + maxIdxCnt = cnt - 1; } /** @@ -156,7 +160,7 @@ long entry(short order, long qAddr) { * @throws GridOffHeapOutOfMemoryException If failed. */ long offer(int part, long addr, int hash) throws GridOffHeapOutOfMemoryException { - return lrus[addIdx.getAndIncrement() % cnt].offer(part, addr, hash); + return lrus[incrementAndGet(addIdx, maxIdxCnt)].offer(part, addr, hash); } /** @@ -165,7 +169,7 @@ long offer(int part, long addr, int hash) throws GridOffHeapOutOfMemoryException * @return Queue node address. */ long prePoll() { - int idx = rmvIdx.getAndIncrement(); + int idx = incrementAndGet(rmvIdx, maxIdxCnt); // Must try to poll from each LRU. for (int i = 0; i < lrus.length; i++) { @@ -180,6 +184,7 @@ long prePoll() { /** * Removes polling node from the queue. + * * @param qAddr Queue node address. */ void poll(long qAddr) { @@ -215,6 +220,23 @@ void destruct() { } } + /** + * Atomically increments the given value by one, re-starting from 0 when the specified maximum is reached. + * + * @param value Value to increment. + * @param max Maximum after reaching which the value is reset to 0. + * @return Incremented value. + */ + private int incrementAndGet(AtomicInteger value, int max) { + while (true) { + int cur = value.get(); + int next = cur == max ? 0 : cur + 1; + + if (value.compareAndSet(cur, next)) + return next; + } + } + /** {@inheritDoc} */ @Override public String toString() { return S.toString(GridUnsafeLru.class, this); From f24969f7e908645444df622642967a5f7fd3db23 Mon Sep 17 00:00:00 2001 From: Evgenii Zhuravlev Date: Wed, 19 Jul 2017 19:30:07 +0300 Subject: [PATCH 70/90] IGNITE 5775 JobsProcessor fix bug with delay in compute --- .../processors/job/GridJobProcessor.java | 10 +-- .../IgniteComputeJobOneThreadTest.java | 79 +++++++++++++++++++ .../IgniteComputeGridTestSuite.java | 2 + 3 files changed, 86 insertions(+), 5 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/IgniteComputeJobOneThreadTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobProcessor.java index ea9cbd7dbd272..09f80847832a3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobProcessor.java @@ -1780,6 +1780,11 @@ private class JobEventListener implements GridJobEventListener { return; } + if (!activeJobs.remove(worker.getJobId(), worker)) + cancelledJobs.remove(worker.getJobId(), worker); + + heldJobs.remove(worker.getJobId()); + try { handleCollisions(); } @@ -1787,11 +1792,6 @@ private class JobEventListener implements GridJobEventListener { rwLock.readUnlock(); } } - - if (!activeJobs.remove(worker.getJobId(), worker)) - cancelledJobs.remove(worker.getJobId(), worker); - - heldJobs.remove(worker.getJobId()); } } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteComputeJobOneThreadTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteComputeJobOneThreadTest.java new file mode 100644 index 0000000000000..ccd8fa3765f14 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteComputeJobOneThreadTest.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.internal; + +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCompute; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.lang.IgniteFuture; +import org.apache.ignite.lang.IgniteRunnable; +import org.apache.ignite.spi.collision.fifoqueue.FifoQueueCollisionSpi; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Test of absence of gaps between jobs in compute + */ +public class IgniteComputeJobOneThreadTest extends GridCommonAbstractTest { + @Override protected IgniteConfiguration getConfiguration(String name) throws Exception { + FifoQueueCollisionSpi colSpi = new FifoQueueCollisionSpi(); + colSpi.setParallelJobsNumber(1); + + return super.getConfiguration(name) + .setMetricsUpdateFrequency(10000) + .setCollisionSpi(colSpi); + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + startGrid(0); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** {@inheritDoc} */ + @Override protected long getTestTimeout() { + return 10000; + } + + /** + * @throws Exception If failed. + */ + public void testNoTimeout() throws Exception { + Ignite ignite = ignite(0); + + IgniteFuture fut = null; + + IgniteCompute compute = ignite.compute().withAsync(); + + for (int i = 0; i < 10000; i++) { + compute.run(new IgniteRunnable() { + @Override public void run() { + + } + }); + } + + fut = compute.future(); + fut.get(); + + assertTrue(true); + } +} 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 8a501fdb801d6..c72bfe670b677 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 @@ -63,6 +63,7 @@ import org.apache.ignite.internal.GridTaskResultCacheSelfTest; import org.apache.ignite.internal.GridTaskTimeoutSelfTest; import org.apache.ignite.internal.IgniteComputeEmptyClusterGroupTest; +import org.apache.ignite.internal.IgniteComputeJobOneThreadTest; import org.apache.ignite.internal.IgniteComputeTopologyExceptionTest; import org.apache.ignite.internal.IgniteExecutorServiceTest; import org.apache.ignite.internal.IgniteExplicitImplicitDeploymentSelfTest; @@ -152,6 +153,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(TaskNodeRestartTest.class); suite.addTestSuite(IgniteRoundRobinErrorAfterClientReconnectTest.class); suite.addTestSuite(PublicThreadpoolStarvationTest.class); + suite.addTestSuite(IgniteComputeJobOneThreadTest.class); return suite; } From f3adb9559b42698771b0b9b5116dd535446d2bef Mon Sep 17 00:00:00 2001 From: vsisko Date: Wed, 26 Jul 2017 15:40:19 +0700 Subject: [PATCH 71/90] IGNITE-5781 Visor throws ClassCastException if cache store implementation is other than CacheJdbcPojoStore. --- .../ignite/internal/visor/cache/VisorCacheTypeMetadata.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheTypeMetadata.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheTypeMetadata.java index f17e5889663fc..a9ab8e061d3cb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheTypeMetadata.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheTypeMetadata.java @@ -108,7 +108,7 @@ public static Collection list(Collection qr } // Add JDBC types. - if (factory != null && factory instanceof CacheJdbcPojoStoreFactory) { + if (factory instanceof CacheJdbcPojoStoreFactory) { CacheJdbcPojoStoreFactory jdbcFactory = (CacheJdbcPojoStoreFactory) factory; JdbcType[] jdbcTypes = jdbcFactory.getTypes(); From 8c992fb8ba33a0c0ac5c0fb741ee8ffd515c0f31 Mon Sep 17 00:00:00 2001 From: Evgenii Zhuravlev Date: Tue, 1 Aug 2017 17:46:27 +0300 Subject: [PATCH 72/90] IGNITE-5775 Fix removing jobs from activeJobs for jobAlwaysActivate --- .../ignite/internal/processors/job/GridJobProcessor.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobProcessor.java index f88ea07d069bc..7889a5579c7bb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobProcessor.java @@ -1769,6 +1769,11 @@ private class JobEventListener implements GridJobEventListener { maxFinishedJobsTime.setIfGreater(execTime); if (jobAlwaysActivate) { + if (!activeJobs.remove(worker.getJobId(), worker)) + cancelledJobs.remove(worker.getJobId(), worker); + + heldJobs.remove(worker.getJobId()); + if (metricsUpdateFreq > -1L) updateJobMetrics(); } From f0f1c82c8f9877d952f639ffe56803043e53415b Mon Sep 17 00:00:00 2001 From: nikolay_tikhonov Date: Fri, 4 Aug 2017 14:58:52 +0300 Subject: [PATCH 73/90] IGNITE-5290 Partial back port commit from master (42293fa sboikov on 29.05.2017 at 16:41) Signed-off-by: nikolay_tikhonov --- .../CacheContinuousQueryHandler.java | 152 +++++--- .../CacheContinuousQueryListener.java | 5 + .../CacheContinuousQueryManager.java | 50 ++- .../continuous/GridContinuousProcessor.java | 2 - .../IgniteCacheMessageWriteTimeoutTest.java | 2 + ...ousQueryConcurrentPartitionUpdateTest.java | 341 ++++++++++++++++++ ...ntinuousQueryFailoverAbstractSelfTest.java | 116 +----- ...CacheContinuousQueryOrderingEventTest.java | 21 ++ .../IgniteCacheQuerySelfTestSuite3.java | 3 + 9 files changed, 513 insertions(+), 179 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryConcurrentPartitionUpdateTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java index 17f4308c0709b..165b8b77e0ab4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java @@ -141,7 +141,7 @@ public class CacheContinuousQueryHandler implements GridContinuousHandler private transient boolean keepBinary; /** */ - private transient ConcurrentMap rcvs; + private transient ConcurrentMap rcvs = new ConcurrentHashMap<>(); /** */ private transient ConcurrentMap entryBufs; @@ -340,8 +340,6 @@ public void keepBinary(boolean keepBinary) { ackBuf = new AcknowledgeBuffer(); - rcvs = new ConcurrentHashMap<>(); - this.nodeId = nodeId; this.routineId = routineId; @@ -490,6 +488,16 @@ public void keepBinary(boolean keepBinary) { sendBackupAcknowledge(ackBuf.acknowledgeOnTimeout(), routineId, ctx); } + @Override public void checkQueueOnTimeout(GridKernalContext ctx) { + for (PartitionRecovery rcv : rcvs.values()) { + Collection> evts = rcv.onTimeout( + ctx.cache().safeJcache(cacheName, cacheId), ctx.cache().context().cacheContext(cacheId)); + + if (!evts.isEmpty()) + locLsnr.onUpdated(evts); + } + } + @Override public void skipUpdateEvent(CacheContinuousQueryEvent evt, AffinityTopologyVersion topVer, boolean primary) { assert evt != null; @@ -557,20 +565,6 @@ private void prepareEntry(GridCacheContext cctx, UUID nodeId, CacheContinuousQue entry.prepareMarshal(cctx); } - /** - * Wait topology. - */ - public void waitTopologyFuture(GridKernalContext ctx) throws IgniteCheckedException { - GridCacheContext cctx = cacheContext(ctx); - - if (!cctx.isLocal()) { - cacheContext(ctx).affinity().affinityReadyFuture(initTopVer).get(); - - for (int partId = 0; partId < cacheContext(ctx).affinity().partitions(); partId++) - getOrCreatePartitionRecovery(ctx, partId); - } - } - /** {@inheritDoc} */ @Override public void unregister(UUID routineId, GridKernalContext ctx) { assert routineId != null; @@ -864,31 +858,7 @@ private String taskName() { PartitionRecovery rec = rcvs.get(partId); if (rec == null) { - Long partCntr = null; - - AffinityTopologyVersion initTopVer0 = initTopVer; - - if (initTopVer0 != null) { - GridCacheContext cctx = cacheContext(ctx); - - GridCacheAffinityManager aff = cctx.affinity(); - - if (initUpdCntrsPerNode != null) { - for (ClusterNode node : aff.nodes(partId, initTopVer)) { - Map map = initUpdCntrsPerNode.get(node.id()); - - if (map != null) { - partCntr = map.get(partId); - - break; - } - } - } - else if (initUpdCntrs != null) - partCntr = initUpdCntrs.get(partId); - } - - rec = new PartitionRecovery(ctx.log(CU.CONTINUOUS_QRY_LOG_CATEGORY), initTopVer0, partCntr); + rec = new PartitionRecovery(ctx.log(CU.CONTINUOUS_QRY_LOG_CATEGORY)); PartitionRecovery oldRec = rcvs.putIfAbsent(partId, rec); @@ -936,7 +906,7 @@ private CacheContinuousQueryEntry handleEntry(CacheContinuousQueryEntry e) { /** * */ - private static class PartitionRecovery { + private static class PartitionRecovery { /** Event which means hole in sequence. */ private static final CacheContinuousQueryEntry HOLE = new CacheContinuousQueryEntry(); @@ -949,6 +919,12 @@ private static class PartitionRecovery { /** */ private long lastFiredEvt; + /** */ + private long prevCheckCntr = -1; + + /** */ + private int pendingSize; + /** */ private AffinityTopologyVersion curTop = AffinityTopologyVersion.NONE; @@ -957,19 +933,9 @@ private static class PartitionRecovery { /** * @param log Logger. - * @param topVer Topology version. - * @param initCntr Update counters. */ - PartitionRecovery(IgniteLogger log, AffinityTopologyVersion topVer, @Nullable Long initCntr) { + PartitionRecovery(IgniteLogger log) { this.log = log; - - if (initCntr != null) { - assert topVer.topologyVersion() > 0 : topVer; - - this.lastFiredEvt = initCntr; - - curTop = topVer; - } } /** @@ -1067,6 +1033,17 @@ private static class PartitionRecovery { } } } + else if (!entry.isBackup() && + (entry.topologyVersion() != null && entry.topologyVersion().equals(curTop))) { + + if (log.isDebugEnabled()) + log.debug("Processed message out of order: " + entry); + + return !entry.isFiltered() ? + F.> + asList(new CacheContinuousQueryEvent(cache, cctx, entry)) : + Collections.>emptyList(); + } else { if (log.isDebugEnabled()) log.debug("Skip duplicate continuous query message: " + entry); @@ -1139,6 +1116,73 @@ private static class PartitionRecovery { return entries; } + + /** + * Check queue on stuck. + * + * @param cache Cache. + * @param cctx Cache context. + * + * @return Stacked events. + */ + Collection> onTimeout(IgniteCache cache, GridCacheContext cctx) { + List> res = Collections.emptyList(); + + synchronized (pendingEvts) { + // Visited this the first time. + if (prevCheckCntr == -1) { + prevCheckCntr = lastFiredEvt; + pendingSize = pendingEvts.size(); + + return Collections.emptyList(); + } + + // Pending event not found. + if (pendingEvts.size() == 0) { + prevCheckCntr = lastFiredEvt; + pendingSize = 0; + + if (log.isDebugEnabled()) + log.debug("No stuck events."); + + return Collections.emptyList(); + } + + if (log.isDebugEnabled()) + log.debug("Check stuck events [lastFiredEvt=" + lastFiredEvt + ", " + + "prevCheckCntr=" + prevCheckCntr + ", " + + "pendingSize=" + pendingSize + ", " + + "pendingEvts=" + pendingEvts + "]"); + + if (pendingSize > 0 && prevCheckCntr == lastFiredEvt) { + int pendingSize0 = Math.min(pendingSize, pendingEvts.size()); + + assert pendingSize0 > 0; + + res = new ArrayList<>(pendingSize0); + Iterator> iter = pendingEvts.entrySet().iterator(); + + for (int i = 0; i < pendingSize0; i++) { + Map.Entry e = iter.next(); + + if (e.getValue() != HOLE && !e.getValue().isFiltered()) + res.add(new CacheContinuousQueryEvent(cache, cctx, e.getValue())); + + iter.remove(); + + lastFiredEvt = e.getKey(); + } + } + + prevCheckCntr = lastFiredEvt; + pendingSize = pendingEvts.size(); + + if (log.isDebugEnabled()) + log.debug("Process stuck events [lastFiredEvt=" + lastFiredEvt + ", " + "evts=" + res + "]"); + } + + return res; + } } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryListener.java index 84b22f933e848..c9b486a5d9617 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryListener.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryListener.java @@ -68,6 +68,11 @@ public void onEntryUpdated(CacheContinuousQueryEvent evt, boolean primary, */ public void acknowledgeBackupOnTimeout(GridKernalContext ctx); + /** + * @param ctx Context. + */ + public void checkQueueOnTimeout(GridKernalContext ctx); + /** * @param evt Event * @param topVer Topology version. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java index 91c199184a2de..a5f647aa3d2b9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java @@ -99,6 +99,9 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter { /** */ private static final long BACKUP_ACK_FREQ = 5000; + /** */ + private static final long QUEUE_CHECKER_FREQ = 3000; + /** Listeners. */ private final ConcurrentMap lsnrs = new ConcurrentHashMap8<>(); @@ -139,6 +142,8 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter { cctx.time().schedule(new BackupCleaner(lsnrs, cctx.kernalContext()), BACKUP_ACK_FREQ, BACKUP_ACK_FREQ); } + + cctx.time().schedule(new QueueChecker(lsnrs, cctx.kernalContext()), QUEUE_CHECKER_FREQ, QUEUE_CHECKER_FREQ); } /** {@inheritDoc} */ @@ -655,18 +660,6 @@ private UUID executeQuery0(CacheEntryUpdatedListener locLsnr, autoUnsubscribe, pred).get(); - try { - if (hnd.isQuery() && cctx.userCache() && !onStart) - hnd.waitTopologyFuture(cctx.kernalContext()); - } - catch (IgniteCheckedException e) { - log.warning("Failed to start continuous query.", e); - - cctx.kernalContext().continuous().stopRoutine(id); - - throw new IgniteCheckedException("Failed to start continuous query.", e); - } - if (notifyExisting) { final Iterator it = cctx.cache().allEntries().iterator(); @@ -1196,8 +1189,37 @@ public BackupCleaner(Map lsnrs, GridKernalCo /** {@inheritDoc} */ @Override public void run() { - for (CacheContinuousQueryListener lsnr : lsnrs.values()) - lsnr.acknowledgeBackupOnTimeout(ctx); + if (!lsnrs.isEmpty()) { + for (CacheContinuousQueryListener lsnr : lsnrs.values()) + lsnr.acknowledgeBackupOnTimeout(ctx); + } + } + } + + /** + * Task flash backup queue. + */ + private static final class QueueChecker implements Runnable { + /** Listeners. */ + private final Map lsnrs; + + /** Context. */ + private final GridKernalContext ctx; + + /** + * @param lsnrs Listeners. + */ + QueueChecker(Map lsnrs, GridKernalContext ctx) { + this.lsnrs = lsnrs; + this.ctx = ctx; + } + + /** {@inheritDoc} */ + @Override public void run() { + if (!lsnrs.isEmpty()) { + for (CacheContinuousQueryListener lsnr : lsnrs.values()) + lsnr.checkQueueOnTimeout(ctx); + } } } 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 f0429bc3a3c91..50d29d1f8de6e 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 @@ -240,8 +240,6 @@ public GridContinuousProcessor(GridKernalContext ctx) { if (cctx != null && cntrsPerNode != null && !cctx.isLocal() && cctx.affinityNode()) cntrsPerNode.put(ctx.localNodeId(), cctx.topology().updateCounters(false)); - - routine.handler().updateCounters(topVer, cntrsPerNode, cntrs); } fut.onRemoteRegistered(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageWriteTimeoutTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageWriteTimeoutTest.java index 6256225f0a54e..339e39ac4e98b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageWriteTimeoutTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageWriteTimeoutTest.java @@ -70,6 +70,8 @@ public class IgniteCacheMessageWriteTimeoutTest extends GridCommonAbstractTest { * @throws Exception If failed. */ public void testMessageQueueLimit() throws Exception { + fail("https://ggsystems.atlassian.net/browse/GG-12398"); + startGridsMultiThreaded(3); for (int i = 0; i < 15; i++) { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryConcurrentPartitionUpdateTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryConcurrentPartitionUpdateTest.java new file mode 100644 index 0000000000000..c1675f328d706 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryConcurrentPartitionUpdateTest.java @@ -0,0 +1,341 @@ +/* + * 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.cache.query.continuous; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentSkipListSet; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import javax.cache.event.CacheEntryEvent; +import javax.cache.event.CacheEntryUpdatedListener; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.affinity.Affinity; +import org.apache.ignite.cache.query.CacheQueryEntryEvent; +import org.apache.ignite.cache.query.ContinuousQuery; +import org.apache.ignite.cache.query.QueryCursor; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.util.lang.GridAbsPredicate; +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; + +import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC; +import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; + +/** + * + */ +public class CacheContinuousQueryConcurrentPartitionUpdateTest extends GridCommonAbstractTest { + /** */ + private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); + + /** */ + private static String DEFAULT_CACHE_NAME = "cache-name"; + + /** */ + private boolean client; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + ((TcpDiscoverySpi) cfg.getDiscoverySpi()).setIpFinder(ipFinder); + + cfg.setClientMode(client); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + + super.afterTest(); + } + + /** + * @throws Exception If failed. + */ + public void testConcurrentUpdatePartitionAtomic() throws Exception { + concurrentUpdatePartition(ATOMIC, false); + } + + /** + * @throws Exception If failed. + */ + public void testConcurrentUpdatePartitionTx() throws Exception { + concurrentUpdatePartition(TRANSACTIONAL, false); + } + + /** + * @throws Exception If failed. + */ + public void testConcurrentUpdatePartitionAtomicClient() throws Exception { + concurrentUpdatePartition(ATOMIC, true); + } + + /** + * @throws Exception If failed. + */ + public void testConcurrentUpdatePartitionTxClient() throws Exception { + concurrentUpdatePartition(TRANSACTIONAL, true); + } + + /** + * @param atomicityMode Cache atomicity mode. + * @param clientMode Client mode. + * @throws Exception If failed. + */ + private void concurrentUpdatePartition(CacheAtomicityMode atomicityMode, boolean clientMode) throws Exception { + Ignite srv = startGrid(0); + + client = clientMode; + + Ignite client = startGrid(1); + + CacheConfiguration ccfg = new CacheConfiguration(DEFAULT_CACHE_NAME); + + ccfg.setWriteSynchronizationMode(FULL_SYNC); + ccfg.setAtomicityMode(atomicityMode); + + IgniteCache clientCache = client.createCache(ccfg); + + final AtomicInteger evtCnt = new AtomicInteger(); + + Affinity aff = srv.affinity(DEFAULT_CACHE_NAME); + + final List keys = new ArrayList<>(); + + final int KEYS = 1000; + + for (int i = 0; i < 10_000_000; i++) { + if (aff.partition(i) == 0) { + keys.add(i); + + if (keys.size() == KEYS) + break; + } + } + + assertEquals(KEYS, keys.size()); + + final int THREADS = 10; + final int UPDATES = 1000; + + final IgniteCache cache0 = clientMode ? client.cache(DEFAULT_CACHE_NAME) + : srv.cache(DEFAULT_CACHE_NAME); + + for (int i = 0; i < 10; i++) { + log.info("Iteration: " + i); + + ContinuousQuery qry = new ContinuousQuery<>(); + + final ConcurrentSkipListSet sets = new ConcurrentSkipListSet<>(); + + qry.setLocalListener(new CacheEntryUpdatedListener() { + @Override public void onUpdated(Iterable> evts) { + for (CacheEntryEvent evt : evts) { + assertNotNull(evt.getKey()); + assertNotNull(evt.getValue()); + + CacheQueryEntryEvent e = (CacheQueryEntryEvent)evt.unwrap(CacheQueryEntryEvent.class); + + sets.add(e.getPartitionUpdateCounter()); + + evtCnt.incrementAndGet(); + } + } + }); + + QueryCursor qryCur = clientCache.query(qry); + + GridTestUtils.runMultiThreaded(new Callable() { + @Override public Void call() throws Exception { + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + for (int i = 0; i < UPDATES; i++) + cache0.put(keys.get(rnd.nextInt(KEYS)), i); + + return null; + } + }, THREADS, "update"); + + GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + log.info("Events: " + evtCnt.get()); + + return evtCnt.get() >= THREADS * UPDATES; + } + }, 5000); + + assertEquals(THREADS * UPDATES, evtCnt.get()); + + qryCur.close(); + + evtCnt.set(0); + } + } + + /** + * @throws Exception If failed. + */ + public void testConcurrentUpdatesAndQueryStartAtomic() throws Exception { + concurrentUpdatesAndQueryStart(ATOMIC); + } + + /** + * @throws Exception If failed. + */ + public void testConcurrentUpdatesAndQueryStartTx() throws Exception { + concurrentUpdatesAndQueryStart(TRANSACTIONAL); + } + + /** + * @param atomicityMode Cache atomicity mode. + * @throws Exception If failed. + */ + private void concurrentUpdatesAndQueryStart(CacheAtomicityMode atomicityMode) throws Exception { + Ignite srv = startGrid(0); + + client = true; + + Ignite client = startGrid(1); + + CacheConfiguration ccfg = new CacheConfiguration(DEFAULT_CACHE_NAME); + + ccfg.setWriteSynchronizationMode(FULL_SYNC); + ccfg.setAtomicityMode(atomicityMode); + + IgniteCache clientCache = this.client ? client.createCache(ccfg) : srv.createCache(ccfg); + + Affinity aff = srv.affinity(DEFAULT_CACHE_NAME); + + final List keys = new ArrayList<>(); + + final int KEYS = 1_000; + + for (int i = 0; i < 10_000_000; i++) { + if (aff.partition(i) == 0) { + keys.add(i); + + if (keys.size() == KEYS) + break; + } + } + + assertEquals(KEYS, keys.size()); + + final int THREADS = 10; + final int UPDATES = 100; + + for (int i = 0; i < 5; i++) { + log.info("Iteration: " + i); + + ContinuousQuery qry = new ContinuousQuery<>(); + + final AtomicInteger evtCnt = new AtomicInteger(); + + final ConcurrentSkipListSet sets = new ConcurrentSkipListSet<>(); + + qry.setLocalListener(new CacheEntryUpdatedListener() { + @Override public void onUpdated(Iterable> evts) { + for (CacheEntryEvent evt : evts) { + assertNotNull(evt.getKey()); + assertNotNull(evt.getValue()); + + CacheQueryEntryEvent e = (CacheQueryEntryEvent)evt.unwrap(CacheQueryEntryEvent.class); + + sets.add(e.getPartitionUpdateCounter()); + + if ((Integer)evt.getValue() >= 0) + evtCnt.incrementAndGet(); + } + } + }); + + QueryCursor cur; + + final IgniteCache srvCache = srv.cache(DEFAULT_CACHE_NAME); + + final AtomicBoolean stop = new AtomicBoolean(); + + try { + IgniteInternalFuture fut = GridTestUtils.runMultiThreadedAsync(new Callable() { + @Override public Void call() throws Exception { + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + while (!stop.get()) + srvCache.put(keys.get(rnd.nextInt(KEYS)), rnd.nextInt(100) - 200); + + return null; + } + }, THREADS, "update"); + + U.sleep(1000); + + cur = clientCache.query(qry); + + U.sleep(1000); + + stop.set(true); + + fut.get(); + } + finally { + stop.set(true); + } + + GridTestUtils.runMultiThreadedAsync(new Callable() { + @Override public Void call() throws Exception { + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + for (int i = 0; i < UPDATES; i++) + srvCache.put(keys.get(rnd.nextInt(KEYS)), i); + + return null; + } + }, THREADS, "update"); + + GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + log.info("Events: " + evtCnt.get()); + + return evtCnt.get() >= THREADS * UPDATES; + } + }, 10000); + + if ((THREADS * UPDATES) != evtCnt.get()) + log.info("Entries: " + sets); + + assertEquals(THREADS * UPDATES, evtCnt.get()); + + cur.close(); + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java index d2cb710edbeb5..ffb7557ff1258 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java @@ -142,7 +142,6 @@ public abstract class CacheContinuousQueryFailoverAbstractSelfTest extends GridC TestCommunicationSpi commSpi = new TestCommunicationSpi(); commSpi.setSharedMemoryPort(-1); - commSpi.setIdleConnectionTimeout(100); cfg.setCommunicationSpi(commSpi); @@ -1387,7 +1386,7 @@ public void testBackupQueueCleanupClientQuery() throws Exception { @Override public boolean apply() { return backupQueue.isEmpty(); } - }, 2000); + }, 10000); assertTrue("Backup queue is not cleared: " + backupQueue, backupQueue.size() < BACKUP_ACK_THRESHOLD); @@ -1409,13 +1408,13 @@ public void testBackupQueueCleanupClientQuery() throws Exception { @Override public boolean apply() { return backupQueue.isEmpty(); } - }, ACK_FREQ + 2000); - - assertTrue("Backup queue is not cleared: " + backupQueue, backupQueue.isEmpty()); + }, ACK_FREQ + 20000); if (!latch.await(5, SECONDS)) fail("Failed to wait for notifications [exp=" + keys.size() + ", left=" + lsnr.latch.getCount() + ']'); + assertTrue("Backup queue is not cleared: " + backupQueue, backupQueue.isEmpty()); + cur.close(); assertFalse("Unexpected error during test, see log for details.", err); @@ -1465,9 +1464,7 @@ public void testBackupQueueEvict() throws Exception { @Override public boolean apply() { return backupQueue.isEmpty(); } - }, 2000); - - assertTrue("Backup queue is not cleared: " + backupQueue, backupQueue.size() < BACKUP_ACK_THRESHOLD); + }, 20000); boolean wait = waitForCondition(new GridAbsPredicate() { @Override public boolean apply() { @@ -1477,6 +1474,8 @@ public void testBackupQueueEvict() throws Exception { assertTrue("Entry evicted.", wait); + assertTrue("Backup queue is not cleared: " + backupQueue, backupQueue.size() < BACKUP_ACK_THRESHOLD); + GridTestUtils.waitForCondition(new GridAbsPredicate() { @Override public boolean apply() { return backupQueue.isEmpty(); @@ -2076,107 +2075,6 @@ public void testMultiThreadedFailover() throws Exception { assertFalse("Unexpected error during test, see log for details.", err); } - /** - * @throws Exception If failed. - */ - public void testMultiThreaded() throws Exception { - this.backups = 2; - - final int SRV_NODES = 3; - - startGridsMultiThreaded(SRV_NODES); - - client = true; - - Ignite qryClient = startGrid(SRV_NODES); - - final IgniteCache cache = qryClient.cache(null); - - CacheEventListener1 lsnr = new CacheEventListener1(true); - - ContinuousQuery qry = new ContinuousQuery<>(); - - qry.setLocalListener(lsnr); - - QueryCursor cur = cache.query(qry); - - client = false; - - final int SRV_IDX = SRV_NODES - 1; - - List keys = primaryKeys(ignite(SRV_IDX).cache(null), 10); - - final int THREADS = 10; - - for (int i = 0; i < keys.size(); i++) { - log.info("Iteration: " + i); - - Ignite srv = ignite(SRV_IDX); - - TestCommunicationSpi spi = (TestCommunicationSpi)srv.configuration().getCommunicationSpi(); - - spi.sndFirstOnly = new AtomicBoolean(false); - - final Integer key = keys.get(i); - - final AtomicInteger val = new AtomicInteger(); - - CountDownLatch latch = new CountDownLatch(THREADS); - - lsnr.latch = latch; - - IgniteInternalFuture fut = GridTestUtils.runMultiThreadedAsync(new Callable() { - @Override public Object call() throws Exception { - Integer val0 = val.getAndIncrement(); - - cache.put(key, val0); - - return null; - } - }, THREADS, "update-thread"); - - fut.get(); - - stopGrid(SRV_IDX); - - if (!latch.await(5, SECONDS)) - fail("Failed to wait for notifications [exp=" + THREADS + ", left=" + lsnr.latch.getCount() + ']'); - - assertEquals(THREADS, lsnr.allEvts.size()); - - Set vals = new HashSet<>(); - - boolean err = false; - - for (CacheEntryEvent evt : lsnr.allEvts) { - assertEquals(key, evt.getKey()); - assertNotNull(evt.getValue()); - - if (!vals.add((Integer)evt.getValue())) { - err = true; - - log.info("Extra event: " + evt); - } - } - - for (int v = 0; v < THREADS; v++) { - if (!vals.contains(v)) { - err = true; - - log.info("Event for value not received: " + v); - } - } - - assertFalse("Invalid events, see log for details.", err); - - lsnr.allEvts.clear(); - - startGrid(SRV_IDX); - } - - cur.close(); - } - /** * @param logAll If {@code true} logs all unexpected values. * @param expEvts Expected values. diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryOrderingEventTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryOrderingEventTest.java index 7d975f2a28209..4dcdd32745ac0 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryOrderingEventTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryOrderingEventTest.java @@ -28,6 +28,7 @@ import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import javax.cache.Cache; import javax.cache.configuration.FactoryBuilder; import javax.cache.event.CacheEntryEvent; import javax.cache.event.CacheEntryUpdatedListener; @@ -49,6 +50,7 @@ import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.PA; import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteAsyncCallback; import org.apache.ignite.resources.IgniteInstanceResource; import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; @@ -391,6 +393,25 @@ protected void doOrderingTest( } } + IgniteCache cache = grid(0).cache(ccfg.getName()); + + for (int i = 0; i < KEYS; i++) { + cache.put(new QueryTestKey(i), new QueryTestValue(-1)); + + cache.remove(new QueryTestValue(i)); + } + + GridTestUtils.waitForCondition(new PA() { + @Override public boolean apply() { + return qryCntr.get() >= 4 * KEYS * LISTENER_CNT * NODES; + } + }, 3000L); + + for (BlockingQueue> q : rcvdEvts) + q.clear(); + + qryCntr.set(0); + IgniteInternalFuture f = GridTestUtils.runMultiThreadedAsync(new Runnable() { @Override public void run() { ThreadLocalRandom rnd = ThreadLocalRandom.current(); diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite3.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite3.java index 032dd3b5d8426..47de0ff842418 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite3.java +++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite3.java @@ -21,6 +21,7 @@ import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousBatchAckTest; import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousBatchForceServerModeAckTest; import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryAsyncFilterListenerTest; +import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryConcurrentPartitionUpdateTest; import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryExecuteInPrimaryTest; import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFactoryAsyncFilterRandomOperationTest; import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFactoryFilterRandomOperationTest; @@ -122,6 +123,8 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(CacheKeepBinaryIterationNearEnabledTest.class); suite.addTestSuite(IgniteCacheContinuousQueryBackupQueueTest.class); + suite.addTestSuite(CacheContinuousQueryConcurrentPartitionUpdateTest.class); + return suite; } } From 3d64cf1a023ce71cf522b6ce843669fcf44a3145 Mon Sep 17 00:00:00 2001 From: Ivan Rakov Date: Thu, 30 Mar 2017 14:56:40 +0300 Subject: [PATCH 74/90] IGNITE-3477 - Fixed concurrent read-through (cherry picked from commit 31ffef4) --- .../ignite/internal/processors/cache/GridCacheAdapter.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) 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 7c5a54dfcb852..7aa48199bb28a 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 @@ -2014,9 +2014,13 @@ protected final IgniteInternalFuture> getAllAsync0( CacheObject cacheVal = ctx.toCacheObject(val); while (true) { - GridCacheEntryEx entry = entryEx(key); + GridCacheEntryEx entry = null; try { + entry = entryEx(key); + + entry.unswap(); + EntryGetResult verVal = entry.versionedValue( cacheVal, res.version(), From 737260b070c3178d91ef04f7b200da87c9a1874f Mon Sep 17 00:00:00 2001 From: Evgenii Zhuravlev Date: Thu, 10 Aug 2017 18:54:57 +0300 Subject: [PATCH 75/90] IGNITE-4991 Do not print out system properties when IGNITE_TO_STRING_INCLUDE_SENSITIVE is set to false --- .../main/java/org/apache/ignite/internal/IgniteKernal.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 9bada6c35ce3b..a1997b1adfed7 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 @@ -2213,7 +2213,7 @@ public GridCacheAdapter internalCache(@Nullable String name) { private void ackSystemProperties() { assert log != null; - if (log.isDebugEnabled()) + if (log.isDebugEnabled() && S.INCLUDE_SENSITIVE) for (Map.Entry entry : snapshot().entrySet()) log.debug("System property [" + entry.getKey() + '=' + entry.getValue() + ']'); } @@ -2384,7 +2384,7 @@ private void ackVmArguments(RuntimeMXBean rtBean) { assert log != null; // Ack IGNITE_HOME and VM arguments. - if (log.isInfoEnabled()) { + if (log.isInfoEnabled() && S.INCLUDE_SENSITIVE) { log.info("IGNITE_HOME=" + cfg.getIgniteHome()); log.info("VM arguments: " + rtBean.getInputArguments()); } From b7bf1c09e8404d10eac57c13c9c6720c040d0c8e Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Fri, 14 Jul 2017 20:14:47 +0300 Subject: [PATCH 76/90] IGNITE-5452: GridTimeoutProcessor can hang on stop. This closes #2279. (cherry picked from commit b95c261) --- .../timeout/GridTimeoutProcessor.java | 18 +- .../IgniteTxRemoveTimeoutObjectsTest.java | 194 ++++++++++++++++++ .../timeout/GridTimeoutProcessorSelfTest.java | 68 ++++-- .../testsuites/IgniteCacheTestSuite3.java | 3 + 4 files changed, 265 insertions(+), 18 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxRemoveTimeoutObjectsTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessor.java index 0bbf9c376fc60..7fabd5cadb6a1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessor.java @@ -37,7 +37,7 @@ */ public class GridTimeoutProcessor extends GridProcessorAdapter { /** */ - private final IgniteThread timeoutWorker; + private final TimeoutWorker timeoutWorker; /** Time-based sorted set for timeout objects. */ private final GridConcurrentSkipListSet timeoutObjs = @@ -62,13 +62,12 @@ public class GridTimeoutProcessor extends GridProcessorAdapter { public GridTimeoutProcessor(GridKernalContext ctx) { super(ctx); - timeoutWorker = new IgniteThread(ctx.config().getGridName(), "grid-timeout-worker", - new TimeoutWorker()); + timeoutWorker = new TimeoutWorker(); } /** {@inheritDoc} */ @Override public void start() { - timeoutWorker.start(); + new IgniteThread(timeoutWorker).start(); if (log.isDebugEnabled()) log.debug("Timeout processor started."); @@ -76,7 +75,7 @@ public GridTimeoutProcessor(GridKernalContext ctx) { /** {@inheritDoc} */ @Override public void stop(boolean cancel) throws IgniteCheckedException { - U.interrupt(timeoutWorker); + timeoutWorker.cancel(); U.join(timeoutWorker); if (log.isDebugEnabled()) @@ -159,6 +158,13 @@ private class TimeoutWorker extends GridWorker { timeoutObj.onTimeout(); } catch (Throwable e) { + if (isCancelled() && !(e instanceof Error)){ + if (log.isDebugEnabled()) + log.debug("Error when executing timeout callback: " + timeoutObj); + + return; + } + U.error(log, "Error when executing timeout callback: " + timeoutObj, e); if (e instanceof Error) @@ -170,7 +176,7 @@ private class TimeoutWorker extends GridWorker { } synchronized (mux) { - while (true) { + while (!isCancelled()) { // Access of the first element must be inside of // synchronization block, so we don't miss out // on thread notification events sent from diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxRemoveTimeoutObjectsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxRemoveTimeoutObjectsTest.java new file mode 100644 index 0000000000000..bdb73ad303448 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxRemoveTimeoutObjectsTest.java @@ -0,0 +1,194 @@ +/* + * 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.cache.distributed; + +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.processors.cache.GridCacheAbstractSelfTest; +import org.apache.ignite.internal.processors.timeout.GridTimeoutObject; +import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor; +import org.apache.ignite.internal.util.typedef.G; +import org.apache.ignite.internal.util.typedef.X; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.transactions.Transaction; +import org.apache.ignite.transactions.TransactionTimeoutException; + +import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; +import static org.apache.ignite.transactions.TransactionIsolation.SERIALIZABLE; + +/** + * Test correctness of rollback a transaction with timeout during the grid stop. + */ +public class IgniteTxRemoveTimeoutObjectsTest extends GridCacheAbstractSelfTest { + /** */ + private static final int PUT_CNT = 1000; + + /** {@inheritDoc} */ + @Override protected int gridCount() { + return 3; + } + + /** {@inheritDoc} */ + @Override protected long getTestTimeout() { + return 60_000; + } + + /** + * @throws Exception If failed. + */ + public void testTxRemoveTimeoutObjects() throws Exception { + IgniteCache cache0 = grid(0).cache(null); + IgniteCache cache1 = grid(1).cache(null); + + // start additional grid to be closed. + IgniteCache cacheAdditional = startGrid(gridCount()).cache(null); + + for (int i = 0; i < PUT_CNT; i++) + cache0.put(i, Integer.MAX_VALUE); + + logTimeoutObjectsFrequency(); + + info("Tx1 started"); + try (Transaction tx = grid(gridCount()).transactions().txStart(PESSIMISTIC, SERIALIZABLE, 100, PUT_CNT)) { + try { + for (int i = 0; i < PUT_CNT; i++) { + cacheAdditional.put(i, Integer.MIN_VALUE); + + if (i % 100 == 0) + logTimeoutObjectsFrequency(); + } + + U.sleep(200); + + tx.commit(); + + fail("A timeout should have happened."); + } + catch (Exception e) { + assertTrue(X.hasCause(e, TransactionTimeoutException.class)); + } + } + + assertDoesNotContainLockTimeoutObjects(); + + logTimeoutObjectsFrequency(); + + stopGrid(gridCount()); + + awaitPartitionMapExchange(); + + info("Grid2 closed."); + + assertDoesNotContainLockTimeoutObjects(); + + logTimeoutObjectsFrequency(); + + // Check that the values have not changed and lock can be acquired. + try (Transaction tx2 = grid(1).transactions().txStart(PESSIMISTIC, SERIALIZABLE)) { + info("Tx2 started"); + + for (int i = 0; i < PUT_CNT; i++) { + assertEquals(cache1.get(i).intValue(), Integer.MAX_VALUE); + cache1.put(i, i); + + if (i % (PUT_CNT / 5) == 0) + logTimeoutObjectsFrequency(); + } + + tx2.commit(); + } + + info("Tx2 stopped"); + + // Check that that changes committed. + for (int i = 0; i < PUT_CNT; i++) + assertEquals(cache0.get(i).intValue(), i); + } + + /** + * Fails if at least one grid contains LockTimeoutObjects. + */ + private void assertDoesNotContainLockTimeoutObjects() { + for (Ignite ignite : G.allGrids()) { + for (GridTimeoutObject object : getTimeoutObjects((IgniteEx)ignite)) { + if (object.getClass().getSimpleName().equals("LockTimeoutObject")) + fail("Grids contain LockTimeoutObjects."); + } + } + } + + /** + * Print the number of each timeout object type on each grid to the log. + */ + private void logTimeoutObjectsFrequency() { + StringBuilder sb = new StringBuilder("Timeout objects frequency ["); + + for (Ignite ignite : G.allGrids()) { + IgniteEx igniteEx = (IgniteEx)ignite; + + Map objFreqMap = new HashMap<>(); + + Set objs = getTimeoutObjects(igniteEx); + + for (GridTimeoutObject obj : objs) { + String clsName = obj.getClass().getSimpleName(); + + Integer cnt = objFreqMap.get(clsName); + + if (cnt == null) + objFreqMap.put(clsName, 1); + else + objFreqMap.put(clsName, cnt + 1); + } + + sb.append("[") + .append(igniteEx.name()).append(": size=") + .append(objs.size()).append(", "); + + for (Map.Entry entry : objFreqMap.entrySet()) { + sb.append(entry.getKey()).append("=") + .append(entry.getValue()) + .append(", "); + } + + sb.delete(sb.length() - 2, sb.length()) + .append("]; "); + } + + sb.delete(sb.length() - 2, sb.length()) + .append("]"); + + info(sb.toString() + .replaceAll("distributed.IgniteTxRollbackOnStopTest", "Grid")); + } + + /** + * @param igniteEx IgniteEx. + * @return Set of timeout objects that process on current IgniteEx. + */ + private Set getTimeoutObjects(IgniteEx igniteEx) { + GridTimeoutProcessor timeout = igniteEx.context().timeout(); + + return GridTestUtils.getFieldValue(timeout, timeout.getClass(), "timeoutObjs"); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessorSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessorSelfTest.java index eb248cfd5dced..606b10252b19a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessorSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessorSelfTest.java @@ -40,6 +40,11 @@ public class GridTimeoutProcessorSelfTest extends GridCommonAbstractTest { /** Kernal context. */ private GridTestKernalContext ctx; + /** {@inheritDoc} */ + @Override protected long getTestTimeout() { + return 60_000; + } + /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { ctx = newContext(); @@ -84,7 +89,9 @@ public void testTimeouts() throws Exception { } /** {@inheritDoc} */ - @Override public long endTime() { return endTime; } + @Override public long endTime() { + return endTime; + } /** {@inheritDoc} */ @Override public void onTimeout() { @@ -152,10 +159,14 @@ public void testTimeoutsMultithreaded() throws Exception { private final long endTime = System.currentTimeMillis() + RAND.nextInt(1000) + 500; /** {@inheritDoc} */ - @Override public IgniteUuid timeoutId() { return id; } + @Override public IgniteUuid timeoutId() { + return id; + } /** {@inheritDoc} */ - @Override public long endTime() { return endTime; } + @Override public long endTime() { + return endTime; + } /** {@inheritDoc} */ @Override public void onTimeout() { @@ -307,9 +318,8 @@ public void testTimeoutNeverCalled() throws Exception { assert timeObjs.size() == max; // Remove timeout objects so that they aren't able to times out (supposing the cycle takes less than 500 ms). - for (GridTimeoutObject obj : timeObjs) { + for (GridTimeoutObject obj : timeObjs) ctx.timeout().removeTimeoutObject(obj); - } Thread.sleep(1000); @@ -350,7 +360,9 @@ public void testTimeoutNeverCalledMultithreaded() throws Exception { } /** {@inheritDoc} */ - @Override public long endTime() { return endTime; } + @Override public long endTime() { + return endTime; + } /** {@inheritDoc} */ @Override public void onTimeout() { @@ -370,9 +382,8 @@ public void testTimeoutNeverCalledMultithreaded() throws Exception { // Remove timeout objects so that they aren't able to times out // (supposing the cycle takes less than 500 ms). - for (GridTimeoutObject obj : timeObjs) { + for (GridTimeoutObject obj : timeObjs) ctx.timeout().removeTimeoutObject(obj); - } } }, threads, "timeout-test-worker"); @@ -381,6 +392,9 @@ public void testTimeoutNeverCalledMultithreaded() throws Exception { assert callCnt.get() == 0; } + /** + * @throws Exception If test failed. + */ public void testAddRemoveInterleaving() throws Exception { final AtomicInteger callCnt = new AtomicInteger(0); @@ -430,9 +444,8 @@ public void testAddRemoveInterleaving() throws Exception { // Remove timeout objects so that they aren't able to times out // (supposing the cycle takes less than 500 ms). - for (GridTimeoutObject obj : timeObjs) { + for (GridTimeoutObject obj : timeObjs) ctx.timeout().removeTimeoutObject(obj); - } } }, 100, "timeout-test-worker"); @@ -516,10 +529,14 @@ public void testTimeoutCallOnce() throws Exception { private int cnt; /** {@inheritDoc} */ - @Override public IgniteUuid timeoutId() { return id; } + @Override public IgniteUuid timeoutId() { + return id; + } /** {@inheritDoc} */ - @Override public long endTime() { return endTime; } + @Override public long endTime() { + return endTime; + } /** {@inheritDoc} */ @Override public void onTimeout() { @@ -608,4 +625,31 @@ public void testTimeoutSameEndTime() throws Exception { assert latch.await(3000, MILLISECONDS); } + + /** + * Test that eaten {@link InterruptedException} will not hang on the closing of the grid. + * + * @throws Exception If test failed. + */ + public void testCancelingWithClearedInterruptedFlag() throws Exception { + final CountDownLatch onTimeoutCalled = new CountDownLatch(1); + + ctx.timeout().addTimeoutObject(new GridTimeoutObjectAdapter(10) { + /** {@inheritDoc} */ + @Override public void onTimeout() { + try { + onTimeoutCalled.countDown(); + + // Wait for CacheProcessor has stopped and cause InterruptedException + // which clears interrupted flag. + Thread.sleep(Long.MAX_VALUE); + } + catch (InterruptedException ignore) { + // No-op. + } + } + }); + + onTimeoutCalled.await(); + } } \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java index 0785714c61b0d..87051c3d425ce 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java @@ -40,6 +40,7 @@ import org.apache.ignite.internal.processors.cache.distributed.CacheAsyncOperationsTest; import org.apache.ignite.internal.processors.cache.distributed.GridCacheMixedModeSelfTest; import org.apache.ignite.internal.processors.cache.distributed.IgniteTxGetAfterStopTest; +import org.apache.ignite.internal.processors.cache.distributed.IgniteTxRemoveTimeoutObjectsTest; import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheDaemonNodePartitionedSelfTest; import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedOnlyP2PDisabledByteArrayValuesSelfTest; import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedOnlyP2PEnabledByteArrayValuesSelfTest; @@ -205,6 +206,8 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(CacheAsyncOperationsTest.class); + suite.addTestSuite(IgniteTxRemoveTimeoutObjectsTest.class); + return suite; } } From a76932f6f3d253a292803a0d1e954d42589bdbc6 Mon Sep 17 00:00:00 2001 From: Andrey Gura Date: Wed, 16 Aug 2017 18:00:31 +0300 Subject: [PATCH 77/90] gg-12637 Fixed unevenly partitions distribution in FairAffinityFunction --- .../affinity/fair/FairAffinityFunction.java | 29 ++++++++++++++++++- 1 file changed, 28 insertions(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunction.java b/modules/core/src/main/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunction.java index cffcf108c18df..522dac2144db1 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunction.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunction.java @@ -95,6 +95,15 @@ public class FairAffinityFunction implements AffinityFunction { /** Exclude neighbors warning. */ private transient boolean exclNeighborsWarn; + /** + * NOTE: Use {@code true} value only for new clusters or in case of cluster upgrade with downtime. + * If nodes of cluster have affinity function with different configuration it will lead to different + * assignments calculation on different nodes and, therefore, broken consistent hashing functionality. + * + * Compatibility flag. Lead to better partitions distribution if value is {@code true}. + */ + private boolean ceilIdealPartCnt = false; + /** Logger instance. */ @LoggerResource private transient IgniteLogger log; @@ -280,6 +289,24 @@ public void setExcludeNeighbors(boolean exclNeighbors) { this.exclNeighbors = exclNeighbors; } + /** + * Returns value of {@link #ceilIdealPartCnt} compatibility flag. + * + * @return Value of {@link #ceilIdealPartCnt} compatibility flag. + */ + public boolean isCeilIdealPartitionsCount() { + return ceilIdealPartCnt; + } + + /** + * Sets value of {@link #ceilIdealPartCnt} compatibility flag. + * + * @param ceilIdealPartCnt Indicates that ideal partitions count should be rounded to biggest integer value. + */ + public void setCeilIdealPartitionsCount(boolean ceilIdealPartCnt) { + this.ceilIdealPartCnt = ceilIdealPartCnt; + } + /** {@inheritDoc} */ @Override public List> assignPartitions(AffinityFunctionContext ctx) { List topSnapshot = ctx.currentTopologySnapshot(); @@ -382,7 +409,7 @@ private void assignPending(int tier, if (F.isEmpty(pending)) return; - int idealPartCnt = parts / topSnapshot.size(); + int idealPartCnt = ceilIdealPartCnt ? (int)Math.ceil((double)parts / topSnapshot.size()) : parts / topSnapshot.size(); Map tierMapping = fullMap.tierMapping(tier); From 533128821357c0909710069ea589894d99908474 Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Fri, 10 Feb 2017 16:51:37 +0300 Subject: [PATCH 78/90] GG-12647: Backport IGNITE-3196 Add support for BigDecimals with negative scale in BinaryMarshaller --- .../ignite/internal/binary/BinaryUtils.java | 12 ++--- .../internal/binary/BinaryWriterExImpl.java | 14 +++--- .../binary/BinaryMarshallerSelfTest.java | 30 +++++++++++++ modules/platforms/cpp/odbc/src/utility.cpp | 17 ++++--- .../Compute/ComputeApiTest.cs | 18 ++++---- .../Impl/Binary/BinaryUtils.cs | 44 +++++++++++-------- 6 files changed, 92 insertions(+), 43 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java index fdc54c7871711..ceadd2a59585c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java @@ -65,8 +65,8 @@ import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentHashMap8; -import static org.apache.ignite.IgniteSystemProperties.IGNITE_BINARY_MARSHALLER_USE_STRING_SERIALIZATION_VER_2; import static java.nio.charset.StandardCharsets.UTF_8; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_BINARY_MARSHALLER_USE_STRING_SERIALIZATION_VER_2; /** * Binary utils. @@ -1193,13 +1193,15 @@ public static BigDecimal doReadDecimal(BinaryInputStream in) { int scale = in.readInt(); byte[] mag = doReadByteArray(in); - BigInteger intVal = new BigInteger(mag); + boolean negative = mag[0] < 0; - if (scale < 0) { - scale &= 0x7FFFFFFF; + if (negative) + mag[0] &= 0x7F; + BigInteger intVal = new BigInteger(mag); + + if (negative) intVal = intVal.negate(); - } return new BigDecimal(intVal, scale); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java index 1de0a6533e600..12ce868354afc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java @@ -364,18 +364,20 @@ public void doWriteDecimal(@Nullable BigDecimal val) { out.unsafeWriteByte(GridBinaryMarshaller.DECIMAL); + out.unsafeWriteInt(val.scale()); + BigInteger intVal = val.unscaledValue(); - if (intVal.signum() == -1) { - intVal = intVal.negate(); + boolean negative = intVal.signum() == -1; - out.unsafeWriteInt(val.scale() | 0x80000000); - } - else - out.unsafeWriteInt(val.scale()); + if (negative) + intVal = intVal.negate(); byte[] vals = intVal.toByteArray(); + if (negative) + vals[0] |= -0x80; + out.unsafeWriteInt(vals.length); out.writeByteArray(vals); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java index 6d07c9ba42770..bfc8bd357f292 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java @@ -29,6 +29,7 @@ import java.lang.reflect.Proxy; import java.math.BigDecimal; import java.math.BigInteger; +import java.math.RoundingMode; import java.net.InetSocketAddress; import java.sql.Timestamp; import java.util.AbstractQueue; @@ -178,6 +179,35 @@ public void testDecimal() throws Exception { assertEquals((val = new BigDecimal(new BigInteger("-79228162514264337593543950336"))), marshalUnmarshal(val)); } + + /** + * @throws Exception If failed. + */ + public void testNegativeScaleDecimal() throws Exception { + BigDecimal val; + + assertEquals((val = BigDecimal.valueOf(Long.MAX_VALUE, -1)), marshalUnmarshal(val)); + assertEquals((val = BigDecimal.valueOf(Long.MIN_VALUE, -2)), marshalUnmarshal(val)); + assertEquals((val = BigDecimal.valueOf(Long.MAX_VALUE, -3)), marshalUnmarshal(val)); + assertEquals((val = BigDecimal.valueOf(Long.MIN_VALUE, -4)), marshalUnmarshal(val)); + } + + /** + * @throws Exception If failed. + */ + public void testNegativeScaleRoundingModeDecimal() throws Exception { + BigDecimal val; + + assertEquals((val = BigDecimal.ZERO.setScale(-1, RoundingMode.HALF_UP)), marshalUnmarshal(val)); + assertEquals((val = BigDecimal.valueOf(Long.MAX_VALUE).setScale(-3, RoundingMode.HALF_DOWN)), marshalUnmarshal(val)); + assertEquals((val = BigDecimal.valueOf(Long.MIN_VALUE).setScale(-5, RoundingMode.HALF_EVEN)), marshalUnmarshal(val)); + assertEquals((val = BigDecimal.valueOf(Integer.MAX_VALUE).setScale(-8, RoundingMode.UP)), marshalUnmarshal(val)); + assertEquals((val = BigDecimal.valueOf(Integer.MIN_VALUE).setScale(-10, RoundingMode.DOWN)), marshalUnmarshal(val)); + assertEquals((val = BigDecimal.valueOf(Double.MAX_VALUE).setScale(-12, RoundingMode.CEILING)), marshalUnmarshal(val)); + assertEquals((val = BigDecimal.valueOf(Double.MIN_VALUE).setScale(-15, RoundingMode.FLOOR)), marshalUnmarshal(val)); + } + + /** * @throws Exception If failed. */ diff --git a/modules/platforms/cpp/odbc/src/utility.cpp b/modules/platforms/cpp/odbc/src/utility.cpp index 19be799a10aae..c331332a34756 100644 --- a/modules/platforms/cpp/odbc/src/utility.cpp +++ b/modules/platforms/cpp/odbc/src/utility.cpp @@ -93,8 +93,14 @@ namespace ignite impl::binary::BinaryUtils::ReadInt8Array(reader.GetStream(), mag.data(), static_cast(mag.size())); - int32_t sign = (scale & 0x80000000) ? -1 : 1; - scale = scale & 0x7FFFFFFF; + int32_t sign = 1; + + if (mag[0] < 0) + { + mag[0] &= 0x7F; + + sign = -1; + } common::Decimal res(mag.data(), static_cast(mag.size()), scale, sign); @@ -107,14 +113,15 @@ namespace ignite const common::BigInteger &unscaled = decimal.GetUnscaledValue(); - int32_t signFlag = unscaled.GetSign() == -1 ? 0x80000000 : 0; - - writer.WriteInt32(decimal.GetScale() | signFlag); + writer.WriteInt32(decimal.GetScale()); common::FixedSizeArray magnitude; unscaled.MagnitudeToBytes(magnitude); + if (unscaled.GetSign() == -1) + magnitude[0] |= -0x80; + writer.WriteInt32(magnitude.GetSize()); impl::binary::BinaryUtils::WriteInt8Array(writer.GetStream(), magnitude.GetData(), magnitude.GetSize()); diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs index 1e0287fe56e8f..15d8dba7895be 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs @@ -818,16 +818,16 @@ public void TestEchoDecimal() Assert.AreEqual(val = decimal.Parse("-11,12"), _grid1.GetCompute().ExecuteJavaTask(DecimalTask, new object[] { val, val.ToString() })); // Test echo with overflow. - try - { - _grid1.GetCompute().ExecuteJavaTask(DecimalTask, new object[] { null, decimal.MaxValue.ToString() + 1 }); + var ex = Assert.Throws(() => _grid1.GetCompute() + .ExecuteJavaTask(DecimalTask, new object[] {null, decimal.MaxValue.ToString() + 1})); - Assert.Fail(); - } - catch (IgniteException) - { - // No-op. - } + Assert.AreEqual("Decimal magnitude overflow (must be less than 96 bits): 104", ex.Message); + + // Negative scale. 1E+1 parses to "1 scale -1" on Java side. + ex = Assert.Throws(() => _grid1.GetCompute() + .ExecuteJavaTask(DecimalTask, new object[] {null, "1E+1"})); + + Assert.AreEqual("Decimal value scale overflow (must be between 0 and 28): -1", ex.Message); } /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryUtils.cs index cc5d8a109ce86..5ce77d0ee4347 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryUtils.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryUtils.cs @@ -893,7 +893,9 @@ public static void WriteDecimal(decimal val, IBinaryStream stream) // Write scale and negative flag. int scale = (vals[3] & 0x00FF0000) >> 16; - stream.WriteInt(((vals[3] & 0x80000000) == 0x80000000) ? (int)((uint)scale | 0x80000000) : scale); + stream.WriteInt(scale); + + Boolean neg = vals[3] < 0; if (idx == -1) { @@ -923,13 +925,15 @@ public static void WriteDecimal(decimal val, IBinaryStream stream) if ((part24 & 0x80) == 0x80) { stream.WriteInt(len + 1); + + stream.WriteByte((byte)(neg ? -0x80 : ByteZero)); - stream.WriteByte(ByteZero); + neg = false; } else stream.WriteInt(len); - stream.WriteByte((byte)part24); + stream.WriteByte((byte)(neg ? ((sbyte)part24 | -0x80) : part24)); stream.WriteByte((byte)part16); stream.WriteByte((byte)part8); stream.WriteByte((byte)part0); @@ -940,12 +944,14 @@ public static void WriteDecimal(decimal val, IBinaryStream stream) { stream.WriteInt(len); - stream.WriteByte(ByteZero); + stream.WriteByte((byte)(neg ? -0x80 : ByteZero)); + + neg = false; } else stream.WriteInt(len - 1); - - stream.WriteByte((byte)part16); + + stream.WriteByte((byte)(neg ? ((sbyte)part16 | -0x80) : part16)); stream.WriteByte((byte)part8); stream.WriteByte((byte)part0); } @@ -955,12 +961,14 @@ public static void WriteDecimal(decimal val, IBinaryStream stream) { stream.WriteInt(len - 1); - stream.WriteByte(ByteZero); + stream.WriteByte((byte)(neg ? -0x80 : ByteZero)); + + neg = false; } else stream.WriteInt(len - 2); - - stream.WriteByte((byte)part8); + + stream.WriteByte((byte)(neg ? ((sbyte)part8 | -0x80) : part8)); stream.WriteByte((byte)part0); } else @@ -969,12 +977,14 @@ public static void WriteDecimal(decimal val, IBinaryStream stream) { stream.WriteInt(len - 2); - stream.WriteByte(ByteZero); + stream.WriteByte((byte)(neg ? -0x80 : ByteZero)); + + neg = false; } else stream.WriteInt(len - 3); - stream.WriteByte((byte)part0); + stream.WriteByte((byte)(neg ? ((sbyte)part0 | -0x80) : part0)); } } else @@ -997,18 +1007,16 @@ public static void WriteDecimal(decimal val, IBinaryStream stream) { int scale = stream.ReadInt(); - bool neg; + bool neg = false; + + byte[] mag = ReadByteArray(stream); - if (scale < 0) + if ((sbyte)mag[0] < 0) { - scale = scale & 0x7FFFFFFF; + mag[0] &= 0x7F; neg = true; } - else - neg = false; - - byte[] mag = ReadByteArray(stream); if (scale < 0 || scale > 28) throw new BinaryObjectException("Decimal value scale overflow (must be between 0 and 28): " + scale); From 75febb824b3e261ec1c15224a024e1d4160f2f42 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Tue, 13 Jun 2017 19:41:55 +0300 Subject: [PATCH 79/90] ignite-2.1.1 Extract Ignite updates checker to separate class. Fixed GridUpdateNotifier test. --- .../cluster/GridUpdateNotifier.java | 56 ++++++++-------- .../cluster/HttpIgniteUpdatesChecker.java | 65 +++++++++++++++++++ .../cluster/GridUpdateNotifierSelfTest.java | 10 ++- 3 files changed, 100 insertions(+), 31 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/HttpIgniteUpdatesChecker.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridUpdateNotifier.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridUpdateNotifier.java index 592fdd10bde73..83588c26227ee 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridUpdateNotifier.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridUpdateNotifier.java @@ -64,8 +64,8 @@ class GridUpdateNotifier { /** Sleep milliseconds time for worker thread. */ private static final int WORKER_THREAD_SLEEP_TIME = 5000; - /** Url for request version. */ - private final static String UPDATE_NOTIFIER_URL = "https://ignite.run/update_status_ignite-plain-text.php"; + /** Default url for request Ignite updates. */ + private final static String DEFAULT_IGNITE_UPDATES_URL = "https://ignite.run/update_status_ignite-plain-text.php"; /** Grid version. */ private final String ver; @@ -103,22 +103,27 @@ class GridUpdateNotifier { /** Worker thread to process http request. */ private final Thread workerThread; + /** Http client for getting Ignite updates */ + private final HttpIgniteUpdatesChecker updatesChecker; + /** * Creates new notifier with default values. * - * @param gridName gridName + * @param igniteInstanceName Ignite instance name. * @param ver Compound Ignite version. * @param gw Kernal gateway. * @param pluginProviders Kernal gateway. * @param reportOnlyNew Whether or not to report only new version. + * @param updatesChecker Service for getting Ignite updates * @throws IgniteCheckedException If failed. */ - GridUpdateNotifier(String gridName, String ver, GridKernalGateway gw, Collection pluginProviders, - boolean reportOnlyNew) throws IgniteCheckedException { + GridUpdateNotifier(String igniteInstanceName, String ver, GridKernalGateway gw, Collection pluginProviders, + boolean reportOnlyNew, HttpIgniteUpdatesChecker updatesChecker) throws IgniteCheckedException { try { this.ver = ver; - this.gridName = gridName == null ? "null" : gridName; + this.gridName = igniteInstanceName == null ? "null" : igniteInstanceName; this.gw = gw; + this.updatesChecker = updatesChecker; SB pluginsBuilder = new SB(); @@ -159,6 +164,14 @@ class GridUpdateNotifier { } } + /** + * Creates new notifier with default Ignite updates URL + */ + GridUpdateNotifier(String igniteInstanceName, String ver, GridKernalGateway gw, Collection pluginProviders, + boolean reportOnlyNew) throws IgniteCheckedException { + this(igniteInstanceName, ver, gw, pluginProviders, reportOnlyNew, new HttpIgniteUpdatesChecker(DEFAULT_IGNITE_UPDATES_URL, CHARSET)); + } + /** * Gets system properties. * @@ -313,34 +326,17 @@ private class UpdateChecker extends GridWorker { (!F.isEmpty(vmProps) ? "&vmProps=" + encode(vmProps, CHARSET) : "") + pluginsVers; - URLConnection conn = new URL(UPDATE_NOTIFIER_URL).openConnection(); - if (!isCancelled()) { - conn.setDoOutput(true); - conn.setRequestProperty("Accept-Charset", CHARSET); - conn.setRequestProperty("Content-Type", "application/x-www-form-urlencoded;charset=" + CHARSET); - - conn.setConnectTimeout(3000); - conn.setReadTimeout(3000); - try { - try (OutputStream os = conn.getOutputStream()) { - os.write(postParams.getBytes(CHARSET)); - } - - try (InputStream in = conn.getInputStream()) { - if (in == null) - return; - - BufferedReader reader = new BufferedReader(new InputStreamReader(in, CHARSET)); + String updatesResponse = updatesChecker.getUpdates(postParams); - for (String line; (line = reader.readLine()) != null; ) { - if (line.contains("version")) - latestVer = obtainVersionFrom(line); - else if (line.contains("downloadUrl")) - downloadUrl = obtainDownloadUrlFrom(line); - } + String[] lines = updatesResponse.split("\n"); + for (String line : lines) { + if (line.contains("version")) + latestVer = obtainVersionFrom(line); + else if (line.contains("downloadUrl")) + downloadUrl = obtainDownloadUrlFrom(line); } } catch (IOException e) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/HttpIgniteUpdatesChecker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/HttpIgniteUpdatesChecker.java new file mode 100644 index 0000000000000..c052c0997fe2a --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/HttpIgniteUpdatesChecker.java @@ -0,0 +1,65 @@ +package org.apache.ignite.internal.processors.cluster; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.OutputStream; +import java.net.URL; +import java.net.URLConnection; + +/** + * This class is responsible for getting Ignite updates information via HTTP + */ +public class HttpIgniteUpdatesChecker { + /** Url for request updates. */ + private final String url; + + /** Charset for encoding requests/responses */ + private final String charset; + + /** + * Creates new HTTP Ignite updates checker with following parameters + * @param url URL for getting Ignite updates information + * @param charset Charset for encoding + */ + HttpIgniteUpdatesChecker(String url, String charset) { + this.url = url; + this.charset = charset; + } + + /** + * Gets information about Ignite updates via HTTP + * @param updateRequest HTTP Request parameters + * @return Information about Ignite updates separated by line endings + * @throws IOException If HTTP request was failed + */ + public String getUpdates(String updateRequest) throws IOException { + URLConnection conn = new URL(url).openConnection(); + conn.setDoOutput(true); + conn.setRequestProperty("Accept-Charset", charset); + conn.setRequestProperty("Content-Type", "application/x-www-form-urlencoded;charset=" + charset); + + conn.setConnectTimeout(3000); + conn.setReadTimeout(3000); + + try (OutputStream os = conn.getOutputStream()) { + os.write(updateRequest.getBytes(charset)); + } + + try (InputStream in = conn.getInputStream()) { + if (in == null) + return null; + + BufferedReader reader = new BufferedReader(new InputStreamReader(in, charset)); + + StringBuilder response = new StringBuilder(); + + for (String line; (line = reader.readLine()) != null; ) { + response.append(line).append('\n'); + } + + return response.toString(); + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cluster/GridUpdateNotifierSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cluster/GridUpdateNotifierSelfTest.java index 21b91b6b3c53e..1a20f261550ba 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cluster/GridUpdateNotifierSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cluster/GridUpdateNotifierSelfTest.java @@ -29,6 +29,8 @@ import org.apache.ignite.plugin.PluginProvider; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.apache.ignite.testframework.junits.common.GridCommonTest; +import org.mockito.Matchers; +import org.mockito.Mockito; /** * Update notifier test. @@ -73,8 +75,14 @@ public class GridUpdateNotifierSelfTest extends GridCommonAbstractTest { public void testNotifier() throws Exception { String nodeVer = IgniteProperties.get("ignite.version"); + HttpIgniteUpdatesChecker updatesCheckerMock = Mockito.mock(HttpIgniteUpdatesChecker.class); + + // Return current node version and some other info + Mockito.when(updatesCheckerMock.getUpdates(Matchers.anyString())) + .thenReturn("meta=meta" + "\n" + "version=" + nodeVer + "\n" + "downloadUrl=url"); + GridUpdateNotifier ntf = new GridUpdateNotifier(null, nodeVer, - TEST_GATEWAY, Collections.emptyList(), false); + TEST_GATEWAY, Collections.emptyList(), false, updatesCheckerMock); ntf.checkForNewVersion(log); From f813fc036802349d81d6c313c4c1846ca5ed8fd7 Mon Sep 17 00:00:00 2001 From: Evgeny Stanilovskiy Date: Wed, 1 Feb 2017 14:21:49 +0300 Subject: [PATCH 80/90] ignite-4557 Fixed wrong affinity manager call. --- .../processors/cache/GridCacheAdapter.java | 10 ++-- .../cache/GridCacheAffinityManager.java | 60 +++++++------------ .../processors/cache/GridCacheContext.java | 17 ------ .../cache/GridCacheEvictionManager.java | 6 +- .../processors/cache/GridCacheUtils.java | 20 ------- .../cache/affinity/GridCacheAffinityImpl.java | 16 ++--- .../CacheDataStructuresManager.java | 2 +- .../distributed/dht/GridDhtCacheAdapter.java | 2 +- .../distributed/dht/GridDhtCacheEntry.java | 2 +- .../dht/GridDhtLocalPartition.java | 4 +- .../dht/GridDhtPartitionTopologyImpl.java | 4 +- .../distributed/dht/GridDhtTxRemote.java | 2 +- .../dht/GridPartitionedGetFuture.java | 2 +- .../dht/GridPartitionedSingleGetFuture.java | 2 +- .../dht/atomic/GridDhtAtomicCache.java | 13 ++-- .../GridNearAtomicSingleUpdateFuture.java | 2 +- .../atomic/GridNearAtomicUpdateFuture.java | 4 +- .../dht/colocated/GridDhtColocatedCache.java | 6 +- .../colocated/GridDhtColocatedLockFuture.java | 4 +- .../preloader/GridDhtPartitionDemander.java | 8 +-- .../preloader/GridDhtPartitionSupplier.java | 12 ++-- .../dht/preloader/GridDhtPreloader.java | 4 +- .../distributed/near/GridNearAtomicCache.java | 2 +- .../distributed/near/GridNearCacheEntry.java | 6 +- .../distributed/near/GridNearGetFuture.java | 4 +- .../distributed/near/GridNearLockFuture.java | 2 +- ...OptimisticSerializableTxPrepareFuture.java | 2 +- .../GridNearOptimisticTxPrepareFuture.java | 2 +- .../GridNearPessimisticTxPrepareFuture.java | 2 +- .../near/GridNearTransactionalCache.java | 6 +- .../near/GridNearTxFinishFuture.java | 2 +- .../cache/query/GridCacheQueryManager.java | 9 +-- .../CacheContinuousQueryHandler.java | 1 - .../CacheContinuousQueryManager.java | 2 +- .../cache/transactions/IgniteTxAdapter.java | 4 +- .../transactions/IgniteTxLocalAdapter.java | 4 +- .../transactions/TxDeadlockDetection.java | 2 +- .../datastreamer/DataStreamerImpl.java | 2 +- .../datastructures/GridCacheSetImpl.java | 2 +- .../datastructures/GridSetQueryPredicate.java | 2 +- .../processors/job/GridJobProcessor.java | 2 +- .../cache/CacheAffinityCallSelfTest.java | 4 +- .../GridCacheAbstractFullApiSelfTest.java | 6 +- ...gniteCacheConfigVariationsFullApiTest.java | 6 +- .../IgniteCachePeekModesAbstractTest.java | 8 +-- ...tQueueFailoverDataConsistencySelfTest.java | 2 +- ...teCacheClientNodeChangingTopologyTest.java | 8 +-- .../TxOptimisticDeadlockDetectionTest.java | 2 +- .../TxPessimisticDeadlockDetectionTest.java | 2 +- .../processors/query/h2/IgniteH2Indexing.java | 2 +- .../query/h2/opt/GridH2IndexBase.java | 2 +- 51 files changed, 126 insertions(+), 176 deletions(-) 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 7aa48199bb28a..9b1e0cc7721c1 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 @@ -778,7 +778,7 @@ public String toString() { boolean nearKey; if (!(modes.near && modes.primary && modes.backup)) { - boolean keyPrimary = ctx.affinity().primary(ctx.localNode(), part, topVer); + boolean keyPrimary = ctx.affinity().primaryByPartition(ctx.localNode(), part, topVer); if (keyPrimary) { if (!modes.primary) @@ -787,7 +787,7 @@ public String toString() { nearKey = false; } else { - boolean keyBackup = ctx.affinity().belongs(ctx.localNode(), part, topVer); + boolean keyBackup = ctx.affinity().partitionBelongs(ctx.localNode(), part, topVer); if (keyBackup) { if (!modes.backup) @@ -808,7 +808,7 @@ public String toString() { } } else { - nearKey = !ctx.affinity().belongs(ctx.localNode(), part, topVer); + nearKey = !ctx.affinity().partitionBelongs(ctx.localNode(), part, topVer); if (nearKey) { // Swap and offheap are disabled for near cache. @@ -3813,8 +3813,8 @@ IgniteInternalFuture globalLoadCacheAsync(@Nullable IgniteBiPredicate p /** {@inheritDoc} */ @Override public boolean apply(ClusterNode clusterNode) { return clusterNode.version().compareTo(PartitionSizeLongTask.SINCE_VER) >= 0 && - ((modes.primary && aff.primary(clusterNode, part, topVer)) || - (modes.backup && aff.backup(clusterNode, part, topVer))); + ((modes.primary && aff.primaryByPartition(clusterNode, part, topVer)) || + (modes.backup && aff.backupByPartition(clusterNode, part, topVer))); } }).nodes(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java index 8b7be1b3288e2..d85e76e2bbd82 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java @@ -238,8 +238,8 @@ public Object affinityKey(Object key) { * @param topVer Topology version. * @return Affinity nodes. */ - public List nodes(Object key, AffinityTopologyVersion topVer) { - return nodes(partition(key), topVer); + public List nodesByKey(Object key, AffinityTopologyVersion topVer) { + return nodesByPartition(partition(key), topVer); } /** @@ -247,7 +247,7 @@ public List nodes(Object key, AffinityTopologyVersion topVer) { * @param topVer Topology version. * @return Affinity nodes. */ - public List nodes(int part, AffinityTopologyVersion topVer) { + public List nodesByPartition(int part, AffinityTopologyVersion topVer) { if (cctx.isLocal()) topVer = LOC_CACHE_TOP_VER; @@ -282,8 +282,8 @@ public AffinityAssignment assignment(AffinityTopologyVersion topVer) { * @param topVer Topology version. * @return Primary node for given key. */ - @Nullable public ClusterNode primary(Object key, AffinityTopologyVersion topVer) { - return primary(partition(key), topVer); + @Nullable public ClusterNode primaryByKey(Object key, AffinityTopologyVersion topVer) { + return primaryByPartition(partition(key), topVer); } /** @@ -291,8 +291,8 @@ public AffinityAssignment assignment(AffinityTopologyVersion topVer) { * @param topVer Topology version. * @return Primary node for given key. */ - @Nullable public ClusterNode primary(int part, AffinityTopologyVersion topVer) { - List nodes = nodes(part, topVer); + @Nullable public ClusterNode primaryByPartition(int part, AffinityTopologyVersion topVer) { + List nodes = nodesByPartition(part, topVer); if (nodes.isEmpty()) return null; @@ -306,8 +306,8 @@ public AffinityAssignment assignment(AffinityTopologyVersion topVer) { * @param topVer Topology version. * @return {@code True} if checked node is primary for given key. */ - public boolean primary(ClusterNode n, Object key, AffinityTopologyVersion topVer) { - return F.eq(primary(key, topVer), n); + public boolean primaryByKey(ClusterNode n, Object key, AffinityTopologyVersion topVer) { + return F.eq(primaryByKey(key, topVer), n); } /** @@ -316,8 +316,8 @@ public boolean primary(ClusterNode n, Object key, AffinityTopologyVersion topVer * @param topVer Topology version. * @return {@code True} if checked node is primary for given partition. */ - public boolean primary(ClusterNode n, int part, AffinityTopologyVersion topVer) { - return F.eq(primary(part, topVer), n); + public boolean primaryByPartition(ClusterNode n, int part, AffinityTopologyVersion topVer) { + return F.eq(primaryByPartition(part, topVer), n); } /** @@ -325,8 +325,8 @@ public boolean primary(ClusterNode n, int part, AffinityTopologyVersion topVer) * @param topVer Topology version. * @return Backup nodes. */ - public Collection backups(Object key, AffinityTopologyVersion topVer) { - return backups(partition(key), topVer); + public Collection backupsByKey(Object key, AffinityTopologyVersion topVer) { + return backupsByPartition(partition(key), topVer); } /** @@ -334,8 +334,8 @@ public Collection backups(Object key, AffinityTopologyVersion topVe * @param topVer Topology version. * @return Backup nodes. */ - public Collection backups(int part, AffinityTopologyVersion topVer) { - List nodes = nodes(part, topVer); + private Collection backupsByPartition(int part, AffinityTopologyVersion topVer) { + List nodes = nodesByPartition(part, topVer); assert !F.isEmpty(nodes); @@ -351,35 +351,21 @@ public Collection backups(int part, AffinityTopologyVersion topVer) * @param topVer Topology version. * @return {@code True} if checked node is a backup node for given partition. */ - public boolean backup(ClusterNode n, int part, AffinityTopologyVersion topVer) { - List nodes = nodes(part, topVer); + public boolean backupByPartition(ClusterNode n, int part, AffinityTopologyVersion topVer) { + List nodes = nodesByPartition(part, topVer); assert !F.isEmpty(nodes); return nodes.indexOf(n) > 0; } - /** - * @param keys keys. - * @param topVer Topology version. - * @return Nodes for the keys. - */ - public Collection remoteNodes(Iterable keys, AffinityTopologyVersion topVer) { - Collection> colcol = new GridLeanSet<>(); - - for (Object key : keys) - colcol.add(nodes(key, topVer)); - - return F.view(F.flatCollections(colcol), F.remoteNodes(cctx.localNodeId())); - } - /** * @param key Key to check. * @param topVer Topology version. * @return {@code true} if given key belongs to local node. */ - public boolean localNode(Object key, AffinityTopologyVersion topVer) { - return localNode(partition(key), topVer); + public boolean keyLocalNode(Object key, AffinityTopologyVersion topVer) { + return partitionLocalNode(partition(key), topVer); } /** @@ -387,10 +373,10 @@ public boolean localNode(Object key, AffinityTopologyVersion topVer) { * @param topVer Topology version. * @return {@code true} if given partition belongs to local node. */ - public boolean localNode(int part, AffinityTopologyVersion topVer) { + public boolean partitionLocalNode(int part, AffinityTopologyVersion topVer) { assert part >= 0 : "Invalid partition: " + part; - return nodes(part, topVer).contains(cctx.localNode()); + return nodesByPartition(part, topVer).contains(cctx.localNode()); } /** @@ -399,11 +385,11 @@ public boolean localNode(int part, AffinityTopologyVersion topVer) { * @param topVer Topology version. * @return {@code true} if given partition belongs to specified node. */ - public boolean belongs(ClusterNode node, int part, AffinityTopologyVersion topVer) { + public boolean partitionBelongs(ClusterNode node, int part, AffinityTopologyVersion topVer) { assert node != null; assert part >= 0 : "Invalid partition: " + part; - return nodes(part, topVer).contains(node); + return nodesByPartition(part, topVer).contains(node); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java index 6322f9f03550a..3b44b5096f0b0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java @@ -1589,23 +1589,6 @@ private void map(GridDhtCacheEntry entry, Iterable nodes, } } - /** - * Checks if at least one of the given keys belongs to one of the given partitions. - * - * @param keys Collection of keys to check. - * @param movingParts Collection of partitions to check against. - * @return {@code True} if there exist a key in collection {@code keys} that belongs - * to one of partitions in {@code movingParts} - */ - public boolean hasKey(Iterable keys, Collection movingParts) { - for (K key : keys) { - if (movingParts.contains(affinity().partition(key))) - return true; - } - - return false; - } - /** * Check whether conflict resolution is required. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java index 134e743d4fd6e..f8722d6a4f338 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java @@ -808,7 +808,7 @@ public void touch(GridCacheEntryEx e, AffinityTopologyVersion topVer) { return; // Don't track non-primary entries if evicts are synchronized. - if (!cctx.isNear() && evictSync && !cctx.affinity().primary(cctx.localNode(), e.partition(), topVer)) + if (!cctx.isNear() && evictSync && !cctx.affinity().primaryByPartition(cctx.localNode(), e.partition(), topVer)) return; if (!busyLock.enterBusy()) @@ -910,7 +910,7 @@ public boolean evict(@Nullable GridCacheEntryEx entry, @Nullable GridCacheVersio if (evictSyncAgr) { assert !cctx.isNear(); // Make sure cache is not NEAR. - if (cctx.affinity().backups( + if (cctx.affinity().backupsByKey( entry.key(), cctx.topology().topologyVersion()).contains(cctx.localNode()) && evictSync) @@ -1498,7 +1498,7 @@ void addEvent(DiscoveryEvent evt) { if (!evts.isEmpty()) break; - if (!cctx.affinity().primary(loc, it.next(), topVer)) + if (!cctx.affinity().primaryByPartition(loc, it.next(), topVer)) it.remove(); } 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 8ee77e36c3a77..5c21d7e4a8e98 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 @@ -916,26 +916,6 @@ public static void unwindEvicts(GridCacheSharedContext ctx) { unwindEvicts(cacheCtx); } - /** - * Gets primary node on which given key is cached. - * - * @param ctx Cache. - * @param key Key to find primary node for. - * @return Primary node for the key. - */ - @SuppressWarnings( {"unchecked"}) - @Nullable public static ClusterNode primaryNode(GridCacheContext ctx, Object key) { - assert ctx != null; - assert key != null; - - CacheConfiguration cfg = ctx.cache().configuration(); - - if (cfg.getCacheMode() != PARTITIONED) - return ctx.localNode(); - - return ctx.affinity().primary(key, ctx.affinity().affinityTopologyVersion()); - } - /** * @param asc {@code True} for ascending. * @return Descending order comparator. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/affinity/GridCacheAffinityImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/affinity/GridCacheAffinityImpl.java index 9e85bad610432..11361a27d384a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/affinity/GridCacheAffinityImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/affinity/GridCacheAffinityImpl.java @@ -82,21 +82,21 @@ public GridCacheAffinityImpl(GridCacheContext cctx) { @Override public boolean isPrimary(ClusterNode n, K key) { A.notNull(n, "n", key, "key"); - return cctx.affinity().primary(n, key, topologyVersion()); + return cctx.affinity().primaryByKey(n, key, topologyVersion()); } /** {@inheritDoc} */ @Override public boolean isBackup(ClusterNode n, K key) { A.notNull(n, "n", key, "key"); - return cctx.affinity().backups(key, topologyVersion()).contains(n); + return cctx.affinity().backupsByKey(key, topologyVersion()).contains(n); } /** {@inheritDoc} */ @Override public boolean isPrimaryOrBackup(ClusterNode n, K key) { A.notNull(n, "n", key, "key"); - return cctx.affinity().belongs(n, cctx.affinity().partition(key), topologyVersion()); + return cctx.affinity().partitionBelongs(n, cctx.affinity().partition(key), topologyVersion()); } /** {@inheritDoc} */ @@ -126,7 +126,7 @@ public GridCacheAffinityImpl(GridCacheContext cctx) { AffinityTopologyVersion topVer = topologyVersion(); for (int partsCnt = partitions(), part = 0; part < partsCnt; part++) { - for (ClusterNode affNode : cctx.affinity().nodes(part, topVer)) { + for (ClusterNode affNode : cctx.affinity().nodesByPartition(part, topVer)) { if (n.id().equals(affNode.id())) { parts.add(part); @@ -142,7 +142,7 @@ public GridCacheAffinityImpl(GridCacheContext cctx) { @Override public ClusterNode mapPartitionToNode(int part) { A.ensure(part >= 0 && part < partitions(), "part >= 0 && part < total partitions"); - return F.first(cctx.affinity().nodes(part, topologyVersion())); + return F.first(cctx.affinity().nodesByPartition(part, topologyVersion())); } /** {@inheritDoc} */ @@ -204,7 +204,7 @@ public GridCacheAffinityImpl(GridCacheContext cctx) { Map> res = new HashMap<>(nodesCnt, 1.0f); for (K key : keys) { - ClusterNode primary = cctx.affinity().primary(key, topVer); + ClusterNode primary = cctx.affinity().primaryByKey(key, topVer); if (primary == null) throw new IgniteException("Failed to get primary node [topVer=" + topVer + ", key=" + key + ']'); @@ -227,14 +227,14 @@ public GridCacheAffinityImpl(GridCacheContext cctx) { @Override public Collection mapKeyToPrimaryAndBackups(K key) { A.notNull(key, "key"); - return cctx.affinity().nodes(partition(key), topologyVersion()); + return cctx.affinity().nodesByPartition(partition(key), topologyVersion()); } /** {@inheritDoc} */ @Override public Collection mapPartitionToPrimaryAndBackups(int part) { A.ensure(part >= 0 && part < partitions(), "part >= 0 && part < total partitions"); - return cctx.affinity().nodes(part, topologyVersion()); + return cctx.affinity().nodesByPartition(part, topologyVersion()); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java index 366a4a920b9ee..2b3080981ec36 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java @@ -455,7 +455,7 @@ private void removeSetData(IgniteUuid setId, AffinityTopologyVersion topVer) thr Collection keys = new ArrayList<>(BATCH_SIZE); for (SetItemKey key : set) { - if (!loc && !aff.primary(cctx.localNode(), key, topVer)) + if (!loc && !aff.primaryByKey(cctx.localNode(), key, topVer)) continue; keys.add(key); 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 be7fa5525deba..2fa934b79db71 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 @@ -933,7 +933,7 @@ public void sendTtlUpdateRequest(@Nullable final IgniteCacheExpiryPolicy expiryP AffinityTopologyVersion topVer = ctx.shared().exchange().readyAffinityVersion(); for (Map.Entry e : entries.entrySet()) { - List nodes = ctx.affinity().nodes(e.getKey(), topVer); + List nodes = ctx.affinity().nodesByKey(e.getKey(), topVer); for (int i = 0; i < nodes.size(); i++) { ClusterNode node = nodes.get(i); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java index cf4085ba02335..39571ff4720e6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java @@ -402,7 +402,7 @@ public Collection readers() throws GridCacheEntryRemovedException { } // If remote node is (primary?) or back up, don't add it as a reader. - if (cctx.affinity().belongs(node, partition(), topVer)) { + if (cctx.affinity().partitionBelongs(node, partition(), topVer)) { if (log.isDebugEnabled()) log.debug("Ignoring near reader because remote node is affinity node [locNodeId=" + cctx.localNodeId() + ", rmtNodeId=" + nodeId + ", key=" + key + ']'); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java index 668a1cdc006d9..b21463843bbcf 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java @@ -614,7 +614,7 @@ void onUnlock() { * @return {@code True} if local node is primary for this partition. */ public boolean primary(AffinityTopologyVersion topVer) { - return cctx.affinity().primary(cctx.localNode(), id, topVer); + return cctx.affinity().primaryByPartition(cctx.localNode(), id, topVer); } /** @@ -622,7 +622,7 @@ public boolean primary(AffinityTopologyVersion topVer) { * @return {@code True} if local node is backup for this partition. */ public boolean backup(AffinityTopologyVersion topVer) { - return cctx.affinity().backup(cctx.localNode(), id, topVer); + return cctx.affinity().backupByPartition(cctx.localNode(), id, topVer); } /** 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 1ad0ff0538fa5..23485f985767d 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 @@ -568,7 +568,7 @@ else if (!node2part.nodeId().equals(loc.id())) { for (int p = 0; p < num; p++) { GridDhtLocalPartition locPart = localPartition(p, topVer, false, false); - if (cctx.affinity().localNode(p, topVer)) { + if (cctx.affinity().partitionLocalNode(p, topVer)) { // This partition will be created during next topology event, // which obviously has not happened at this point. if (locPart == null) { @@ -691,7 +691,7 @@ private GridDhtLocalPartition localPartition(int p, try { loc = locParts.get(p); - boolean belongs = cctx.affinity().localNode(p, topVer); + boolean belongs = cctx.affinity().partitionLocalNode(p, topVer); if (loc != null && loc.state() == EVICTED) { locParts.set(p, loc = null); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxRemote.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxRemote.java index 8942ef9d9178c..399736e8f4278 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxRemote.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxRemote.java @@ -286,7 +286,7 @@ IgniteUuid remoteFutureId() { return true; // Check if we are on the backup node. - return !cacheCtx.affinity().backups(key, topVer).contains(cctx.localNode()); + return !cacheCtx.affinity().backupsByKey(key, topVer).contains(cctx.localNode()); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java index c41711c245028..519239aecb532 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java @@ -380,7 +380,7 @@ private boolean map( ) { int part = cctx.affinity().partition(key); - List affNodes = cctx.affinity().nodes(part, topVer); + List affNodes = cctx.affinity().nodesByPartition(part, topVer); if (affNodes.isEmpty()) { onDone(serverNotFoundError(topVer)); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java index 2b5624b489b6f..a3f6b72376e8f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java @@ -325,7 +325,7 @@ private void map(AffinityTopologyVersion topVer) { @Nullable private ClusterNode mapKeyToNode(AffinityTopologyVersion topVer) { int part = cctx.affinity().partition(key); - List affNodes = cctx.affinity().nodes(part, topVer); + List affNodes = cctx.affinity().nodesByPartition(part, topVer); if (affNodes.isEmpty()) { onDone(serverNotFoundError(topVer)); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java index dc6d3dd8271dd..16e51ee0c2815 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java @@ -2469,7 +2469,7 @@ private UpdateSingleResult updateSingle( assert !(newConflictVer instanceof GridCacheVersionEx) : newConflictVer; - boolean primary = !req.fastMap() || ctx.affinity().primary(ctx.localNode(), entry.partition(), + boolean primary = !req.fastMap() || ctx.affinity().primaryByPartition(ctx.localNode(), entry.partition(), req.topologyVersion()); Object writeVal = op == TRANSFORM ? req.entryProcessor(i) : req.writeValue(i); @@ -2559,7 +2559,7 @@ else if (conflictCtx.isMerge()) if (hasNear) { if (primary && updRes.sendToDht()) { - if (!ctx.affinity().belongs(node, entry.partition(), topVer)) { + if (!ctx.affinity().partitionBelongs(node, entry.partition(), topVer)) { // If put the same value as in request then do not need to send it back. if (op == TRANSFORM || writeVal != updRes.newValue()) { res.addNearValue(i, @@ -2690,7 +2690,7 @@ else if (F.contains(readers, node.id())) // Reader became primary or backup. Map storeMap = req.fastMap() ? F.view(putMap, new P1() { @Override public boolean apply(CacheObject key) { - return ctx.affinity().primary(ctx.localNode(), key, req.topologyVersion()); + return ctx.affinity().primaryByKey(ctx.localNode(), key, req.topologyVersion()); } }) : putMap; @@ -2713,7 +2713,7 @@ else if (F.contains(readers, node.id())) // Reader became primary or backup. Collection storeKeys = req.fastMap() ? F.view(rmvKeys, new P1() { @Override public boolean apply(Object key) { - return ctx.affinity().primary(ctx.localNode(), key, req.topologyVersion()); + return ctx.affinity().primaryByKey(ctx.localNode(), key, req.topologyVersion()); } }) : rmvKeys; @@ -2752,7 +2752,8 @@ else if (F.contains(readers, node.id())) // Reader became primary or backup. assert writeVal != null || op == DELETE : "null write value found."; - boolean primary = !req.fastMap() || ctx.affinity().primary(ctx.localNode(), entry.key(), + boolean primary = !req.fastMap() || ctx.affinity().primaryByPartition(ctx.localNode(), + entry.partition(), req.topologyVersion()); Collection readers = null; @@ -2848,7 +2849,7 @@ else if (F.contains(readers, node.id())) // Reader became primary or backup. if (hasNear) { if (primary) { - if (!ctx.affinity().belongs(node, entry.partition(), topVer)) { + if (!ctx.affinity().partitionBelongs(node, entry.partition(), topVer)) { int idx = firstEntryIdx + i; if (req.operation() == TRANSFORM) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java index 7376affca4455..891a20c2884a7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java @@ -543,7 +543,7 @@ private GridNearAtomicAbstractUpdateRequest mapSingleUpdate(AffinityTopologyVers else val = EntryProcessorResourceInjectorProxy.wrap(cctx.kernalContext(), (EntryProcessor)val); - ClusterNode primary = cctx.affinity().primary(cacheKey, topVer); + ClusterNode primary = cctx.affinity().primaryByKey(cacheKey, topVer); if (primary == null) throw new ClusterTopologyServerNotFoundException("Failed to map keys for cache (all partition nodes " + diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java index a252d9ae30c2b..9bdd1becb7600 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java @@ -937,7 +937,7 @@ else if (conflictRmvVals != null) { else val = EntryProcessorResourceInjectorProxy.wrap(cctx.kernalContext(), (EntryProcessor)val); - ClusterNode primary = cctx.affinity().primary(cacheKey.partition(), topVer); + ClusterNode primary = cctx.affinity().primaryByPartition(cacheKey.partition(), topVer); if (primary == null) throw new ClusterTopologyServerNotFoundException("Failed to map keys for cache (all partition nodes " + @@ -988,7 +988,7 @@ private List mapKey(KeyCacheObject key, AffinityTopologyVersion top // If we can send updates in parallel - do it. return fastMap ? cctx.topology().nodes(affMgr.partition(key), topVer) : - Collections.singletonList(affMgr.primary(key, topVer)); + Collections.singletonList(affMgr.primaryByKey(key, topVer)); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java index c9fc983d4850c..e1e0ec2bf087e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java @@ -174,7 +174,7 @@ public GridDistributedCacheEntry entryExx( AffinityTopologyVersion topVer, boolean allowDetached ) { - return allowDetached && !ctx.affinity().primary(ctx.localNode(), key, topVer) ? + return allowDetached && !ctx.affinity().primaryByKey(ctx.localNode(), key, topVer) ? createEntry(key) : entryExx(key, topVer); } @@ -670,7 +670,7 @@ else if (!skipVals && ctx.config().isStatisticsEnabled()) assert topVer.compareTo(AffinityTopologyVersion.ZERO) > 0; // Send request to remove from remote nodes. - ClusterNode primary = ctx.affinity().primary(key, topVer); + ClusterNode primary = ctx.affinity().primaryByKey(key, topVer); if (primary == null) { if (log.isDebugEnabled()) @@ -790,7 +790,7 @@ public void removeLocks(long threadId, GridCacheVersion ver, Collection keys, AffinityTopologyVe boolean explicit = false; for (KeyCacheObject key : keys) { - if (!cctx.affinity().primary(cctx.localNode(), key, topVer)) { + if (!cctx.affinity().primaryByKey(cctx.localNode(), key, topVer)) { // Remove explicit locks added so far. for (KeyCacheObject k : keys) cctx.mvcc().removeExplicitLock(threadId, cctx.txKey(k), lockVer); @@ -1285,7 +1285,7 @@ private GridNearLockMapping map( ) throws IgniteCheckedException { assert mapping == null || mapping.node() != null; - ClusterNode primary = cctx.affinity().primary(key, topVer); + ClusterNode primary = cctx.affinity().primaryByKey(key, topVer); if (primary == null) throw new ClusterTopologyServerNotFoundException("Failed to lock keys " + diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java index daae1e2aa144a..8586b69b5961d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java @@ -598,7 +598,7 @@ public void handleSupplyMessage( for (Map.Entry e : supply.infos().entrySet()) { int p = e.getKey(); - if (cctx.affinity().localNode(p, topVer)) { + if (cctx.affinity().partitionLocalNode(p, topVer)) { GridDhtLocalPartition part = top.localPartition(p, topVer, true); assert part != null; @@ -668,7 +668,7 @@ public void handleSupplyMessage( // Only request partitions based on latest topology version. for (Integer miss : supply.missed()) { - if (cctx.affinity().localNode(miss, topVer)) + if (cctx.affinity().partitionLocalNode(miss, topVer)) fut.partitionMissed(id, miss); } @@ -1359,7 +1359,7 @@ private void demandFromNode( for (Map.Entry e : supply.infos().entrySet()) { int p = e.getKey(); - if (cctx.affinity().localNode(p, topVer)) { + if (cctx.affinity().partitionLocalNode(p, topVer)) { GridDhtLocalPartition part = top.localPartition(p, topVer, true); assert part != null; @@ -1436,7 +1436,7 @@ private void demandFromNode( // Only request partitions based on latest topology version. for (Integer miss : s.supply().missed()) { - if (cctx.affinity().localNode(miss, topVer)) + if (cctx.affinity().partitionLocalNode(miss, topVer)) fut.partitionMissed(node.id(), miss); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplier.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplier.java index b082c4736cb9c..994242354c05d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplier.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplier.java @@ -302,7 +302,7 @@ public void handleDemandMessage(int idx, UUID id, GridDhtPartitionDemandMessage (Iterator)sctx.entryIt : loc.allEntries().iterator(); while (entIt.hasNext()) { - if (!cctx.affinity().belongs(node, part, d.topologyVersion())) { + if (!cctx.affinity().partitionBelongs(node, part, d.topologyVersion())) { // Demander no longer needs this partition, so we send '-1' partition and move on. s.missed(part); @@ -387,7 +387,7 @@ else if (log.isDebugEnabled()) boolean prepared = false; while (iter.hasNext()) { - if (!cctx.affinity().belongs(node, part, d.topologyVersion())) { + if (!cctx.affinity().partitionBelongs(node, part, d.topologyVersion())) { // Demander no longer needs this partition, // so we send '-1' partition and move on. s.missed(part); @@ -510,7 +510,7 @@ else if (log.isDebugEnabled()) (Iterator)sctx.entryIt : entries.iterator(); while (lsnrIt.hasNext()) { - if (!cctx.affinity().belongs(node, part, d.topologyVersion())) { + if (!cctx.affinity().partitionBelongs(node, part, d.topologyVersion())) { // Demander no longer needs this partition, // so we send '-1' partition and move on. s.missed(part); @@ -808,7 +808,7 @@ private void processOldDemandMessage(GridDhtPartitionDemandMessage d, UUID id) { boolean partMissing = false; for (GridCacheEntryEx e : loc.allEntries()) { - if (!cctx.affinity().belongs(node, part, d.topologyVersion())) { + if (!cctx.affinity().partitionBelongs(node, part, d.topologyVersion())) { // Demander no longer needs this partition, so we send '-1' partition and move on. s.missed(part); @@ -859,7 +859,7 @@ else if (log.isDebugEnabled()) boolean prepared = false; for (Map.Entry e : iter) { - if (!cctx.affinity().belongs(node, part, d.topologyVersion())) { + if (!cctx.affinity().partitionBelongs(node, part, d.topologyVersion())) { // Demander no longer needs this partition, // so we send '-1' partition and move on. s.missed(part); @@ -947,7 +947,7 @@ else if (log.isDebugEnabled()) swapLsnr = null; for (GridCacheEntryInfo info : entries) { - if (!cctx.affinity().belongs(node, part, d.topologyVersion())) { + if (!cctx.affinity().partitionBelongs(node, part, d.topologyVersion())) { // Demander no longer needs this partition, // so we send '-1' partition and move on. s.missed(part); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java index 692e7c0de45a6..c012e452220ac 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java @@ -289,7 +289,7 @@ private IgniteCheckedException stopError() { } // If partition belongs to local node. - if (cctx.affinity().localNode(p, topVer)) { + if (cctx.affinity().partitionLocalNode(p, topVer)) { GridDhtLocalPartition part = top.localPartition(p, topVer, true); assert part != null; @@ -349,7 +349,7 @@ private IgniteCheckedException stopError() { * @return Picked owners. */ private Collection pickedOwners(int p, AffinityTopologyVersion topVer) { - Collection affNodes = cctx.affinity().nodes(p, topVer); + Collection affNodes = cctx.affinity().nodesByPartition(p, topVer); int affCnt = affNodes.size(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java index b843e4e6d3db1..41632ef338b58 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java @@ -161,7 +161,7 @@ public void processNearAtomicUpdateResponse( if (F.contains(failed, key)) continue; - if (ctx.affinity().belongs(ctx.localNode(), ctx.affinity().partition(key), req.topologyVersion())) { // Reader became backup. + if (ctx.affinity().partitionBelongs(ctx.localNode(), ctx.affinity().partition(key), req.topologyVersion())) { // Reader became backup. GridCacheEntryEx entry = peekEx(key); if (entry != null && entry.markObsolete(ver)) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java index 30fc213fc63e2..d022805a76bf2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java @@ -112,7 +112,7 @@ public GridNearCacheEntry( return false; } - if (cctx.affinity().backup(cctx.localNode(), part, topVer)) { + if (cctx.affinity().backupByPartition(cctx.localNode(), part, topVer)) { this.topVer = AffinityTopologyVersion.NONE; return false; @@ -162,7 +162,7 @@ public void initializeFromDht(AffinityTopologyVersion topVer) throws GridCacheEn } } - ClusterNode primaryNode = cctx.affinity().primary(key, topVer); + ClusterNode primaryNode = cctx.affinity().primaryByKey(key, topVer); if (primaryNode == null) this.topVer = AffinityTopologyVersion.NONE; @@ -686,7 +686,7 @@ private void primaryNode(UUID nodeId, AffinityTopologyVersion topVer) { ClusterNode primary = null; try { - primary = cctx.affinity().primary(part, topVer); + primary = cctx.affinity().primaryByPartition(part, topVer); } catch (IllegalStateException ignore) { // Do not have affinity history. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java index cb47498be0167..fb2843c02eec9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java @@ -413,7 +413,7 @@ private Map map( ) { int part = cctx.affinity().partition(key); - List affNodes = cctx.affinity().nodes(part, topVer); + List affNodes = cctx.affinity().nodesByPartition(part, topVer); if (affNodes.isEmpty()) { onDone(serverNotFoundError(topVer)); @@ -726,7 +726,7 @@ private Map loadEntries( info.unmarshalValue(cctx, cctx.deploy().globalLoader()); // Entries available locally in DHT should not be loaded into near cache for reading. - if (!cctx.affinity().localNode(info.key(), cctx.affinity().affinityTopologyVersion())) { + if (!cctx.affinity().keyLocalNode(info.key(), cctx.affinity().affinityTopologyVersion())) { GridNearCacheEntry entry = savedEntries.get(info.key()); if (entry == null) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java index 491b4ece0ea13..8035655237635 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java @@ -1371,7 +1371,7 @@ private GridNearLockMapping map( ) throws IgniteCheckedException { assert mapping == null || mapping.node() != null; - ClusterNode primary = cctx.affinity().primary(key, topVer); + ClusterNode primary = cctx.affinity().primaryByKey(key, topVer); if (primary == null) throw new ClusterTopologyServerNotFoundException("Failed to lock keys " + diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java index 4cbfb27d34857..f6ca77a1ef96e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java @@ -527,7 +527,7 @@ private void map( GridCacheContext cacheCtx = entry.context(); List nodes = cacheCtx.isLocal() ? - cacheCtx.affinity().nodes(entry.key(), topVer) : + cacheCtx.affinity().nodesByKey(entry.key(), topVer) : cacheCtx.topology().nodes(cacheCtx.affinity().partition(entry.key()), topVer); txMapping.addMapping(nodes); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java index 91cfbda6e15d3..7e446f3baa0d2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java @@ -601,7 +601,7 @@ private GridDistributedTxMapping map( nodes = cacheCtx.topology().nodes(cached0.partition(), topVer); else nodes = cacheCtx.isLocal() ? - cacheCtx.affinity().nodes(entry.key(), topVer) : + cacheCtx.affinity().nodesByKey(entry.key(), topVer) : cacheCtx.topology().nodes(cacheCtx.affinity().partition(entry.key()), topVer); txMapping.addMapping(nodes); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java index 5c09398b4ecb1..28cd3584362cf 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java @@ -194,7 +194,7 @@ private void preparePessimistic() { GridCacheContext cacheCtx = txEntry.context(); List nodes = cacheCtx.isLocal() ? - cacheCtx.affinity().nodes(txEntry.key(), topVer) : + cacheCtx.affinity().nodesByKey(txEntry.key(), topVer) : cacheCtx.topology().nodes(cacheCtx.affinity().partition(txEntry.key()), topVer); ClusterNode primary = F.first(nodes); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java index b3eb7551f05a0..940dd809779a3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java @@ -476,7 +476,7 @@ private void processLockResponse(UUID nodeId, GridNearLockResponse res) { * @return {@code True} if entry is locally mapped as a primary or back up node. */ protected boolean isNearLocallyMapped(GridCacheEntryEx e, AffinityTopologyVersion topVer) { - return ctx.affinity().belongs(ctx.localNode(), e.partition(), topVer); + return ctx.affinity().partitionBelongs(ctx.localNode(), e.partition(), topVer); } /** @@ -548,7 +548,7 @@ protected boolean evictNearEntry(GridCacheEntryEx e, GridCacheVersion obsoleteVe topVer = cand.topologyVersion(); // Send request to remove from remote nodes. - ClusterNode primary = ctx.affinity().primary(key, topVer); + ClusterNode primary = ctx.affinity().primaryByKey(key, topVer); if (primary == null) { if (log.isDebugEnabled()) @@ -668,7 +668,7 @@ public void removeLocks(GridCacheVersion ver, Collection keys) { map = U.newHashMap(affNodes.size()); } - ClusterNode primary = ctx.affinity().primary(key, cand.topologyVersion()); + ClusterNode primary = ctx.affinity().primaryByKey(key, cand.topologyVersion()); if (primary == null) { if (log.isDebugEnabled()) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java index 46604c7524527..6d3f21f095865 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java @@ -358,7 +358,7 @@ else if (tx.implicit() && tx.isSystemInvalidate()) { // Finish implicit transact GridCacheContext cacheCtx = e.context(); try { - if (e.op() != NOOP && !cacheCtx.affinity().localNode(e.key(), topVer)) { + if (e.op() != NOOP && !cacheCtx.affinity().keyLocalNode(e.key(), topVer)) { GridCacheEntryEx entry = cacheCtx.cache().peekEx(e.key()); if (entry != null) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java index 7efb746d4428e..47f1bed98680c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java @@ -1580,11 +1580,11 @@ protected void runQuery(GridCacheQueryInfo qryInfo) { // Other types are filtered in indexing manager. if (!cctx.isReplicated() && qry.type() == SCAN && qry.partition() == null && cctx.config().getCacheMode() != LOCAL && !incBackups && - !cctx.affinity().primary(cctx.localNode(), key, topVer)) { + !cctx.affinity().primaryByKey(cctx.localNode(), key, topVer)) { if (log.isDebugEnabled()) log.debug("Ignoring backup element [row=" + row + ", cacheMode=" + cctx.config().getCacheMode() + ", incBackups=" + incBackups + - ", primary=" + cctx.affinity().primary(cctx.localNode(), key, topVer) + ']'); + ", primary=" + cctx.affinity().primaryByKey(cctx.localNode(), key, topVer) + ']'); continue; } @@ -1592,7 +1592,8 @@ protected void runQuery(GridCacheQueryInfo qryInfo) { V val = row.getValue(); if (log.isDebugEnabled()) { - ClusterNode primaryNode = CU.primaryNode(cctx, key); + ClusterNode primaryNode = cctx.affinity().primaryByKey(key, + cctx.affinity().affinityTopologyVersion()); log.debug(S.toString("Record", "key", key, true, @@ -2355,7 +2356,7 @@ public Collection sqlMetadata() throws IgniteCheckedExcept return new IgniteBiPredicate() { @Override public boolean apply(K k, V v) { - return cache.context().affinity().primary(ctx.discovery().localNode(), k, NONE); + return cache.context().affinity().primaryByKey(ctx.discovery().localNode(), k, NONE); } }; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java index 165b8b77e0ab4..926c7ce58b293 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java @@ -56,7 +56,6 @@ import org.apache.ignite.internal.managers.deployment.GridDeploymentInfoBean; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.GridCacheAdapter; -import org.apache.ignite.internal.processors.cache.GridCacheAffinityManager; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheDeploymentManager; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicAbstractUpdateFuture; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java index a5f647aa3d2b9..12b02f0fa9386 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java @@ -372,7 +372,7 @@ public void onEntryExpired(GridCacheEntryEx e, KeyCacheObject key, CacheObject o if (F.isEmpty(lsnrCol)) return; - boolean primary = cctx.affinity().primary(cctx.localNode(), e.partition(), AffinityTopologyVersion.NONE); + boolean primary = cctx.affinity().primaryByPartition(cctx.localNode(), e.partition(), AffinityTopologyVersion.NONE); if (cctx.isReplicated() || primary) { boolean recordIgniteEvt = cctx.gridEvents().isRecordable(EVT_CACHE_QUERY_OBJECT_READ); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java index 18c301124405f..b07a1175c436d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java @@ -1288,7 +1288,7 @@ protected void batchStoreCommit(Iterable writeEntries) throws Ign if (!skip && skipNonPrimary) { skip = e.cached().isNear() || e.cached().detached() || - !e.context().affinity().primary(e.cached().partition(), topologyVersion()).isLocal(); + !e.context().affinity().primaryByPartition(e.cached().partition(), topologyVersion()).isLocal(); } if (!skip && !local() && // Update local store at backups only if needed. @@ -1707,7 +1707,7 @@ protected boolean isNearLocallyMapped(IgniteTxEntry e, boolean primaryOnly) { int part = cached != null ? cached.partition() : cacheCtx.affinity().partition(e.key()); - List affNodes = cacheCtx.affinity().nodes(part, topologyVersion()); + List affNodes = cacheCtx.affinity().nodesByPartition(part, topologyVersion()); e.locallyMapped(F.contains(affNodes, cctx.localNode())); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java index 1a9b082b3793f..7ac439807a61f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java @@ -1112,7 +1112,7 @@ assert isWriteToStoreFromDhtValid(stores) : * @return {@code True} if local node is current primary for given entry. */ private boolean primaryLocal(GridCacheEntryEx entry) { - return entry.context().affinity().primary(cctx.localNode(), entry.partition(), AffinityTopologyVersion.NONE); + return entry.context().affinity().primaryByPartition(cctx.localNode(), entry.partition(), AffinityTopologyVersion.NONE); } /** @@ -1413,7 +1413,7 @@ private Collection enlistRead( finally { if (entry != null && readCommitted()) { if (cacheCtx.isNear()) { - if (cacheCtx.affinity().belongs(cacheCtx.localNode(), entry.partition(), topVer)) { + if (cacheCtx.affinity().partitionBelongs(cacheCtx.localNode(), entry.partition(), topVer)) { if (entry.markObsolete(xidVer)) cacheCtx.cache().removeEntry(entry); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetection.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetection.java index 70d938e144ead..67d00ea782923 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetection.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetection.java @@ -401,7 +401,7 @@ private void mapTxKeys(@Nullable Set txKeys, Map dataNodes(AffinityTopologyVersion topVer) throws Collection nodes; if (collocated) { - List nodes0 = ctx.affinity().nodes(hdrPart, topVer); + List nodes0 = ctx.affinity().nodesByPartition(hdrPart, topVer); nodes = !nodes0.isEmpty() ? Collections.singleton(nodes0.contains(ctx.localNode()) ? ctx.localNode() : F.first(nodes0)) : nodes0; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridSetQueryPredicate.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridSetQueryPredicate.java index e8b2cc7f0f4b8..bc6c1827a0774 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridSetQueryPredicate.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridSetQueryPredicate.java @@ -91,7 +91,7 @@ public IgniteUuid setId() { /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public boolean apply(K k, V v) { - return !filter || ctx.affinity().primary(ctx.localNode(), k, ctx.affinity().affinityTopologyVersion()); + return !filter || ctx.affinity().primaryByKey(ctx.localNode(), k, ctx.affinity().affinityTopologyVersion()); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobProcessor.java index 7889a5579c7bb..992e02a6fc570 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobProcessor.java @@ -1568,7 +1568,7 @@ public PartitionsReservation(int[] cacheIds, int partId, } } finally { - if (checkPartMapping && !cctx.affinity().primary(partId, topVer).id().equals(ctx.localNodeId())) + if (checkPartMapping && !cctx.affinity().primaryByPartition(partId, topVer).id().equals(ctx.localNodeId())) throw new IgniteException("Failed partition reservation. " + "Partition is not primary on the node. [partition=" + partId + ", cacheName=" + cctx.name() + ", nodeId=" + ctx.localNodeId() + ", topology=" + topVer + ']'); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheAffinityCallSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheAffinityCallSelfTest.java index 92e2b9b2ba21d..b0337d62967e0 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheAffinityCallSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheAffinityCallSelfTest.java @@ -214,12 +214,12 @@ public CheckCallable(Object key, AffinityTopologyVersion topVer) { ClusterNode loc = ignite.cluster().localNode(); - if (loc.equals(aff.primary(key, topVer))) + if (loc.equals(aff.primaryByKey(key, topVer))) return true; AffinityTopologyVersion topVer0 = new AffinityTopologyVersion(topVer.topologyVersion() + 1, 0); - assertEquals(loc, aff.primary(key, topVer0)); + assertEquals(loc, aff.primaryByKey(key, topVer0)); } return null; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java index 1cfb330da8cd6..53b4900410033 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java @@ -5837,7 +5837,7 @@ public CheckEntriesTask(Collection keys) { int size = 0; for (String key : keys) { - if (ctx.affinity().localNode(key, ctx.discovery().topologyVersionEx())) { + if (ctx.affinity().keyLocalNode(key, ctx.discovery().topologyVersionEx())) { GridCacheEntryEx e = ctx.isNear() ? ctx.near().dht().peekEx(key) : ctx.cache().peekEx(key); @@ -5873,7 +5873,7 @@ private static class CheckCacheSizeTask extends TestIgniteIdxRunnable { int size = 0; for (String key : map.keySet()) - if (ctx.affinity().localNode(key, ctx.discovery().topologyVersionEx())) + if (ctx.affinity().keyLocalNode(key, ctx.discovery().topologyVersionEx())) size++; assertEquals("Incorrect key size on cache #" + idx, size, ignite.cache(ctx.name()).localSize(ALL)); @@ -6116,7 +6116,7 @@ public CheckKeySizeTask(Collection keys) { int size = 0; for (String key : keys) - if (ctx.affinity().localNode(key, ctx.discovery().topologyVersionEx())) + if (ctx.affinity().keyLocalNode(key, ctx.discovery().topologyVersionEx())) size++; assertEquals("Incorrect key size on cache #" + idx, size, ignite.cache(null).localSize(ALL)); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigVariationsFullApiTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigVariationsFullApiTest.java index 6b0e1932e1f2c..d4449f9ee9228 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigVariationsFullApiTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigVariationsFullApiTest.java @@ -5548,7 +5548,7 @@ private static class CheckEntriesTask extends TestIgniteIdxRunnable { int size = 0; for (String key : keys) { - if (ctx.affinity().localNode(key, ctx.discovery().topologyVersionEx())) { + if (ctx.affinity().keyLocalNode(key, ctx.discovery().topologyVersionEx())) { GridCacheEntryEx e = ctx.isNear() ? ctx.near().dht().peekEx(key) : ctx.cache().peekEx(key); @@ -5589,7 +5589,7 @@ private static class CheckCacheSizeTask extends TestIgniteIdxRunnable { int size = 0; for (String key : map.keySet()) - if (ctx.affinity().localNode(key, ctx.discovery().topologyVersionEx())) + if (ctx.affinity().keyLocalNode(key, ctx.discovery().topologyVersionEx())) size++; assertEquals("Incorrect key size on cache #" + idx, size, ignite.cache(ctx.name()).localSize(ALL)); @@ -5850,7 +5850,7 @@ public CheckKeySizeTask(Collection keys, String s) { int size = 0; for (String key : keys) - if (ctx.affinity().localNode(key, ctx.discovery().topologyVersionEx())) + if (ctx.affinity().keyLocalNode(key, ctx.discovery().topologyVersionEx())) size++; assertEquals("Incorrect key size on cache #" + idx, size, ignite.cache(cacheName).localSize(ALL)); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePeekModesAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePeekModesAbstractTest.java index 6c577c63ab693..fac24ccd00045 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePeekModesAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePeekModesAbstractTest.java @@ -1009,9 +1009,9 @@ private T2 swapKeysCount(int nodeIdx, int part) throws IgniteC //And then find out whether they are primary or backup ones. int primaryCnt = 0; int backupCnt = 0; - if (affinity.primary(ctx.localNode(), part, topVer)) + if (affinity.primaryByPartition(ctx.localNode(), part, topVer)) primaryCnt = cnt; - else if (affinity.backup(ctx.localNode(), part, topVer)) + else if (affinity.primaryByPartition(ctx.localNode(), part, topVer)) backupCnt = cnt; return new T2<>(primaryCnt, backupCnt); } @@ -1081,9 +1081,9 @@ private T2 offheapKeysCount(int nodeIdx, int part) throws Igni //And then find out whether they are primary or backup ones. int primaryCnt = 0; int backupCnt = 0; - if (affinity.primary(ctx.localNode(), part, topVer)) + if (affinity.primaryByPartition(ctx.localNode(), part, topVer)) primaryCnt = cnt; - else if (affinity.backup(ctx.localNode(), part, topVer)) + else if (affinity.backupByPartition(ctx.localNode(), part, topVer)) backupCnt = cnt; return new T2<>(primaryCnt, backupCnt); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractQueueFailoverDataConsistencySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractQueueFailoverDataConsistencySelfTest.java index 45b4b9f299aa3..aeca2fb2279ff 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractQueueFailoverDataConsistencySelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractQueueFailoverDataConsistencySelfTest.java @@ -365,7 +365,7 @@ private int primaryQueueNode(IgniteQueue queue) { for (int i = 0; i < gridCount(); i++) { for (GridCacheEntryEx e : ((IgniteKernal)grid(i)).context().cache().internalCache(cctx.name()).allEntries()) { - if (aff.primary(grid(i).localNode(), e.key(), AffinityTopologyVersion.NONE) + if (aff.primaryByKey(grid(i).localNode(), e.key(), AffinityTopologyVersion.NONE) && e.key().value(cctx.cacheObjectContext(), false) instanceof GridCacheQueueHeaderKey) return i; } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java index b4ef11a82c0c1..8709d05664d94 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java @@ -853,13 +853,13 @@ public void testPessimisticTx2() throws Exception { GridCacheAffinityManager aff = ignite0.context().cache().internalCache(null).context().affinity(); - List nodes1 = aff.nodes(key1, topVer1); - List nodes2 = aff.nodes(key1, topVer2); + List nodes1 = aff.nodesByKey(key1, topVer1); + List nodes2 = aff.nodesByKey(key1, topVer2); assertEquals(nodes1, nodes2); - nodes1 = aff.nodes(key2, topVer1); - nodes2 = aff.nodes(key2, topVer2); + nodes1 = aff.nodesByKey(key2, topVer1); + nodes2 = aff.nodesByKey(key2, topVer2); assertFalse(nodes1.get(0).equals(nodes2.get(0))); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionTest.java index aa240aae439a9..b909de90fd42a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionTest.java @@ -286,7 +286,7 @@ private void doTestDeadlock( key = keys.get(1); ClusterNode primaryNode = - ((IgniteCacheProxy)cache).context().affinity().primary(key, NONE); + ((IgniteCacheProxy)cache).context().affinity().primaryByKey(key, NONE); List primaryKeys = primaryKeys(grid(primaryNode).cache(CACHE_NAME), 5, key + (100 * threadNum)); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxPessimisticDeadlockDetectionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxPessimisticDeadlockDetectionTest.java index 83eb908d21978..ced8b61166876 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxPessimisticDeadlockDetectionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxPessimisticDeadlockDetectionTest.java @@ -293,7 +293,7 @@ private void doTestDeadlock( key = keys.get(1); ClusterNode primaryNode = - ((IgniteCacheProxy)cache).context().affinity().primary(key, NONE); + ((IgniteCacheProxy)cache).context().affinity().primaryByKey(key, NONE); List primaryKeys = primaryKeys(grid(primaryNode).cache(CACHE_NAME), 5, key + (100 * threadNum)); diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java index 8b2993f263cf7..155edeeeeb229 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java @@ -2061,7 +2061,7 @@ private void createSqlFunctions(String schema, Class[] clss) throws IgniteChe return new IgniteBiPredicate() { @Override public boolean apply(K k, V v) { - return aff.primary(locNode, k, topVer0); + return aff.primaryByKey(locNode, k, topVer0); } }; } diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java index 22b94c7ac796b..3700774260ded 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java @@ -612,7 +612,7 @@ private ClusterNode rangeNode(GridCacheContext cctx, GridH2QueryContext qct node = cctx.discovery().node(nodeId); } else // Get primary node for current topology version. - node = cctx.affinity().primary(affKeyObj, qctx.topologyVersion()); + node = cctx.affinity().primaryByKey(affKeyObj, qctx.topologyVersion()); if (node == null) // Node was not found, probably topology changed and we need to retry the whole query. throw new GridH2RetryException("Failed to find node."); From 4b20d03c148c83bf3665d3296ecbf8f768a43e0c Mon Sep 17 00:00:00 2001 From: Andrey Gura Date: Wed, 30 Aug 2017 18:12:17 +0300 Subject: [PATCH 81/90] gg-12723 Optimistic tx recovery fixed --- .../cache/distributed/dht/GridDhtTxLocal.java | 5 +- ...iteTxRecoveryAfterStoreCommitSelfTest.java | 363 ++++++++++++++++++ .../IgniteCacheTxRecoverySelfTestSuite.java | 10 +- 3 files changed, 369 insertions(+), 9 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteTxRecoveryAfterStoreCommitSelfTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java index b659abb8ef679..9220c9ebd7e91 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java @@ -437,9 +437,12 @@ public IgniteInternalFuture prepareAsync( if (state() != PREPARING) { if (!state(PREPARING)) { - if (state() == PREPARED && isSystemInvalidate()) + if (state() == PREPARED && isSystemInvalidate()) { fut.complete(); + return fut; + } + if (setRollbackOnly()) { if (timeout == -1) fut.onError(new IgniteTxTimeoutCheckedException("Transaction timed out and was rolled back: " + diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteTxRecoveryAfterStoreCommitSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteTxRecoveryAfterStoreCommitSelfTest.java new file mode 100644 index 0000000000000..fa37f23ed887f --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteTxRecoveryAfterStoreCommitSelfTest.java @@ -0,0 +1,363 @@ +/* + * 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.cache.distributed.dht; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.Callable; +import java.util.concurrent.CountDownLatch; +import javax.cache.Cache; +import javax.cache.configuration.Factory; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteException; +import org.apache.ignite.cache.store.CacheStore; +import org.apache.ignite.cache.store.CacheStoreAdapter; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.IgniteKernal; +import org.apache.ignite.internal.processors.cache.GridCacheAbstractSelfTest; +import org.apache.ignite.internal.processors.cache.GridCacheAdapter; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheAdapter; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager; +import org.apache.ignite.internal.transactions.IgniteTxHeuristicCheckedException; +import org.apache.ignite.internal.util.lang.GridAbsPredicate; +import org.apache.ignite.internal.util.typedef.G; +import org.apache.ignite.lang.IgniteCallable; +import org.apache.ignite.lang.IgniteFutureTimeoutException; +import org.apache.ignite.resources.IgniteInstanceResource; +import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; +import org.apache.ignite.spi.discovery.tcp.TestTcpDiscoverySpi; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.transactions.Transaction; +import org.apache.ignite.transactions.TransactionIsolation; +import org.apache.ignite.util.TestTcpCommunicationSpi; +import org.jsr166.ConcurrentHashMap8; + +import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC; +import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; + +/** + * Tests that transaction is invalidated in case of {@link IgniteTxHeuristicCheckedException}. + */ +public class IgniteTxRecoveryAfterStoreCommitSelfTest extends GridCacheAbstractSelfTest { + /** Cache name. */ + private static final String CACHE_NAME = "cache"; + + /** Store map. */ + public static final Map storeMap = new ConcurrentHashMap8<>(); + + /** */ + private static volatile CountDownLatch storeCommitLatch; + + /** */ + private static volatile CountDownLatch nodeFailLatch; + + /** {@inheritDoc} */ + @Override protected int gridCount() { + return 5; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + // No-op. + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + startGridsMultiThreaded(gridCount()); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + // No-op + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** + * @return Index of node starting transaction. + */ + protected int originatingNode() { + return 0; + } + + @Override protected long getTestTimeout() { + return 300_000; + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + TcpCommunicationSpi comm = new TestTcpCommunicationSpi(); + + comm.setSharedMemoryPort(-1); + + TestTcpDiscoverySpi discoSpi = new TestTcpDiscoverySpi(); + + discoSpi.setIpFinder(GridCacheAbstractSelfTest.ipFinder); + + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName) + .setFailureDetectionTimeout(5_000) + .setDiscoverySpi(discoSpi) + .setCommunicationSpi(comm); + + if (igniteInstanceName.endsWith("0")) + cfg.setUserAttributes(Collections.singletonMap("ORIGINATOR", true)); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected CacheConfiguration cacheConfiguration(String igniteInstanceName) throws Exception { + CacheConfiguration cfg = super.cacheConfiguration(igniteInstanceName); + + cfg.setName(CACHE_NAME); + + cfg.setCacheStoreFactory(new TestStoreFactory()); + + cfg.setReadThrough(true); + cfg.setWriteThrough(true); + + return cfg; + } + + /** + * @throws Exception If failed. + */ + public void testManyKeysCommit() throws Exception { + Collection keys = new ArrayList<>(200); + + for (int i = 0; i < 20; i++) + keys.add(i); + + testTxOriginatingNodeFails(keys); + } + + /** + * @param keys Keys to update. + * @throws Exception If failed. + */ + protected void testTxOriginatingNodeFails(final Collection keys) throws Exception { + assertFalse(keys.isEmpty()); + + final Collection grids = new ArrayList<>(); + + ClusterNode txNode = grid(originatingNode()).localNode(); + + for (int i = 1; i < gridCount(); i++) + grids.add((IgniteKernal)grid(i)); + + final Map expectedStoreState = new HashMap<>(); + + final String initVal = "initialValue"; + + for (Integer key : keys) { + grid(originatingNode()).cache(CACHE_NAME).put(key, initVal); + + expectedStoreState.put(key, String.valueOf(key)); + } + + Map> nodeMap = new HashMap<>(); + + info("Node being checked: " + grid(1).localNode().id()); + + for (Integer key : keys) { + Collection nodes = new ArrayList<>(); + + nodes.addAll(grid(1).affinity(CACHE_NAME).mapKeyToPrimaryAndBackups(key)); + + nodes.remove(txNode); + + nodeMap.put(key, nodes); + } + + info("Starting tx [values=" + expectedStoreState + ", topVer=" + + grid(1).context().discovery().topologyVersion() + ']'); + + final IgniteEx originatingNodeGrid = grid(originatingNode()); + + storeCommitLatch = new CountDownLatch(1); + + nodeFailLatch = new CountDownLatch(1); + + GridTestUtils.runAsync(new Callable() { + @Override public Void call() throws Exception { + IgniteCache cache = originatingNodeGrid.cache(CACHE_NAME); + + assertNotNull(cache); + + Transaction tx = originatingNodeGrid.transactions().txStart(OPTIMISTIC, TransactionIsolation.SERIALIZABLE); + + try { + cache.putAll(expectedStoreState); + + info("Before commit"); + + tx.commit(); + } + catch (IgniteFutureTimeoutException ignored) { + info("Failed to wait for commit future completion"); + } + + return null; + } + }); + + nodeFailLatch.await(); + + for (Integer key : expectedStoreState.keySet()) + assertEquals(expectedStoreState.get(key), storeMap.get(key)); + + info(">>> Stopping originating node " + txNode); + + ((TestTcpDiscoverySpi)grid(originatingNode()).context().config().getDiscoverySpi()).simulateNodeFailure(); + ((TestTcpCommunicationSpi)grid(originatingNode()).context().config().getCommunicationSpi()).simulateNodeFailure(); + + storeCommitLatch.countDown(); + + G.stop(grid(originatingNode()).name(), true); + + info(">>> Stopped originating node: " + txNode.id()); + + boolean txFinished = GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + for (IgniteKernal g : grids) { + GridCacheAdapter cache = g.internalCache(CACHE_NAME); + + IgniteTxManager txMgr = cache.isNear() ? + ((GridNearCacheAdapter)cache).dht().context().tm() : + cache.context().tm(); + + int txNum = txMgr.idMapSize(); + + if (txNum != 0) + return false; + } + + return true; + } + }, 300_000); + + assertTrue(txFinished); + + info("Transactions finished."); + + for (Map.Entry> e : nodeMap.entrySet()) { + final Integer key = e.getKey(); + + final String val = expectedStoreState.get(key); + + assertFalse(e.getValue().isEmpty()); + + for (ClusterNode node : e.getValue()) { + final UUID checkNodeId = node.id(); + + compute(G.ignite(checkNodeId).cluster().forNode(node)).call(new IgniteCallable() { + /** */ + @IgniteInstanceResource + private Ignite ignite; + + @Override public Void call() throws Exception { + IgniteCache cache = ignite.cache(CACHE_NAME); + + assertNotNull(cache); + + assertEquals("Failed to check entry value on node: " + checkNodeId, + val, cache.get(key)); + + return null; + } + }); + } + } + + for (Map.Entry e : expectedStoreState.entrySet()) { + for (Ignite g : G.allGrids()) + assertEquals(e.getValue(), g.cache(CACHE_NAME).get(e.getKey())); + } + } + + /** + * + */ + public static class TestStoreFactory implements Factory { + @IgniteInstanceResource + Ignite ignite; + + /** {@inheritDoc} */ + @Override public CacheStore create() { + return new TestStore(ignite.cluster().localNode().attribute("ORIGINATOR") != null); + } + } + + /** + * + */ + public static class TestStore extends CacheStoreAdapter { + /** */ + private boolean originatorNodeFlag; + + /** */ + public TestStore(boolean originatorNodeFlag) { + + this.originatorNodeFlag = originatorNodeFlag; + } + + /** {@inheritDoc} */ + @Override public Object load(Object key) { + return storeMap.get(key); + } + + /** {@inheritDoc} */ + @Override public void write(Cache.Entry entry) { + storeMap.put(entry.getKey(), entry.getValue()); + } + + /** {@inheritDoc} */ + @Override public void delete(Object key) { + storeMap.remove(key); + } + + @Override public void sessionEnd(boolean commit) { + if (!originatorNodeFlag) + return; + + if (storeCommitLatch != null) { + try { + nodeFailLatch.countDown(); + + storeCommitLatch.await(); + + throw new IgniteException(); + } + catch (InterruptedException e) { + throw new IgniteException(e); + } + } + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTxRecoverySelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTxRecoverySelfTestSuite.java index c7c8db67fe31e..e901cf5489220 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTxRecoverySelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTxRecoverySelfTestSuite.java @@ -18,14 +18,7 @@ package org.apache.ignite.testsuites; import junit.framework.TestSuite; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheColocatedTxPessimisticOriginatingNodeFailureSelfTest; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedNearDisabledTxOriginatingNodeFailureSelfTest; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedTxOriginatingNodeFailureSelfTest; -import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCacheCommitDelayTxRecoveryTest; -import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCachePartitionedNearDisabledPrimaryNodeFailureRecoveryTest; -import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCachePartitionedPrimaryNodeFailureRecoveryTest; -import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCachePartitionedTwoBackupsPrimaryNodeFailureRecoveryTest; -import org.apache.ignite.internal.processors.cache.distributed.dht.TxRecoveryStoreEnabledTest; +import org.apache.ignite.internal.processors.cache.distributed.dht.*; import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheNearTxPessimisticOriginatingNodeFailureSelfTest; import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCacheReplicatedTxOriginatingNodeFailureSelfTest; import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCacheReplicatedTxPessimisticOriginatingNodeFailureSelfTest; @@ -56,6 +49,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(GridCacheReplicatedTxPessimisticOriginatingNodeFailureSelfTest.class); suite.addTestSuite(TxRecoveryStoreEnabledTest.class); + suite.addTestSuite(IgniteTxRecoveryAfterStoreCommitSelfTest.class); return suite; } From dbc340a37664f8e5313ab085af34f59ce963032b Mon Sep 17 00:00:00 2001 From: Konstantin Boudnik Date: Mon, 5 Jun 2017 19:47:02 -0700 Subject: [PATCH 82/90] IGNITE-5413. Ignite shouldn't expose nor send (clear-text) env variables to a 3rd endpoint --- .../ignite/internal/processors/cluster/ClusterProcessor.java | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) 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 6500cf3e8faee..1d02749d009cd 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 @@ -58,7 +58,7 @@ public class ClusterProcessor extends GridProcessorAdapter { private IgniteClusterImpl cluster; /** */ - private final AtomicBoolean notifyEnabled = new AtomicBoolean(); + private final AtomicBoolean notifyEnabled = new AtomicBoolean(false); /** */ @GridToStringExclude @@ -74,9 +74,6 @@ public class ClusterProcessor extends GridProcessorAdapter { public ClusterProcessor(GridKernalContext ctx) { super(ctx); - notifyEnabled.set(IgniteSystemProperties.getBoolean(IGNITE_UPDATE_NOTIFIER, - Boolean.parseBoolean(IgniteProperties.get("ignite.update.notifier.enabled.by.default")))); - cluster = new IgniteClusterImpl(ctx); } From 825409fb17e414cdb376a96074b62262eead859a Mon Sep 17 00:00:00 2001 From: Sergey Chugunov Date: Thu, 13 Jul 2017 18:34:01 +0300 Subject: [PATCH 83/90] Functionality of GridVersionSelfTest is debated now --- .../java/org/apache/ignite/internal/GridVersionSelfTest.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridVersionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridVersionSelfTest.java index 4751a0c4fb49b..13af907489f4f 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/GridVersionSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/GridVersionSelfTest.java @@ -32,6 +32,8 @@ public class GridVersionSelfTest extends GridCommonAbstractTest { * @throws Exception If failed. */ public void testVersions() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-5413"); + String propVal = System.getProperty(IGNITE_UPDATE_NOTIFIER); System.setProperty(IGNITE_UPDATE_NOTIFIER, "true"); From 41d92442c16cce673523295dfe0a7ffac686003a Mon Sep 17 00:00:00 2001 From: Sergey Chugunov Date: Thu, 13 Jul 2017 17:32:06 +0300 Subject: [PATCH 84/90] Functionality of muted test is debated now --- .../internal/IgniteUpdateNotifierPerClusterSettingSelfTest.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteUpdateNotifierPerClusterSettingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteUpdateNotifierPerClusterSettingSelfTest.java index a255f15345782..9fca528a691d5 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteUpdateNotifierPerClusterSettingSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteUpdateNotifierPerClusterSettingSelfTest.java @@ -66,6 +66,8 @@ public class IgniteUpdateNotifierPerClusterSettingSelfTest extends GridCommonAbs * @throws Exception If failed. */ public void testNotifierEnabledForCluster() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-5413"); + checkNotifierStatusForCluster(true); } From d8fe81f827b4db1800276e5d76ead86e1648c224 Mon Sep 17 00:00:00 2001 From: mcherkasov Date: Wed, 16 Aug 2017 00:24:07 +0300 Subject: [PATCH 85/90] IgniteCacheNearRestartRollbackSelfTest#testRestarts is muted. --- .../distributed/IgniteCacheNearRestartRollbackSelfTest.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheNearRestartRollbackSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheNearRestartRollbackSelfTest.java index 3f242b5726867..aea4d7782d858 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheNearRestartRollbackSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheNearRestartRollbackSelfTest.java @@ -132,6 +132,8 @@ protected CacheConfiguration cacheConfiguration(String gridName) */ @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter") public void testRestarts() throws Exception { + fail("https://ggsystems.atlassian.net/browse/GG-12398"); + startGrids(4); Ignite tester = ignite(3); From 882f4b40883b11a2cda5a86b1fa2f2af9f103d32 Mon Sep 17 00:00:00 2001 From: Slava Koptilin Date: Thu, 31 Aug 2017 16:50:33 +0300 Subject: [PATCH 86/90] Fixed flaky test IgniteCacheEntryListener* --- .../query/continuous/CacheContinuousQueryEntry.java | 9 +-------- .../query/continuous/CacheContinuousQueryHandler.java | 2 -- 2 files changed, 1 insertion(+), 10 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEntry.java index 366a1e05fa46c..ffbbc2c1e05ef 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEntry.java @@ -199,13 +199,6 @@ void markFiltered() { depInfo = null; } - /** - * @param topVer Topology version. - */ - void topologyVersion(AffinityTopologyVersion topVer) { - this.topVer = topVer; - } - /** * @return Size include this event and filtered. */ @@ -222,7 +215,7 @@ CacheContinuousQueryEntry forBackupQueue() { return this; CacheContinuousQueryEntry e = - new CacheContinuousQueryEntry(cacheId, null, null, null, null, keepBinary, part, updateCntr, null); + new CacheContinuousQueryEntry(cacheId, null, null, null, null, keepBinary, part, updateCntr, topVer); e.flags = flags; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java index 926c7ce58b293..0cdbec5013b15 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java @@ -465,8 +465,6 @@ public void keepBinary(boolean keepBinary) { for (CacheContinuousQueryEntry e : backupQueue0) { if (!e.isFiltered()) prepareEntry(cctx, nodeId, e); - - e.topologyVersion(topVer); } ctx.continuous().addBackupNotification(nodeId, routineId, backupQueue0, topic); From fcdf7a874147c1c9e4241a6663e320c82588d4bd Mon Sep 17 00:00:00 2001 From: Slava Koptilin Date: Fri, 1 Sep 2017 14:52:32 +0300 Subject: [PATCH 87/90] ignite-6053: fixed clear() on local cache --- .../processors/cache/GridCacheAdapter.java | 42 +++++++-- .../local/GridCacheLocalFullApiSelfTest.java | 92 ++++++++++++++++++- 2 files changed, 127 insertions(+), 7 deletions(-) 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 9b1e0cc7721c1..2d2b4f2ce12b4 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 @@ -1196,8 +1196,17 @@ public List> splitClearLocally(boolean srv, bool * @throws IgniteCheckedException In case of error. */ private void clear(@Nullable Set keys) throws IgniteCheckedException { - executeClearTask(keys, false).get(); - executeClearTask(keys, true).get(); + if (isLocal()) { + if (keys == null) + clearLocally(true, false, false); + else + clearLocallyAll(keys, true, false, false); + } + else { + executeClearTask(keys, false).get(); + + executeClearTask(keys, true).get(); + } } /** @@ -1205,13 +1214,34 @@ private void clear(@Nullable Set keys) throws IgniteCheckedExceptio * @return Future. */ private IgniteInternalFuture clearAsync(@Nullable final Set keys) { - return executeClearTask(keys, false).chain(new CX1, Object>() { - @Override public Object applyx(IgniteInternalFuture fut) throws IgniteCheckedException { - executeClearTask(keys, true).get(); + if (isLocal()) + return clearLocallyAsync(keys); + else { + return executeClearTask(keys, false).chain(new CX1, Object>() { + @Override public Object applyx(IgniteInternalFuture fut) throws IgniteCheckedException { + executeClearTask(keys, true).get(); + + return null; + } + }); + } + } + + /** + * @param keys Keys to clear. + * @return Clear future. + */ + private IgniteInternalFuture clearLocallyAsync(@Nullable Set keys) { + return ctx.closures().callLocalSafe(new Callable() { + @Override public Object call() throws Exception { + if (keys == null) + clearLocally(true, false, false); + else + clearLocallyAll(keys, true, false, false); return null; } - }); + }, false); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalFullApiSelfTest.java index f499c26d547c4..8a844bcaa6daa 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalFullApiSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalFullApiSelfTest.java @@ -17,7 +17,9 @@ package org.apache.ignite.internal.processors.cache.local; +import java.util.Arrays; import java.util.Collection; +import java.util.HashSet; import java.util.Map; import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.CacheMode; @@ -89,4 +91,92 @@ public void testMapKeysToNodes() throws Exception { for (String key : keys) assert "key1".equals(key) || "key2".equals(key); } -} \ No newline at end of file + + /** + * @throws Exception If failed. + */ + public void testLocalClearAsync() throws Exception { + localCacheClear(true); + } + + /** + * @throws Exception If failed. + */ + public void testLocalClear() throws Exception { + localCacheClear(false); + } + + /** + * @param async If {@code true} uses async method. + * @throws Exception If failed. + */ + private void localCacheClear(boolean async) throws Exception { + // In addition to the existing tests, it confirms the data is cleared only on one node, + // not on all nodes that have local caches with same names. + try { + startGrid(1); + + IgniteCache cache = jcache(); + + IgniteCache asyncCache = cache.withAsync(); + + for (int i = 0; i < 5; i++) { + cache.put(String.valueOf(i), i); + jcache(1).put(String.valueOf(i), i); + } + + if (async) { + asyncCache.clear("4"); + + asyncCache.future().get(); + } + else + cache.clear("4"); + + assertNull(peek(cache, "4")); + assertNotNull(peek(jcache(1), "4")); + + if (async) { + asyncCache.clearAll(new HashSet<>(Arrays.asList("2", "3"))); + + asyncCache.future().get(); + } + else + cache.clearAll(new HashSet<>(Arrays.asList("2", "3"))); + + for (int i = 2; i < 4; i++) { + assertNull(peek(cache, String.valueOf(i))); + assertNotNull(peek(jcache(1), String.valueOf(i))); + } + + if (async) { + asyncCache.clear(); + + asyncCache.future().get(); + } + else + cache.clear(); + + for (int i = 0; i < 2; i++) { + assertNull(peek(cache, String.valueOf(i))); + assertNotNull(peek(jcache(1), String.valueOf(i))); + } + + if (async) { + IgniteCache asyncCache1 = jcache(1).withAsync(); + + asyncCache1.clear(); + + asyncCache1.future().get(); + } + else + jcache(1).clear(); + + for (int i = 0; i < 2; i++) + assert jcache(i).localSize() == 0; + } + finally { + stopGrid(1); + } + } +} From eb6db758af64610745957ad5b0d165302d7972d3 Mon Sep 17 00:00:00 2001 From: Slava Koptilin Date: Fri, 1 Sep 2017 18:14:05 +0300 Subject: [PATCH 88/90] ignite-6053: fixed clear() on local cache ('keys' parameter must be final) --- .../ignite/internal/processors/cache/GridCacheAdapter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 2d2b4f2ce12b4..d947aa9cb13b0 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 @@ -1231,7 +1231,7 @@ private IgniteInternalFuture clearAsync(@Nullable final Set keys * @param keys Keys to clear. * @return Clear future. */ - private IgniteInternalFuture clearLocallyAsync(@Nullable Set keys) { + private IgniteInternalFuture clearLocallyAsync(@Nullable final Set keys) { return ctx.closures().callLocalSafe(new Callable() { @Override public Object call() throws Exception { if (keys == null) From da99e6e474ce1f8f1cc8df9c4b2015b40101a450 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Wed, 6 Sep 2017 16:46:16 +0300 Subject: [PATCH 89/90] GG-12690 - Deregister local continuous query handlers on cache stop. --- .../continuous/GridContinuousProcessor.java | 15 +++++++++++++++ 1 file changed, 15 insertions(+) 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 50d29d1f8de6e..5a11d617bee92 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 @@ -568,6 +568,21 @@ public void onCacheStop(GridCacheContext ctx) { if (hnd.isQuery() && F.eq(ctx.name(), hnd.cacheName())) it.remove(); } + + // Deregister local handlers. + Iterator> it2 = locInfos.entrySet().iterator(); + + while (it2.hasNext()) { + Map.Entry entry = it2.next(); + + GridContinuousHandler hnd = entry.getValue().hnd; + + if (hnd.isQuery() && F.eq(ctx.name(), hnd.cacheName())) { + unregisterHandler(entry.getKey(), hnd, true); + + it2.remove(); + } + } } /** From f1ead1191de686c5789a13091c40467a3039930b Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Thu, 7 Sep 2017 11:54:56 +0300 Subject: [PATCH 90/90] GG-12690 - Deregister local continuous query handlers on cache stop. --- .../query/continuous/CacheContinuousQueryHandler.java | 7 +++++++ .../query/continuous/CacheContinuousQueryManager.java | 2 +- .../continuous/GridContinuousProcessor.java | 11 ++++++++++- 3 files changed, 18 insertions(+), 2 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java index 0cdbec5013b15..ad11013678db5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java @@ -227,6 +227,13 @@ public void internal(boolean internal) { this.internal = internal; } + /** + * @return Internal query. + */ + public boolean internal() { + return internal; + } + /** * @param notifyExisting Notify existing. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java index 12b02f0fa9386..b3f7d6e93f7b5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java @@ -839,7 +839,7 @@ GridContinuousHandler.RegisterStatus registerListener(UUID lsnrId, * @param internal Internal flag. * @param id Listener ID. */ - void unregisterListener(boolean internal, UUID id) { + public void unregisterListener(boolean internal, UUID id) { CacheContinuousQueryListener lsnr; if (internal) { 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 5a11d617bee92..6b1d0beaf02e7 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 @@ -61,6 +61,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheProcessor; import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryHandler; +import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryManager; import org.apache.ignite.internal.processors.timeout.GridTimeoutObject; import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.future.GridFutureAdapter; @@ -578,9 +579,17 @@ public void onCacheStop(GridCacheContext ctx) { GridContinuousHandler hnd = entry.getValue().hnd; if (hnd.isQuery() && F.eq(ctx.name(), hnd.cacheName())) { + it2.remove(); + + assert hnd instanceof CacheContinuousQueryHandler : hnd; + + CacheContinuousQueryHandler hnd0 = (CacheContinuousQueryHandler)hnd; + unregisterHandler(entry.getKey(), hnd, true); - it2.remove(); + CacheContinuousQueryManager qryMgr = ctx.continuousQueries(); + + qryMgr.unregisterListener(hnd0.internal(), entry.getKey()); } } }