From 841887ee3ace1cbe988168e1027ae32500079456 Mon Sep 17 00:00:00 2001 From: nikolay_tikhonov Date: Thu, 16 Mar 2017 15:42:46 +0300 Subject: [PATCH 001/357] Merge remote-tracking branch ignite-1.7.9 into ignite-1.9.2 Conflicts: docs/RELEASE_NOTES.txt docs/community/RELEASE_NOTES.txt modules/compatibility/src/test/java/org/gridgain/grid/compatibility/tests/GridCompatibilityAbstractTest.java modules/core/src/main/resources/gridgain.properties modules/visor/src/main/scala/org/gridgain/visor/gui/tabs/sql/VisorSqlViewerTab.scala --- .../ignite/internal/visor/cache/VisorCacheClearTask.java | 1 + .../internal/processors/query/h2/IgniteH2Indexing.java | 5 ----- 2 files changed, 1 insertion(+), 5 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheClearTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheClearTask.java index b588c18efd4af..62b4762be615d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheClearTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheClearTask.java @@ -25,6 +25,7 @@ import org.apache.ignite.internal.visor.VisorJob; import org.apache.ignite.internal.visor.VisorOneNodeTask; import org.apache.ignite.lang.IgniteBiTuple; +import org.apache.ignite.lang.IgniteCallable; import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.resources.JobContextResource; 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 740b737032ffb..407819746d4b5 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 @@ -1286,11 +1286,6 @@ public GridCloseableIterator> queryLocalSql(@Nullable runs.put(run.id(), run); - GridRunningQueryInfo run = new GridRunningQueryInfo(qryIdGen.incrementAndGet(), qry, SQL, spaceName, - U.currentTimeMillis(), null, true); - - runs.put(run.id(), run); - try { ResultSet rs = executeSqlQueryWithTimer(spaceName, conn, sql, params, true, 0, cancel); From a2b4751f5eefd70a5a1aa26652c9671240125f78 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Fri, 17 Mar 2017 14:57:48 +0300 Subject: [PATCH 002/357] 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 3a496d82d56b005f2468eab5834e4a45133742b0 Mon Sep 17 00:00:00 2001 From: Sergi Vladykin Date: Sat, 18 Mar 2017 14:46:00 +0300 Subject: [PATCH 003/357] ignite-1.9 - test added --- .../query/IgniteSqlSplitterSelfTest.java | 28 +++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java index 8eae54950214b..7570184c036f7 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java @@ -152,6 +152,34 @@ public void testOffsetLimit() throws Exception { } } + /** + * + */ + public void testReplicatedOnlyTables() { + IgniteCache p = ignite(0).getOrCreateCache(cacheConfig("p", true, + Integer.class, Value.class)); + IgniteCache r = ignite(0).getOrCreateCache(cacheConfig("r", false, + Integer.class, Value.class)); + + try { + int cnt = 1000; + + for (int i = 0; i < cnt; i++) + r.put(i, new Value(i, -i)); + + // Query data from replicated table using partitioned cache. + assertEquals(cnt, p.query(new SqlFieldsQuery("select 1 from \"r\".Value")).getAll().size()); + + List> res = p.query(new SqlFieldsQuery("select count(1) from \"r\".Value")).getAll(); + assertEquals(1, res.size()); + assertEquals(cnt, ((Number)res.get(0).get(0)).intValue()); + } + finally { + p.destroy(); + r.destroy(); + } + } + /** * @throws Exception If failed. */ From c4de164392ddc114c88d5a6eba0ff0b13d32542f Mon Sep 17 00:00:00 2001 From: AMRepo Date: Mon, 20 Mar 2017 16:31:15 +0300 Subject: [PATCH 004/357] 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 84bd00b70caf2ae0b1de97e25536f6fa2e9a8c47 Mon Sep 17 00:00:00 2001 From: oleg-ostanin Date: Tue, 21 Mar 2017 15:12:35 +0300 Subject: [PATCH 005/357] IGNITE-4822 Fixed change jvm options for benchmarks --- .../config/benchmark-atomic.properties | 18 +++++++-------- .../config/benchmark-bin-identity.properties | 10 +++----- .../config/benchmark-cache-load.properties | 10 +++----- .../config/benchmark-client-mode.properties | 10 +++----- .../config/benchmark-compute.properties | 18 +++++++-------- .../config/benchmark-failover.properties | 18 ++++++--------- .../config/benchmark-full.properties | 18 ++++++--------- .../config/benchmark-multicast.properties | 10 +++----- .../benchmark-put-indexed-val.properties | 18 +++++++-------- .../benchmark-query-put-separated.properties | 10 +++----- .../config/benchmark-query.properties | 18 +++++++-------- .../config/benchmark-remote-sample.properties | 18 +++++++-------- .../config/benchmark-remote.properties | 18 ++++++--------- .../config/benchmark-sample.properties | 18 +++++++-------- .../config/benchmark-sql-dml.properties | 18 +++++++-------- .../config/benchmark-store.properties | 10 +++----- .../yardstick/config/benchmark-tx.properties | 18 +++++++-------- modules/yardstick/config/benchmark.properties | 18 ++++++--------- .../test-max-int-values-offheap.properties | 23 +++++++------------ .../test-max-int-values-onheap.properties | 23 +++++++------------ .../test-max-int-values-swap.properties | 23 +++++++------------ 21 files changed, 134 insertions(+), 211 deletions(-) diff --git a/modules/yardstick/config/benchmark-atomic.properties b/modules/yardstick/config/benchmark-atomic.properties index 722638490d866..967a603e7d02a 100644 --- a/modules/yardstick/config/benchmark-atomic.properties +++ b/modules/yardstick/config/benchmark-atomic.properties @@ -22,16 +22,14 @@ JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false" # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses. # JVM_OPTS=${JVM_OPTS}" \ -# -XX:+UseParNewGC \ -# -XX:+UseConcMarkSweepGC \ -# -XX:+UseTLAB \ -# -XX:NewSize=128m \ -# -XX:MaxNewSize=128m \ -# -XX:MaxTenuringThreshold=0 \ -# -XX:SurvivorRatio=1024 \ -# -XX:+UseCMSInitiatingOccupancyOnly \ -# -XX:CMSInitiatingOccupancyFraction=60 \ -#" +# -Xms6g \ +# -Xmx6g \ +# -Xloggc:./gc${now0}.log \ +# -XX:+PrintGCDetails \ +# -verbose:gc \ +# -XX:+UseParNewGC \ +# -XX:+UseConcMarkSweepGC \ +# " # List of default probes. # Add DStatProbe or VmStatProbe if your OS supports it (e.g. if running on Linux). diff --git a/modules/yardstick/config/benchmark-bin-identity.properties b/modules/yardstick/config/benchmark-bin-identity.properties index 6468d8f9f7142..d5d18b6d02276 100644 --- a/modules/yardstick/config/benchmark-bin-identity.properties +++ b/modules/yardstick/config/benchmark-bin-identity.properties @@ -24,17 +24,13 @@ JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false" # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses. JVM_OPTS=${JVM_OPTS}" \ +-Xms6g \ +-Xmx6g \ +-Xloggc:./gc${now0}.log \ -XX:+PrintGCDetails \ -verbose:gc \ -XX:+UseParNewGC \ -XX:+UseConcMarkSweepGC \ --XX:+UseTLAB \ --XX:NewSize=128m \ --XX:MaxNewSize=128m \ --XX:MaxTenuringThreshold=0 \ --XX:SurvivorRatio=1024 \ --XX:+UseCMSInitiatingOccupancyOnly \ --XX:CMSInitiatingOccupancyFraction=60 \ " #Ignite version diff --git a/modules/yardstick/config/benchmark-cache-load.properties b/modules/yardstick/config/benchmark-cache-load.properties index 12e07c33a0705..9c1ab093d6788 100644 --- a/modules/yardstick/config/benchmark-cache-load.properties +++ b/modules/yardstick/config/benchmark-cache-load.properties @@ -21,17 +21,13 @@ JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false" # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses. JVM_OPTS=${JVM_OPTS}" \ +-Xms6g \ +-Xmx6g \ +-Xloggc:./gc${now0}.log \ -XX:+PrintGCDetails \ -verbose:gc \ -XX:+UseParNewGC \ -XX:+UseConcMarkSweepGC \ --XX:+UseTLAB \ --XX:NewSize=128m \ --XX:MaxNewSize=128m \ --XX:MaxTenuringThreshold=0 \ --XX:SurvivorRatio=1024 \ --XX:+UseCMSInitiatingOccupancyOnly \ --XX:CMSInitiatingOccupancyFraction=60 \ " # List of default probes. # Add DStatProbe or VmStatProbe if your OS supports it (e.g. if running on Linux). diff --git a/modules/yardstick/config/benchmark-client-mode.properties b/modules/yardstick/config/benchmark-client-mode.properties index 0723fcb9cc03f..ad501f118dd79 100644 --- a/modules/yardstick/config/benchmark-client-mode.properties +++ b/modules/yardstick/config/benchmark-client-mode.properties @@ -24,17 +24,13 @@ JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false" # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses. JVM_OPTS=${JVM_OPTS}" \ +-Xms6g \ +-Xmx6g \ +-Xloggc:./gc${now0}.log \ -XX:+PrintGCDetails \ -verbose:gc \ -XX:+UseParNewGC \ -XX:+UseConcMarkSweepGC \ --XX:+UseTLAB \ --XX:NewSize=128m \ --XX:MaxNewSize=128m \ --XX:MaxTenuringThreshold=0 \ --XX:SurvivorRatio=1024 \ --XX:+UseCMSInitiatingOccupancyOnly \ --XX:CMSInitiatingOccupancyFraction=60 \ " #Ignite version ver="RELEASE-" diff --git a/modules/yardstick/config/benchmark-compute.properties b/modules/yardstick/config/benchmark-compute.properties index 66cb93030a39a..df35c54e1de88 100644 --- a/modules/yardstick/config/benchmark-compute.properties +++ b/modules/yardstick/config/benchmark-compute.properties @@ -22,16 +22,14 @@ JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false" # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses. # JVM_OPTS=${JVM_OPTS}" \ -# -XX:+UseParNewGC \ -# -XX:+UseConcMarkSweepGC \ -# -XX:+UseTLAB \ -# -XX:NewSize=128m \ -# -XX:MaxNewSize=128m \ -# -XX:MaxTenuringThreshold=0 \ -# -XX:SurvivorRatio=1024 \ -# -XX:+UseCMSInitiatingOccupancyOnly \ -# -XX:CMSInitiatingOccupancyFraction=60 \ -#" +# -Xms6g \ +# -Xmx6g \ +# -Xloggc:./gc${now0}.log \ +# -XX:+PrintGCDetails \ +# -verbose:gc \ +# -XX:+UseParNewGC \ +# -XX:+UseConcMarkSweepGC \ +# " # List of default probes. # Add DStatProbe or VmStatProbe if your OS supports it (e.g. if running on Linux). diff --git a/modules/yardstick/config/benchmark-failover.properties b/modules/yardstick/config/benchmark-failover.properties index 4c6a12a61dec4..a85d165d3f2bb 100644 --- a/modules/yardstick/config/benchmark-failover.properties +++ b/modules/yardstick/config/benchmark-failover.properties @@ -27,17 +27,13 @@ JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false -ea" # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses. JVM_OPTS=${JVM_OPTS}" \ - -XX:+PrintGCDetails \ - -verbose:gc \ - -XX:+UseParNewGC \ - -XX:+UseConcMarkSweepGC \ - -XX:+UseTLAB \ - -XX:NewSize=128m \ - -XX:MaxNewSize=128m \ - -XX:MaxTenuringThreshold=0 \ - -XX:SurvivorRatio=1024 \ - -XX:+UseCMSInitiatingOccupancyOnly \ - -XX:CMSInitiatingOccupancyFraction=60 \ +-Xms6g \ +-Xmx6g \ +-Xloggc:./gc${now0}.log \ +-XX:+PrintGCDetails \ +-verbose:gc \ +-XX:+UseParNewGC \ +-XX:+UseConcMarkSweepGC \ " #Ignite version diff --git a/modules/yardstick/config/benchmark-full.properties b/modules/yardstick/config/benchmark-full.properties index b64612ec56435..96da0a6e4d390 100644 --- a/modules/yardstick/config/benchmark-full.properties +++ b/modules/yardstick/config/benchmark-full.properties @@ -24,17 +24,13 @@ JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false" # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses. JVM_OPTS=${JVM_OPTS}" \ - -XX:+PrintGCDetails \ - -verbose:gc \ - -XX:+UseParNewGC \ - -XX:+UseConcMarkSweepGC \ - -XX:+UseTLAB \ - -XX:NewSize=128m \ - -XX:MaxNewSize=128m \ - -XX:MaxTenuringThreshold=0 \ - -XX:SurvivorRatio=1024 \ - -XX:+UseCMSInitiatingOccupancyOnly \ - -XX:CMSInitiatingOccupancyFraction=60 \ +-Xms6g \ +-Xmx6g \ +-Xloggc:./gc${now0}.log \ +-XX:+PrintGCDetails \ +-verbose:gc \ +-XX:+UseParNewGC \ +-XX:+UseConcMarkSweepGC \ " #Ignite version diff --git a/modules/yardstick/config/benchmark-multicast.properties b/modules/yardstick/config/benchmark-multicast.properties index c10d0c67da605..7f76495291d6c 100644 --- a/modules/yardstick/config/benchmark-multicast.properties +++ b/modules/yardstick/config/benchmark-multicast.properties @@ -24,17 +24,13 @@ JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false" # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses. JVM_OPTS=${JVM_OPTS}" \ +-Xms6g \ +-Xmx6g \ +-Xloggc:./gc${now0}.log \ -XX:+PrintGCDetails \ -verbose:gc \ -XX:+UseParNewGC \ -XX:+UseConcMarkSweepGC \ --XX:+UseTLAB \ --XX:NewSize=128m \ --XX:MaxNewSize=128m \ --XX:MaxTenuringThreshold=0 \ --XX:SurvivorRatio=1024 \ --XX:+UseCMSInitiatingOccupancyOnly \ --XX:CMSInitiatingOccupancyFraction=60 \ " #Ignite version diff --git a/modules/yardstick/config/benchmark-put-indexed-val.properties b/modules/yardstick/config/benchmark-put-indexed-val.properties index 0d699c5492d8a..152d72f8f89a9 100644 --- a/modules/yardstick/config/benchmark-put-indexed-val.properties +++ b/modules/yardstick/config/benchmark-put-indexed-val.properties @@ -21,16 +21,14 @@ # JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false" # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses. -JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false \ - -XX:+UseParNewGC \ - -XX:+UseConcMarkSweepGC \ - -XX:+UseTLAB \ - -XX:NewSize=128m \ - -XX:MaxNewSize=128m \ - -XX:MaxTenuringThreshold=0 \ - -XX:SurvivorRatio=1024 \ - -XX:+UseCMSInitiatingOccupancyOnly \ - -XX:CMSInitiatingOccupancyFraction=60 \ +JVM_OPTS=${JVM_OPTS}" \ +-Xms6g \ +-Xmx6g \ +-Xloggc:./gc${now0}.log \ +-XX:+PrintGCDetails \ +-verbose:gc \ +-XX:+UseParNewGC \ +-XX:+UseConcMarkSweepGC \ " # List of default probes. diff --git a/modules/yardstick/config/benchmark-query-put-separated.properties b/modules/yardstick/config/benchmark-query-put-separated.properties index 500e4faae4850..b4437bf368bc8 100644 --- a/modules/yardstick/config/benchmark-query-put-separated.properties +++ b/modules/yardstick/config/benchmark-query-put-separated.properties @@ -24,17 +24,13 @@ JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false" # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses. JVM_OPTS=${JVM_OPTS}" \ +-Xms6g \ +-Xmx6g \ +-Xloggc:./gc${now0}.log \ -XX:+PrintGCDetails \ -verbose:gc \ -XX:+UseParNewGC \ -XX:+UseConcMarkSweepGC \ --XX:+UseTLAB \ --XX:NewSize=128m \ --XX:MaxNewSize=128m \ --XX:MaxTenuringThreshold=0 \ --XX:SurvivorRatio=1024 \ --XX:+UseCMSInitiatingOccupancyOnly \ --XX:CMSInitiatingOccupancyFraction=60 \ " #Ignite version diff --git a/modules/yardstick/config/benchmark-query.properties b/modules/yardstick/config/benchmark-query.properties index af9b07e22725c..9192512b5e419 100644 --- a/modules/yardstick/config/benchmark-query.properties +++ b/modules/yardstick/config/benchmark-query.properties @@ -21,16 +21,14 @@ # JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false" # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses. -JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false \ - -XX:+UseParNewGC \ - -XX:+UseConcMarkSweepGC \ - -XX:+UseTLAB \ - -XX:NewSize=128m \ - -XX:MaxNewSize=128m \ - -XX:MaxTenuringThreshold=0 \ - -XX:SurvivorRatio=1024 \ - -XX:+UseCMSInitiatingOccupancyOnly \ - -XX:CMSInitiatingOccupancyFraction=60 \ +JVM_OPTS=${JVM_OPTS}" \ +-Xms6g \ +-Xmx6g \ +-Xloggc:./gc${now0}.log \ +-XX:+PrintGCDetails \ +-verbose:gc \ +-XX:+UseParNewGC \ +-XX:+UseConcMarkSweepGC \ " # List of default probes. diff --git a/modules/yardstick/config/benchmark-remote-sample.properties b/modules/yardstick/config/benchmark-remote-sample.properties index 5f158ac9a6cb0..7e97f43f6531b 100644 --- a/modules/yardstick/config/benchmark-remote-sample.properties +++ b/modules/yardstick/config/benchmark-remote-sample.properties @@ -22,16 +22,14 @@ JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false" # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses. # JVM_OPTS=${JVM_OPTS}" \ -# -XX:+UseParNewGC \ -# -XX:+UseConcMarkSweepGC \ -# -XX:+UseTLAB \ -# -XX:NewSize=128m \ -# -XX:MaxNewSize=128m \ -# -XX:MaxTenuringThreshold=0 \ -# -XX:SurvivorRatio=1024 \ -# -XX:+UseCMSInitiatingOccupancyOnly \ -# -XX:CMSInitiatingOccupancyFraction=60 \ -#" +# -Xms6g \ +# -Xmx6g \ +# -Xloggc:./gc${now0}.log \ +# -XX:+PrintGCDetails \ +# -verbose:gc \ +# -XX:+UseParNewGC \ +# -XX:+UseConcMarkSweepGC \ +# " # List of default probes. # Add DStatProbe or VmStatProbe if your OS supports it (e.g. if running on Linux). diff --git a/modules/yardstick/config/benchmark-remote.properties b/modules/yardstick/config/benchmark-remote.properties index 0cda79e7ef44e..4d671d3ad92ff 100644 --- a/modules/yardstick/config/benchmark-remote.properties +++ b/modules/yardstick/config/benchmark-remote.properties @@ -24,17 +24,13 @@ JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false" # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses. JVM_OPTS=${JVM_OPTS}" \ - -XX:+PrintGCDetails \ - -verbose:gc \ - -XX:+UseParNewGC \ - -XX:+UseConcMarkSweepGC \ - -XX:+UseTLAB \ - -XX:NewSize=128m \ - -XX:MaxNewSize=128m \ - -XX:MaxTenuringThreshold=0 \ - -XX:SurvivorRatio=1024 \ - -XX:+UseCMSInitiatingOccupancyOnly \ - -XX:CMSInitiatingOccupancyFraction=60 \ +-Xms6g \ +-Xmx6g \ +-Xloggc:./gc${now0}.log \ +-XX:+PrintGCDetails \ +-verbose:gc \ +-XX:+UseParNewGC \ +-XX:+UseConcMarkSweepGC \ " #Ignite version diff --git a/modules/yardstick/config/benchmark-sample.properties b/modules/yardstick/config/benchmark-sample.properties index 1932ae7b7ba71..81e9a1b52ad25 100644 --- a/modules/yardstick/config/benchmark-sample.properties +++ b/modules/yardstick/config/benchmark-sample.properties @@ -22,16 +22,14 @@ JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false" # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses. # JVM_OPTS=${JVM_OPTS}" \ -# -XX:+UseParNewGC \ -# -XX:+UseConcMarkSweepGC \ -# -XX:+UseTLAB \ -# -XX:NewSize=128m \ -# -XX:MaxNewSize=128m \ -# -XX:MaxTenuringThreshold=0 \ -# -XX:SurvivorRatio=1024 \ -# -XX:+UseCMSInitiatingOccupancyOnly \ -# -XX:CMSInitiatingOccupancyFraction=60 \ -#" +# -Xms6g \ +# -Xmx6g \ +# -Xloggc:./gc${now0}.log \ +# -XX:+PrintGCDetails \ +# -verbose:gc \ +# -XX:+UseParNewGC \ +# -XX:+UseConcMarkSweepGC \ +# " # List of default probes. # Add DStatProbe or VmStatProbe if your OS supports it (e.g. if running on Linux). diff --git a/modules/yardstick/config/benchmark-sql-dml.properties b/modules/yardstick/config/benchmark-sql-dml.properties index a8bb4537765c7..faa62c9dd5bf9 100644 --- a/modules/yardstick/config/benchmark-sql-dml.properties +++ b/modules/yardstick/config/benchmark-sql-dml.properties @@ -21,16 +21,14 @@ # JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false" # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses. -JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false \ - -XX:+UseParNewGC \ - -XX:+UseConcMarkSweepGC \ - -XX:+UseTLAB \ - -XX:NewSize=128m \ - -XX:MaxNewSize=128m \ - -XX:MaxTenuringThreshold=0 \ - -XX:SurvivorRatio=1024 \ - -XX:+UseCMSInitiatingOccupancyOnly \ - -XX:CMSInitiatingOccupancyFraction=60 \ +JVM_OPTS=${JVM_OPTS}" \ +-Xms6g \ +-Xmx6g \ +-Xloggc:./gc${now0}.log \ +-XX:+PrintGCDetails \ +-verbose:gc \ +-XX:+UseParNewGC \ +-XX:+UseConcMarkSweepGC \ " # List of default probes. diff --git a/modules/yardstick/config/benchmark-store.properties b/modules/yardstick/config/benchmark-store.properties index ea8487f749096..2f89d0e3257c6 100644 --- a/modules/yardstick/config/benchmark-store.properties +++ b/modules/yardstick/config/benchmark-store.properties @@ -26,17 +26,13 @@ JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false" # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses. JVM_OPTS=${JVM_OPTS}" \ +-Xms6g \ +-Xmx6g \ +-Xloggc:./gc${now0}.log \ -XX:+PrintGCDetails \ -verbose:gc \ -XX:+UseParNewGC \ -XX:+UseConcMarkSweepGC \ --XX:+UseTLAB \ --XX:NewSize=128m \ --XX:MaxNewSize=128m \ --XX:MaxTenuringThreshold=0 \ --XX:SurvivorRatio=1024 \ --XX:+UseCMSInitiatingOccupancyOnly \ --XX:CMSInitiatingOccupancyFraction=60 \ " #Ignite version diff --git a/modules/yardstick/config/benchmark-tx.properties b/modules/yardstick/config/benchmark-tx.properties index 4744dee0311ac..cc48c5309599d 100644 --- a/modules/yardstick/config/benchmark-tx.properties +++ b/modules/yardstick/config/benchmark-tx.properties @@ -22,16 +22,14 @@ JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false" # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses. # JVM_OPTS=${JVM_OPTS}" \ -# -XX:+UseParNewGC \ -# -XX:+UseConcMarkSweepGC \ -# -XX:+UseTLAB \ -# -XX:NewSize=128m \ -# -XX:MaxNewSize=128m \ -# -XX:MaxTenuringThreshold=0 \ -# -XX:SurvivorRatio=1024 \ -# -XX:+UseCMSInitiatingOccupancyOnly \ -# -XX:CMSInitiatingOccupancyFraction=60 \ -#" +# -Xms6g \ +# -Xmx6g \ +# -Xloggc:./gc${now0}.log \ +# -XX:+PrintGCDetails \ +# -verbose:gc \ +# -XX:+UseParNewGC \ +# -XX:+UseConcMarkSweepGC \ +# " # List of default probes, comma separated. # Add DStatProbe or VmStatProbe if your OS supports it (e.g. if running on Linux). diff --git a/modules/yardstick/config/benchmark.properties b/modules/yardstick/config/benchmark.properties index 1b5d5fe966020..cccbf2afb482b 100644 --- a/modules/yardstick/config/benchmark.properties +++ b/modules/yardstick/config/benchmark.properties @@ -24,17 +24,13 @@ JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false" # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses. JVM_OPTS=${JVM_OPTS}" \ - -XX:+PrintGCDetails \ - -verbose:gc \ - -XX:+UseParNewGC \ - -XX:+UseConcMarkSweepGC \ - -XX:+UseTLAB \ - -XX:NewSize=128m \ - -XX:MaxNewSize=128m \ - -XX:MaxTenuringThreshold=0 \ - -XX:SurvivorRatio=1024 \ - -XX:+UseCMSInitiatingOccupancyOnly \ - -XX:CMSInitiatingOccupancyFraction=60 \ +-Xms6g \ +-Xmx6g \ +-Xloggc:./gc${now0}.log \ +-XX:+PrintGCDetails \ +-verbose:gc \ +-XX:+UseParNewGC \ +-XX:+UseConcMarkSweepGC \ " #Ignite version diff --git a/modules/yardstick/config/test-max-int-values-offheap.properties b/modules/yardstick/config/test-max-int-values-offheap.properties index 42769ec1a013b..838986da1bbda 100644 --- a/modules/yardstick/config/test-max-int-values-offheap.properties +++ b/modules/yardstick/config/test-max-int-values-offheap.properties @@ -21,21 +21,14 @@ # JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false" # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses. -JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false \ - -XX:+PrintGCDetails \ - -XX:-PrintGCTimeStamps \ - -verbose:gc \ - -Xmx8g \ - -Xms8g \ - -XX:+UseParNewGC \ - -XX:+UseConcMarkSweepGC \ - -XX:+UseTLAB \ - -XX:NewSize=1g \ - -XX:MaxNewSize=1g \ - -XX:MaxTenuringThreshold=0 \ - -XX:SurvivorRatio=1024 \ - -XX:+UseCMSInitiatingOccupancyOnly \ - -XX:CMSInitiatingOccupancyFraction=60 \ +JVM_OPTS=${JVM_OPTS}" \ +-Xms8g \ +-Xmx8g \ +-Xloggc:./gc${now0}.log \ +-XX:+PrintGCDetails \ +-verbose:gc \ +-XX:+UseParNewGC \ +-XX:+UseConcMarkSweepGC \ " # List of default probes. diff --git a/modules/yardstick/config/test-max-int-values-onheap.properties b/modules/yardstick/config/test-max-int-values-onheap.properties index 0faf5f3d58c08..c83f46997a2ca 100644 --- a/modules/yardstick/config/test-max-int-values-onheap.properties +++ b/modules/yardstick/config/test-max-int-values-onheap.properties @@ -21,21 +21,14 @@ # JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false" # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses. -JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false \ - -XX:+PrintGCDetails \ - -XX:-PrintGCTimeStamps \ - -verbose:gc \ - -Xmx92g \ - -Xms32g \ - -XX:+UseParNewGC \ - -XX:+UseConcMarkSweepGC \ - -XX:+UseTLAB \ - -XX:NewSize=4g \ - -XX:MaxNewSize=4g \ - -XX:MaxTenuringThreshold=0 \ - -XX:SurvivorRatio=1024 \ - -XX:+UseCMSInitiatingOccupancyOnly \ - -XX:CMSInitiatingOccupancyFraction=60 \ +JVM_OPTS=${JVM_OPTS}" \ +-Xms92g \ +-Xmx32g \ +-Xloggc:./gc${now0}.log \ +-XX:+PrintGCDetails \ +-verbose:gc \ +-XX:+UseParNewGC \ +-XX:+UseConcMarkSweepGC \ " # List of default probes. diff --git a/modules/yardstick/config/test-max-int-values-swap.properties b/modules/yardstick/config/test-max-int-values-swap.properties index 900e8ac1b77cb..c462ada25cd46 100644 --- a/modules/yardstick/config/test-max-int-values-swap.properties +++ b/modules/yardstick/config/test-max-int-values-swap.properties @@ -21,21 +21,14 @@ # JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false" # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses. -JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false \ - -XX:+PrintGCDetails \ - -XX:-PrintGCTimeStamps \ - -verbose:gc \ - -Xmx8g \ - -Xms8g \ - -XX:+UseParNewGC \ - -XX:+UseConcMarkSweepGC \ - -XX:+UseTLAB \ - -XX:NewSize=1g \ - -XX:MaxNewSize=1g \ - -XX:MaxTenuringThreshold=0 \ - -XX:SurvivorRatio=1024 \ - -XX:+UseCMSInitiatingOccupancyOnly \ - -XX:CMSInitiatingOccupancyFraction=60 \ +JVM_OPTS=${JVM_OPTS}" \ +-Xms8g \ +-Xmx8g \ +-Xloggc:./gc${now0}.log \ +-XX:+PrintGCDetails \ +-verbose:gc \ +-XX:+UseParNewGC \ +-XX:+UseConcMarkSweepGC \ " # List of default probes. From e0c012d977b6db13dfdf2fb8347677998287c1e4 Mon Sep 17 00:00:00 2001 From: Igor Sapego Date: Tue, 21 Mar 2017 17:50:06 +0300 Subject: [PATCH 006/357] 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 8b3860f5bd9518985a8a516cd14ff24a3a615f4c Mon Sep 17 00:00:00 2001 From: Igor Sapego Date: Tue, 21 Mar 2017 17:54:51 +0300 Subject: [PATCH 007/357] 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 d7aa0a2ec2294..97222c94c635a 100644 --- a/assembly/release-fabric-base.xml +++ b/assembly/release-fabric-base.xml @@ -179,6 +179,12 @@ /platforms/cpp/docs + + + modules/platforms/cpp/bin + /platforms/cpp/bin + + bin From a54b7c6d76974f833e451aff947456ea0013226b Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Tue, 28 Mar 2017 18:49:03 +0300 Subject: [PATCH 008/357] IGNITE-4826: Fix eviction to swap if segmented index is used. This closes #1628. --- .../query/h2/opt/GridH2TreeIndex.java | 4 +-- .../IgniteSqlSegmentedIndexSelfTest.java | 30 +++++++++++++++++-- 2 files changed, 30 insertions(+), 4 deletions(-) diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java index 2873211fdf175..663d86386b8de 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java @@ -286,8 +286,8 @@ protected ConcurrentNavigableMap treeForRead(in * @param row Search row. * @return Row. */ - public GridH2Row findOne(GridSearchRowPointer row) { - int seg = threadLocalSegment(); + GridH2Row findOne(GridSearchRowPointer row) { + int seg = segmentForRow(row); return segments[seg].get(row); } diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSegmentedIndexSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSegmentedIndexSelfTest.java index f8c9dd5b64de2..800138c1f32ac 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSegmentedIndexSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSegmentedIndexSelfTest.java @@ -27,6 +27,7 @@ import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheKeyConfiguration; import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.eviction.fifo.FifoEvictionPolicy; import org.apache.ignite.cache.query.SqlFieldsQuery; import org.apache.ignite.cache.query.annotations.QuerySqlField; import org.apache.ignite.configuration.CacheConfiguration; @@ -34,6 +35,7 @@ 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.spi.swapspace.file.FileSwapSpaceSpi; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; /** @@ -63,6 +65,8 @@ public class IgniteSqlSegmentedIndexSelfTest extends GridCommonAbstractTest { cfg.setDiscoverySpi(disco); + cfg.setSwapSpaceSpi(new FileSwapSpaceSpi()); + return cfg; } @@ -77,7 +81,7 @@ public class IgniteSqlSegmentedIndexSelfTest extends GridCommonAbstractTest { * @param idxTypes Indexed types. * @return Cache configuration. */ - private static CacheConfiguration cacheConfig(String name, boolean partitioned, Class... idxTypes) { + protected CacheConfiguration cacheConfig(String name, boolean partitioned, Class... idxTypes) { return new CacheConfiguration() .setName(name) .setCacheMode(partitioned ? CacheMode.PARTITIONED : CacheMode.REPLICATED) @@ -103,6 +107,28 @@ public void testSingleNodeIndexSegmentation() throws Exception { checkLocalQueryWithSegmentedIndex(); } + /** + * Run tests on single-node grid + * @throws Exception If failed. + */ + public void testSingleNodeIndexSegmentationWithSwapEnabled() throws Exception { + startGridsMultiThreaded(1, true); + + final IgniteCache cache = ignite(0).createCache(cacheConfig("org", true, Integer.class, Organization.class) + .setOffHeapMaxMemory(-1) + .setSwapEnabled(true) + .setEvictionPolicy(new FifoEvictionPolicy(10))); + + for (int i = 0; i < 20; i++) + cache.put(i, new Organization("org-" + i)); + + String select0 = "select name from \"org\".Organization"; + + List> result = cache.query(new SqlFieldsQuery(select0)).getAll(); + + assertEquals(20, result.size()); + } + /** * Run tests on multi-node grid * @throws Exception If failed. @@ -170,7 +196,7 @@ public void checkLocalQueryWithSegmentedIndex() throws Exception { Set localOrgIds = new HashSet<>(); - for(Cache.Entry e : c2.localEntries()) + for (Cache.Entry e : c2.localEntries()) localOrgIds.add(e.getKey()); int expectedPersons = 0; From e01aee0b1bbdb8ff5583728e539df165029f682d Mon Sep 17 00:00:00 2001 From: nikolay_tikhonov Date: Fri, 31 Mar 2017 20:19:52 +0300 Subject: [PATCH 009/357] Fixed SSL issue. Signed-off-by: nikolay_tikhonov --- .../tcp/TcpCommunicationSpi.java | 35 +++++++++++++++++-- 1 file changed, 32 insertions(+), 3 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 791a7d52dbd0b..89ecc36d2ace1 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 @@ -3194,7 +3194,7 @@ else if (log.isDebugEnabled()) buf = ByteBuffer.allocate(1000); - ByteBuffer decode = null; + ByteBuffer decode = ByteBuffer.allocate(2 * buf.capacity()); buf.order(ByteOrder.nativeOrder()); @@ -3207,13 +3207,17 @@ else if (log.isDebugEnabled()) buf.flip(); - decode = sslHnd.decode(buf); + ByteBuffer decode0 = sslHnd.decode(buf); - i += decode.remaining(); + i += decode0.remaining(); + + decode = appendAndResizeIfNeeded(decode, decode0); buf.clear(); } + decode.flip(); + rcvCnt = decode.getLong(1); if (decode.limit() > 9) { @@ -3301,6 +3305,31 @@ else if (log.isDebugEnabled()) "is node stopping?) [senderNodeId=" + sndId + ", msg=" + msg + ']'); } + /** + * @param target Target buffer to append to. + * @param src Source buffer to get data. + * @return Original or expanded buffer. + */ + private ByteBuffer appendAndResizeIfNeeded(ByteBuffer target, ByteBuffer src) { + if (target.remaining() < src.remaining()) { + int newSize = Math.max(target.capacity() * 2, target.capacity() + src.remaining()); + + ByteBuffer tmp = ByteBuffer.allocate(newSize); + + tmp.order(target.order()); + + target.flip(); + + tmp.put(target); + + target = tmp; + } + + target.put(src); + + return target; + } + /** * Stops service threads to simulate node failure. * From 9d8de41b6e8ed6ec8a29179b68a05c93d9077045 Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Mon, 3 Apr 2017 14:19:56 +0700 Subject: [PATCH 010/357] IGNITE-4659 Cleanup after merge. --- .../activities-user-dialog.jade | 36 -------- .../form-field-datepicker.jade | 55 ------------- .../list-of-registered-users.jade | 58 ------------- .../ui-grid-header/ui-grid-header.jade | 27 ------ .../states/configuration/caches/affinity.jade | 82 ------------------- 5 files changed, 258 deletions(-) delete mode 100644 modules/web-console/frontend/app/components/activities-user-dialog/activities-user-dialog.jade delete mode 100644 modules/web-console/frontend/app/components/form-field-datepicker/form-field-datepicker.jade delete mode 100644 modules/web-console/frontend/app/components/list-of-registered-users/list-of-registered-users.jade delete mode 100644 modules/web-console/frontend/app/components/ui-grid-header/ui-grid-header.jade delete mode 100644 modules/web-console/frontend/app/modules/states/configuration/caches/affinity.jade diff --git a/modules/web-console/frontend/app/components/activities-user-dialog/activities-user-dialog.jade b/modules/web-console/frontend/app/components/activities-user-dialog/activities-user-dialog.jade deleted file mode 100644 index 074851ca55d66..0000000000000 --- a/modules/web-console/frontend/app/components/activities-user-dialog/activities-user-dialog.jade +++ /dev/null @@ -1,36 +0,0 @@ -//- - Licensed to the Apache Software Foundation (ASF) under one or more - contributor license agreements. See the NOTICE file distributed with - this work for additional information regarding copyright ownership. - The ASF licenses this file to You under the Apache License, Version 2.0 - (the "License"); you may not use this file except in compliance with - the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. - -.modal(tabindex='-1' role='dialog') - .modal-dialog - .modal-content - .modal-header - h4.modal-title - i.fa.fa-info-circle - | Activities: {{ ctrl.user.userName }} - .modal-body.modal-body-with-scroll(id='activities-user-dialog') - table.table.table-striped.table-bordered.table-hover(scrollable-container='#activities-user-dialog' st-table='displayedRows' st-safe-src='ctrl.data') - thead - th.text-center(st-sort='action | translate') Description - th.text-center(st-sort='action') Action - th.text-center(st-sort='amount') Visited - tbody - tr(ng-repeat='row in displayedRows') - td.text-left {{ row.action | translate }} - td.text-left {{ row.action }} - td.text-left {{ row.amount }} - .modal-footer - button.btn.btn-primary(id='confirm-btn-confirm' ng-click='$hide()') Close diff --git a/modules/web-console/frontend/app/components/form-field-datepicker/form-field-datepicker.jade b/modules/web-console/frontend/app/components/form-field-datepicker/form-field-datepicker.jade deleted file mode 100644 index 2578cf4b17189..0000000000000 --- a/modules/web-console/frontend/app/components/form-field-datepicker/form-field-datepicker.jade +++ /dev/null @@ -1,55 +0,0 @@ -//- - Licensed to the Apache Software Foundation (ASF) under one or more - contributor license agreements. See the NOTICE file distributed with - this work for additional information regarding copyright ownership. - The ASF licenses this file to You under the Apache License, Version 2.0 - (the "License"); you may not use this file except in compliance with - the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. - -mixin ignite-form-field-datepicker(label, model, name, disabled, required, placeholder, tip) - mixin form-field-input() - input.form-control( - id='{{ #{name} }}Input' - name='{{ #{name} }}' - - placeholder=placeholder - - data-ng-model=model - - data-ng-required=required && '#{required}' - data-ng-disabled=disabled && '#{disabled}' - - bs-datepicker - data-date-format='MMM yyyy' - data-start-view='1' - data-min-view='1' - data-max-date='today' - - data-container='body > .wrapper' - - tabindex='0' - - onkeydown="return false" - - data-ignite-form-panel-field='' - )&attributes(attributes.attributes) - - .ignite-form-field - +ignite-form-field__label(label, name, required) - .ignite-form-field__control - if tip - i.tipField.icon-help(bs-tooltip='' data-title=tip) - - if block - block - - .input-tip - +form-field-input(attributes=attributes) diff --git a/modules/web-console/frontend/app/components/list-of-registered-users/list-of-registered-users.jade b/modules/web-console/frontend/app/components/list-of-registered-users/list-of-registered-users.jade deleted file mode 100644 index 119591036d9da..0000000000000 --- a/modules/web-console/frontend/app/components/list-of-registered-users/list-of-registered-users.jade +++ /dev/null @@ -1,58 +0,0 @@ -//- - Licensed to the Apache Software Foundation (ASF) under one or more - contributor license agreements. See the NOTICE file distributed with - this work for additional information regarding copyright ownership. - The ASF licenses this file to You under the Apache License, Version 2.0 - (the "License"); you may not use this file except in compliance with - the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. - -include /app/helpers/jade/mixins.jade -include /app/components/form-field-datepicker/form-field-datepicker.jade - -mixin grid-settings() - i.fa.fa-bars(data-animation='am-flip-x' bs-dropdown='' aria-haspopup='true' aria-expanded='expanded' data-auto-close='1' data-trigger='click') - ul.select.dropdown-menu(role='menu') - li(ng-repeat='item in $ctrl.gridOptions.categories|filter:{selectable:true}') - a(ng-click='$ctrl.toggleColumns(item, !item.visible)') - i.fa.fa-check-square-o.pull-left(ng-if='item.visible') - i.fa.fa-square-o.pull-left(ng-if='!item.visible') - span {{::item.name}} - li.divider - li - a(ng-click='$ctrl.selectAllColumns()') Select all - li - a(ng-click='$ctrl.clearAllColumns()') Clear all - li.divider - li - a(ng-click='$hide()') Close - -.panel.panel-default - .panel-heading.ui-grid-settings - +grid-settings - label Total users: - strong {{ $ctrl.gridOptions.data.length }}    - label Showing users: - strong {{ $ctrl.gridApi.grid.getVisibleRows().length }} - sub(ng-show='users.length === $ctrl.gridApi.grid.getVisibleRows().length') all - - form.pull-right(ng-form=form novalidate) - -var form = 'admin' - - button.btn.btn-primary(ng-click='$ctrl.exportCsv()' bs-tooltip data-title='Export table to csv') Export - - .ui-grid-settings-dateperiod - +ignite-form-field-datepicker('Period:', '$ctrl.params.startDate', '"period"') - - .ui-grid-settings-filter - +ignite-form-field-text('Exclude:', '$ctrl.params.companiesExclude', '"exclude"', false, false, 'Exclude by company name...') - - .panel-collapse - .grid.ui-grid--ignite(ui-grid='$ctrl.gridOptions' ui-grid-resize-columns ui-grid-selection ui-grid-exporter ui-grid-pinning) diff --git a/modules/web-console/frontend/app/components/ui-grid-header/ui-grid-header.jade b/modules/web-console/frontend/app/components/ui-grid-header/ui-grid-header.jade deleted file mode 100644 index 7e44d94671452..0000000000000 --- a/modules/web-console/frontend/app/components/ui-grid-header/ui-grid-header.jade +++ /dev/null @@ -1,27 +0,0 @@ -//- - Licensed to the Apache Software Foundation (ASF) under one or more - contributor license agreements. See the NOTICE file distributed with - this work for additional information regarding copyright ownership. - The ASF licenses this file to You under the Apache License, Version 2.0 - (the "License"); you may not use this file except in compliance with - the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. - -.ui-grid-header.ui-grid-header--subcategories(role='rowgroup') - .ui-grid-top-panel - .ui-grid-header-viewport - .ui-grid-header-canvas - .ui-grid-header-cell-wrapper(ng-style='colContainer.headerCellWrapperStyle()') - .ui-grid-header-cell-row(role='row') - .ui-grid-header-span.ui-grid-header-cell.ui-grid-clearfix(ng-repeat='cat in grid.options.categories') - div(ng-show='(colContainer.renderedColumns|uiGridSubcategories:cat.name).length > 1') - .ui-grid-cell-contents {{ cat.name }} - .ui-grid-header-cell-row - .ui-grid-header-cell.ui-grid-clearfix(ng-repeat='col in (colContainer.renderedColumns|uiGridSubcategories:cat.name) track by col.uid' ui-grid-header-cell='' col='col' render-index='$index') diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/affinity.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/affinity.jade deleted file mode 100644 index 3c4746bbca794..0000000000000 --- a/modules/web-console/frontend/app/modules/states/configuration/caches/affinity.jade +++ /dev/null @@ -1,82 +0,0 @@ -//- - Licensed to the Apache Software Foundation (ASF) under one or more - contributor license agreements. See the NOTICE file distributed with - this work for additional information regarding copyright ownership. - The ASF licenses this file to You under the Apache License, Version 2.0 - (the "License"); you may not use this file except in compliance with - the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. - -include /app/helpers/jade/mixins.jade - --var form = 'affinity' --var model = 'backupItem' --var affModel = model + '.affinity' --var affMapModel = model + '.affinityMapper' --var rendezvousAff = affModel + '.kind === "Rendezvous"' --var fairAff = affModel + '.kind === "Fair"' --var customAff = affModel + '.kind === "Custom"' --var customAffMapper = affMapModel + '.kind === "Custom"' --var rendPartitionsRequired = rendezvousAff + ' && ' + affModel + '.Rendezvous.affinityBackupFilter' --var fairPartitionsRequired = fairAff + ' && ' + affModel + '.Fair.affinityBackupFilter' - -.panel.panel-default(ng-form=form novalidate) - .panel-heading(bs-collapse-toggle='' ng-click='ui.loadPanel("#{form}")') - ignite-form-panel-chevron - label Affinity Collocation - ignite-form-field-tooltip.tipLabel - | Collocate data with data to improve performance and scalability of your application#[br] - | #[a(href="http://apacheignite.gridgain.org/docs/affinity-collocation" target="_blank") More info] - ignite-form-revert - .panel-collapse(role='tabpanel' bs-collapse-target id=form) - .panel-body(ng-if='ui.isPanelLoaded("#{form}")') - .col-sm-6 - .settings-row - +dropdown('Function:', affModel + '.kind', '"AffinityKind"', 'true', 'Default', - '[\ - {value: "Rendezvous", label: "Rendezvous"},\ - {value: "Fair", label: "Fair"},\ - {value: "Custom", label: "Custom"},\ - {value: undefined, label: "Default"}\ - ]', - 'Key topology resolver to provide mapping from keys to nodes\ -
    \ -
  • Rendezvous - Based on Highest Random Weight algorithm
  • \ -
  • Fair - Tries to ensure that all nodes get equal number of partitions with minimum amount of reassignments between existing nodes
  • \ -
  • Custom - Custom implementation of key affinity fynction
  • \ -
  • Default - By default rendezvous affinity function with 1024 partitions is used
  • \ -
') - .panel-details(ng-if=rendezvousAff) - .details-row - +number-required('Partitions', affModel + '.Rendezvous.partitions', '"RendPartitions"', 'true', rendPartitionsRequired, '1024', '1', 'Number of partitions') - .details-row - +java-class('Backup filter', affModel + '.Rendezvous.affinityBackupFilter', '"RendAffinityBackupFilter"', 'true', 'false', - 'Backups will be selected from all nodes that pass this filter') - .details-row - +checkbox('Exclude neighbors', affModel + '.Rendezvous.excludeNeighbors', '"RendExcludeNeighbors"', - 'Exclude same - host - neighbors from being backups of each other and specified number of backups') - .panel-details(ng-if=fairAff) - .details-row - +number-required('Partitions', affModel + '.Fair.partitions', '"FairPartitions"', 'true', fairPartitionsRequired, '256', '1', 'Number of partitions') - .details-row - +java-class('Backup filter', affModel + '.Fair.affinityBackupFilter', '"FairAffinityBackupFilter"', 'true', 'false', - 'Backups will be selected from all nodes that pass this filter') - .details-row - +checkbox('Exclude neighbors', affModel + '.Fair.excludeNeighbors', '"FairExcludeNeighbors"', - 'Exclude same - host - neighbors from being backups of each other and specified number of backups') - .panel-details(ng-if=customAff) - .details-row - +java-class('Class name:', affModel + '.Custom.className', '"AffCustomClassName"', 'true', customAff, - 'Custom key affinity function implementation class name') - .settings-row - +java-class('Mapper:', model + '.affinityMapper', '"AffMapCustomClassName"', 'true', 'false', - 'Provide custom affinity key for any given key') - .col-sm-6 - +preview-xml-java(model, 'cacheAffinity') From 925ee11c2002729b1264148ee5db5700ded5a7b7 Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Tue, 4 Apr 2017 16:06:01 +0700 Subject: [PATCH 011/357] Fixed typo. (cherry picked from commit 3b84f34) --- .../app/modules/states/configuration/clusters/general.pug | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general.pug b/modules/web-console/frontend/app/modules/states/configuration/clusters/general.pug index be56a6c44d08f..dfc49d61ef4c2 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/general.pug +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general.pug @@ -49,7 +49,7 @@ include /app/helpers/jade/mixins
  • AWS S3 - AWS S3 based IP finder that automatically discover cluster nodes on Amazon EC2 cloud
  • \
  • Apache jclouds - Apache jclouds multi cloud toolkit based IP finder for cloud platforms with unstable IP addresses
  • \
  • Google cloud storage - Google Cloud Storage based IP finder that automatically discover cluster nodes on Google Compute Engine cluster
  • \ -
  • JDBC - JDBC based IP finder that use database to store node IP addres
  • \ +
  • JDBC - JDBC based IP finder that use database to store node IP address
  • \
  • Shared filesystem - Shared filesystem based IP finder that use file to store node IP address
  • \
  • Apache ZooKeeper - Apache ZooKeeper based IP finder when you use ZooKeeper to coordinate your distributed environment
  • \ ') From ce4b078c1c97cae4136c318ae38b27a50fe383cc Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Tue, 4 Apr 2017 16:14:56 +0700 Subject: [PATCH 012/357] master Updated version. (cherry picked from commit 5469626) --- .../frontend/app/modules/configuration/Version.service.js | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/web-console/frontend/app/modules/configuration/Version.service.js b/modules/web-console/frontend/app/modules/configuration/Version.service.js index 3fc719946729f..164bd20c69a0f 100644 --- a/modules/web-console/frontend/app/modules/configuration/Version.service.js +++ b/modules/web-console/frontend/app/modules/configuration/Version.service.js @@ -24,7 +24,7 @@ const numberComparator = (a, b) => a > b ? 1 : a < b ? -1 : 0; export default class IgniteVersion { /** Current product version. */ - static ignite = '1.8.0'; + static ignite = '1.9.0'; /** * Tries to parse product version from it's string representation. From b7ab27301b59bf93fc73b52fdf8e0bcf124fec1d Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Thu, 6 Apr 2017 14:43:50 +0300 Subject: [PATCH 013/357] 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 b214211eb3461746b6931c0623e086bb208e5dda Mon Sep 17 00:00:00 2001 From: Anton Vinogradov Date: Thu, 6 Apr 2017 15:00:12 +0300 Subject: [PATCH 014/357] IGNITE-4644 Value from IgniteQueue in atomic mode could be lost --- .../java/org/apache/ignite/IgniteSystemProperties.java | 5 +++++ .../datastructures/GridAtomicCacheQueueImpl.java | 7 +++++-- 2 files changed, 10 insertions(+), 2 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 f0270986fc4b5..2f88d47a4c285 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -304,6 +304,11 @@ public final class IgniteSystemProperties { */ public static final String IGNITE_ATOMIC_DEFERRED_ACK_TIMEOUT = "IGNITE_ATOMIC_DEFERRED_ACK_TIMEOUT"; + /** + * Atomic cache deferred update timeout. + */ + public static final String IGNITE_ATOMIC_CACHE_QUEUE_RETRY_TIMEOUT = "IGNITE_ATOMIC_CACHE_QUEUE_RETRY_TIMEOUT"; + /** * One phase commit deferred ack request timeout. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridAtomicCacheQueueImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridAtomicCacheQueueImpl.java index 58d3efefd7605..3cee09e0be28a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridAtomicCacheQueueImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridAtomicCacheQueueImpl.java @@ -28,12 +28,14 @@ import org.apache.ignite.internal.util.typedef.internal.U; import org.jetbrains.annotations.Nullable; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_ATOMIC_CACHE_QUEUE_RETRY_TIMEOUT; + /** * {@link org.apache.ignite.IgniteQueue} implementation using atomic cache. */ public class GridAtomicCacheQueueImpl extends GridCacheQueueAdapter { /** */ - private static final long RETRY_TIMEOUT = 3000; + private static final long RETRY_TIMEOUT = Integer.getInteger(IGNITE_ATOMIC_CACHE_QUEUE_RETRY_TIMEOUT, 10000); /** * @param queueName Queue name. @@ -94,7 +96,8 @@ public GridAtomicCacheQueueImpl(String queueName, GridCacheQueueHeader hdr, Grid return data; } - U.warn(log, "Failed to get item, will retry poll [queue=" + queueName + ", idx=" + idx + ']'); + U.warn(log, "Failed to get item due to poll timeout [queue=" + queueName + ", idx=" + idx + "]. " + + "Poll timeout can be redefined by 'IGNITE_ATOMIC_CACHE_QUEUE_RETRY_TIMEOUT' system property."); } } catch (IgniteCheckedException e) { From 014161427fb603b6df7c8ecc3c0904f5df47a21d Mon Sep 17 00:00:00 2001 From: Denis Magda Date: Mon, 13 Feb 2017 20:33:32 -0500 Subject: [PATCH 015/357] IGNITE-4159: Kubernetes IP finder. (cherry picked from commit 37c0a22) --- modules/kubernetes/DEVNOTES.txt | 63 ++++ modules/kubernetes/README.txt | 33 ++ modules/kubernetes/config/Dockerfile | 45 +++ modules/kubernetes/config/example-kube.xml | 44 +++ .../kubernetes/config/ignite-deployment.yaml | 26 ++ modules/kubernetes/config/ignite-service.yaml | 14 + modules/kubernetes/config/run.sh | 50 +++ modules/kubernetes/licenses/apache-2.0.txt | 202 +++++++++++ modules/kubernetes/pom.xml | 93 ++++++ .../TcpDiscoveryKubernetesIpFinder.java | 315 ++++++++++++++++++ .../tcp/ipfinder/kubernetes/package-info.java | 22 ++ ...cpDiscoveryKubernetesIpFinderSelfTest.java | 93 ++++++ .../tcp/ipfinder/kubernetes/package-info.java | 22 ++ .../testsuites/IgniteKubernetesTestSuite.java | 41 +++ pom.xml | 1 + 15 files changed, 1064 insertions(+) create mode 100644 modules/kubernetes/DEVNOTES.txt create mode 100644 modules/kubernetes/README.txt create mode 100644 modules/kubernetes/config/Dockerfile create mode 100644 modules/kubernetes/config/example-kube.xml create mode 100644 modules/kubernetes/config/ignite-deployment.yaml create mode 100644 modules/kubernetes/config/ignite-service.yaml create mode 100644 modules/kubernetes/config/run.sh create mode 100644 modules/kubernetes/licenses/apache-2.0.txt create mode 100644 modules/kubernetes/pom.xml create mode 100644 modules/kubernetes/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/kubernetes/TcpDiscoveryKubernetesIpFinder.java create mode 100644 modules/kubernetes/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/kubernetes/package-info.java create mode 100644 modules/kubernetes/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/kubernetes/TcpDiscoveryKubernetesIpFinderSelfTest.java create mode 100644 modules/kubernetes/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/kubernetes/package-info.java create mode 100644 modules/kubernetes/src/test/java/org/apache/ignite/testsuites/IgniteKubernetesTestSuite.java diff --git a/modules/kubernetes/DEVNOTES.txt b/modules/kubernetes/DEVNOTES.txt new file mode 100644 index 0000000000000..b2a8173587bf7 --- /dev/null +++ b/modules/kubernetes/DEVNOTES.txt @@ -0,0 +1,63 @@ +Building and testing Kubernetes module +========================================= + +The instructions provide a guidance on how to build and test Ignite Kubernetes IP finder in Kubernetes environment. + +To test the IP finder you have to build the whole Apache Ignite project, package the binary as a Docker image and +feed the image to your kubernetes environment. + +Building Apache Ignite +========================= + +Use the command below to assemble an Apache Ignite binary: + mvn clean package -Prelease -Dignite.edition=fabric-lgpl -DskipTests + +Note, if you alter the build instruction somehow make sure to update the files under 'config' folder if needed. + +Installing Docker and Minikube +============================== + +Install Docker and Minikube for testing purpose in your development environment. + +Once this is done, make sure that Minikube sees Docker images registered locally: + eval $(minikube docker-env) + +Start Minikube: + minikube start --vm-driver=xhyve + +Assembling Apache Ignite Docker Image +===================================== + +Create a folder for all the files to be placed in the Docker image and copy the following there: + - Apache Ignite binary in a zip form built at the step above. + - Dockerfile from `ignite-kubernetes/config/Dockerfile`. + - Ignite configuration with enabled Kubernetes IP finder from `ignite-kubernetes/config/example-kube.xml`. + - The executable file that will start an Ignite node process from `ignite-kubernetes/config/run.sh` + +Go to the folder and execute a command below to prepare the image: + docker build -t ignite-kube:v1 . + +Creating containerized Ignite pods and Ignite lookup service +============================================================ + +Start the Kubernetes service that is used for IP addresses lookup. Use `ignite-kubernetes/config/ignite-service.yaml`: + kubectl create -f {path_to}/ignite-service.yaml + +Create and deploy Ignite pods using `ignite-kubernetes/config/ignite-deployment.yaml` configuration: + kubectl create -f {path_to}/ignite-deployment.yaml + +Make sure that the pods were deployed and running properly: + kubectl get pod + kubectl logs {pod name} + +Increase or decrease number of Ignite pods checking that Kubernetes IP finder works as expected: + kubectl scale --replicas=4 -f {path_to}/ignite-deployment.yaml + +Docker Image Redeployment +========================= + +If you need to redeploy the docker image after it gets updated and you prefer not to change the image version then +delete a current Kubernetes Ignite deployment (don't delete the service): + kubectl delete deployment ignite-cluster + +After that you are free to build and deploy an updated docker image using the same commands as listed above. diff --git a/modules/kubernetes/README.txt b/modules/kubernetes/README.txt new file mode 100644 index 0000000000000..a9a5a092ce2bb --- /dev/null +++ b/modules/kubernetes/README.txt @@ -0,0 +1,33 @@ +Apache Ignite Kubernetes Module +------------------------ + +Apache Ignite Kubernetes module provides a TCP Discovery IP Finder that uses a dedicated Kubernetes service +for IP addresses lookup of Apache Ignite pods containerized by Kubernetes. + +To enable Kubernetes module when starting a standalone node, move 'optional/ignite-kubernetes' folder to +'libs' folder before running 'ignite.{sh|bat}' script. The content of the module folder will +be added to classpath in this case. + +Importing Kubernetes Module In Maven Project +------------------------------------- + +If you are using Maven to manage dependencies of your project, you can add Kubernetes module +dependency like this (replace '${ignite.version}' with actual Ignite version you are +interested in): + + + ... + + ... + + org.apache.ignite + ignite-kubernetes + ${ignite.version} + + ... + + ... + diff --git a/modules/kubernetes/config/Dockerfile b/modules/kubernetes/config/Dockerfile new file mode 100644 index 0000000000000..4e08ce80e2147 --- /dev/null +++ b/modules/kubernetes/config/Dockerfile @@ -0,0 +1,45 @@ +# Use Java 8 image as default one. +FROM java:8 + +# Set Apache Ignite version. +ENV IGNITE_VERSION 2.0.0-SNAPSHOT + +# Set IGNITE_HOME variable. +ENV IGNITE_HOME /opt/ignite/apache-ignite-fabric-lgpl-${IGNITE_VERSION}-bin + +# Setting a path to an Apache Ignite configuration file. Used by run.sh script below. +ENV CONFIG_URI ${IGNITE_HOME}/config/example-kube.xml + +# Make sure kubernetes lib is copied to 'libs' folder. +ENV OPTION_LIBS ignite-kubernetes + +# Disabling quiet mode. +ENV IGNITE_QUIET=false + +# Install or update needed tools. +RUN apt-get update && apt-get install -y --no-install-recommends unzip + +# Creating and setting a working directory for following commands. +WORKDIR /opt/ignite + +# Copying local Apache Ignite build to the docker image. +COPY ./apache-ignite-fabric-lgpl-${IGNITE_VERSION}-bin.zip apache-ignite-fabric-lgpl-${IGNITE_VERSION}-bin.zip + +# Unpacking the build. +RUN unzip apache-ignite-fabric-lgpl-${IGNITE_VERSION}-bin.zip +RUN rm apache-ignite-fabric-lgpl-${IGNITE_VERSION}-bin.zip + +# Copying the executable file and setting permissions. +COPY ./run.sh $IGNITE_HOME/ +RUN chmod +x $IGNITE_HOME/run.sh + +# Copying the configuration. +COPY ./example-kube.xml $IGNITE_HOME/config + +# Starting an Apache Ignite node. +CMD $IGNITE_HOME/run.sh + +# Exposing the ports. +EXPOSE 11211 47100 47500 49112 + + diff --git a/modules/kubernetes/config/example-kube.xml b/modules/kubernetes/config/example-kube.xml new file mode 100644 index 0000000000000..bc04463f71a88 --- /dev/null +++ b/modules/kubernetes/config/example-kube.xml @@ -0,0 +1,44 @@ + + + + + + + + + + + + + + + + + + + + diff --git a/modules/kubernetes/config/ignite-deployment.yaml b/modules/kubernetes/config/ignite-deployment.yaml new file mode 100644 index 0000000000000..ed5c102a666a6 --- /dev/null +++ b/modules/kubernetes/config/ignite-deployment.yaml @@ -0,0 +1,26 @@ +# An example of a Kubernetes configuration for Ignite pods deployment. +apiVersion: extensions/v1beta1 +kind: Deployment +metadata: + # Custom Ignite cluster's name. + name: ignite-cluster +spec: + # Number of nodes to be started by Kubernetes initially. + replicas: 2 + template: + metadata: + labels: + # Must be equal to Ignite's Kubernetes service name. + app: ignite + spec: + containers: + # Custom Ignite node's pod name. + - name: ignite-node + # Custom Ignite Docker image name. + image: ignite-kube:v1 + ports: + # Ports to open. + - containerPort: 11211 # REST port number. + - containerPort: 47100 # communication SPI port number. + - containerPort: 47500 # discovery SPI port number. + - containerPort: 49112 # JMX port number. diff --git a/modules/kubernetes/config/ignite-service.yaml b/modules/kubernetes/config/ignite-service.yaml new file mode 100644 index 0000000000000..07b751624cbbf --- /dev/null +++ b/modules/kubernetes/config/ignite-service.yaml @@ -0,0 +1,14 @@ +# An example of a Kubernetes configuration for Ignite lookup service deployment. +apiVersion: v1 +kind: Service +metadata: + # Name of Ignite Service used by Kubernetes IP finder for IP addresses lookup. + # The name must be equal to TcpDiscoveryKubernetesIpFinder.setServiceName parameter. + name: ignite +spec: + clusterIP: None + ports: + - port: 9042 # some custom port (optional). + selector: + # Must be equal to the label set for Ignite pods. + app: ignite diff --git a/modules/kubernetes/config/run.sh b/modules/kubernetes/config/run.sh new file mode 100644 index 0000000000000..dbf287118aa41 --- /dev/null +++ b/modules/kubernetes/config/run.sh @@ -0,0 +1,50 @@ +#!/bin/bash +# +# 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. +# + +if [ ! -z "$OPTION_LIBS" ]; then + IFS=, LIBS_LIST=("$OPTION_LIBS") + + for lib in ${LIBS_LIST[@]}; do + cp -r $IGNITE_HOME/libs/optional/"$lib"/* \ + $IGNITE_HOME/libs/ + done +fi + +if [ ! -z "$EXTERNAL_LIBS" ]; then + IFS=, LIBS_LIST=("$EXTERNAL_LIBS") + + for lib in ${LIBS_LIST[@]}; do + echo $lib >> temp + done + + wget -i temp -P $IGNITE_HOME/libs + + rm temp +fi + +QUIET="" + +if [ "$IGNITE_QUIET" = "false" ]; then + QUIET="-v" +fi + +if [ -z $CONFIG_URI ]; then + $IGNITE_HOME/bin/ignite.sh $QUIET +else + $IGNITE_HOME/bin/ignite.sh $QUIET $CONFIG_URI +fi diff --git a/modules/kubernetes/licenses/apache-2.0.txt b/modules/kubernetes/licenses/apache-2.0.txt new file mode 100644 index 0000000000000..d645695673349 --- /dev/null +++ b/modules/kubernetes/licenses/apache-2.0.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed 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. diff --git a/modules/kubernetes/pom.xml b/modules/kubernetes/pom.xml new file mode 100644 index 0000000000000..5d4e5f0c8b509 --- /dev/null +++ b/modules/kubernetes/pom.xml @@ -0,0 +1,93 @@ + + + + + + + 4.0.0 + + + org.apache.ignite + ignite-parent + 1 + ../../parent + + + ignite-kubernetes + 2.0.0-SNAPSHOT + http://ignite.apache.org + + + + org.apache.ignite + ignite-core + ${project.version} + + + + org.codehaus.jackson + jackson-core-asl + ${jackson.version} + + + + org.codehaus.jackson + jackson-mapper-asl + ${jackson.version} + + + + org.apache.ignite + ignite-core + ${project.version} + test-jar + test + + + + org.springframework + spring-beans + ${spring.version} + test + + + + log4j + log4j + test + + + + org.springframework + spring-context + ${spring.version} + test + + + + org.springframework + spring-core + ${spring.version} + test + + + + + diff --git a/modules/kubernetes/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/kubernetes/TcpDiscoveryKubernetesIpFinder.java b/modules/kubernetes/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/kubernetes/TcpDiscoveryKubernetesIpFinder.java new file mode 100644 index 0000000000000..a5bd24f0affcc --- /dev/null +++ b/modules/kubernetes/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/kubernetes/TcpDiscoveryKubernetesIpFinder.java @@ -0,0 +1,315 @@ +/* + * 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.ipfinder.kubernetes; + + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.net.URL; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.security.SecureRandom; +import java.security.cert.X509Certificate; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicBoolean; +import javax.net.ssl.HostnameVerifier; +import javax.net.ssl.HttpsURLConnection; +import javax.net.ssl.SSLContext; +import javax.net.ssl.SSLSession; +import javax.net.ssl.TrustManager; +import javax.net.ssl.X509TrustManager; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.resources.LoggerResource; +import org.apache.ignite.spi.IgniteSpiException; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinderAdapter; +import org.codehaus.jackson.annotate.JsonIgnoreProperties; +import org.codehaus.jackson.map.ObjectMapper; + +/** + * IP finder for automatic lookup of Ignite nodes running in Kubernetes environment. All Ignite nodes have to deployed + * as Kubernetes pods in order to be discovered. An application that uses Ignite client nodes as a gateway to the + * cluster is required to be containerized as well. Applications and Ignite nodes running outside of Kubernetes will + * not be able to reach the containerized counterparts. + *

    + * The implementation is based on a dedicated Kubernetes service that has to be created and should be deployed prior + * Ignite nodes startup. The service will maintain a list of all endpoints (internal IP addresses) of all containerized + * Ignite pods running so far. The name of the service must be equal to {@link #setServiceName(String)} which is + * `ignite` by default. + *

    + * As for Ignite pods, it's recommended to label them in such a way that the service will use the label in its selector + * configuration excluding endpoints of irrelevant Kubernetes pods running in parallel. + *

    + * The IP finder, in its turn, will call this service to retrieve Ignite pods IP addresses. The port will be + * either the one that is set with {@link TcpDiscoverySpi#setLocalPort(int)} or {@link TcpDiscoverySpi#DFLT_PORT}. + * Make sure that all Ignite pods occupy a similar discovery port, otherwise they will not be able to discover each + * other using this IP finder. + *

    Optional configuration

    + *
      + *
    • The Kubernetes service name for IP addresses lookup (see {@link #setServiceName(String)})
    • + *
    • The Kubernetes service namespace for IP addresses lookup (see {@link #setNamespace(String)}
    • + *
    • The host name of the Kubernetes API server (see {@link #setMasterUrl(String)})
    • + *
    • Path to the service token (see {@link #setAccountToken(String)}
    • + *
    + *

    + * Both {@link #registerAddresses(Collection)} and {@link #unregisterAddresses(Collection)} have no effect. + *

    + * Note, this IP finder is only workable when it used in Kubernetes environment. + * Choose another implementation of {@link org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder} for local + * or home network tests. + */ +public class TcpDiscoveryKubernetesIpFinder extends TcpDiscoveryIpFinderAdapter { + /** Grid logger. */ + @LoggerResource + private IgniteLogger log; + + /** Init routine guard. */ + private final AtomicBoolean initGuard = new AtomicBoolean(); + + /** Init routine latch. */ + private final CountDownLatch initLatch = new CountDownLatch(1); + + /** Trust manager. */ + private TrustManager[] trustAll = new TrustManager[] { + new X509TrustManager() { + public void checkServerTrusted(X509Certificate[] certs, String authType) {} + public void checkClientTrusted(X509Certificate[] certs, String authType) {} + public X509Certificate[] getAcceptedIssuers() { return null; } + } + }; + + /** Host verifier. */ + private HostnameVerifier trustAllHosts = new HostnameVerifier() { + public boolean verify(String hostname, SSLSession session) { + return true; + } + }; + + /** Ignite's Kubernetes Service name. */ + private String serviceName = "ignite"; + + /** Ignite Pod setNamespace name. */ + private String namespace = "default"; + + /** Kubernetes API server URL in a string form. */ + private String master = "https://kubernetes.default.svc.cluster.local:443"; + + /** Account token location. */ + private String accountToken = "/var/run/secrets/kubernetes.io/serviceaccount/token"; + + /** Kubernetes API server URL. */ + private URL url; + + /** SSL context */ + private SSLContext ctx; + + /** + * Creates an instance of Kubernetes IP finder. + */ + public TcpDiscoveryKubernetesIpFinder() { + setShared(true); + } + + /** {@inheritDoc} */ + @Override public Collection getRegisteredAddresses() throws IgniteSpiException { + init(); + + Collection addrs = new ArrayList<>(); + + try { + System.out.println("Getting Apache Ignite endpoints from: " + url); + + HttpsURLConnection conn = (HttpsURLConnection)url.openConnection(); + + conn.setHostnameVerifier(trustAllHosts); + + conn.setSSLSocketFactory(ctx.getSocketFactory()); + conn.addRequestProperty("Authorization", "Bearer " + serviceAccountToken(accountToken)); + + // Sending the request and processing a response. + ObjectMapper mapper = new ObjectMapper(); + + Endpoints endpoints = mapper.readValue(conn.getInputStream(), Endpoints.class); + + if (endpoints != null) { + if (endpoints.subsets != null && !endpoints.subsets.isEmpty()) { + for (Subset subset : endpoints.subsets) { + + if (subset.addresses != null && !subset.addresses.isEmpty()) { + for (Address address : subset.addresses) { + addrs.add(new InetSocketAddress(address.ip, 0)); + + System.out.println("Added an address to the list: " + address.ip); + } + } + } + } + } + } + catch (Exception e) { + throw new IgniteSpiException("Failed to retrieve Ignite pods IP addresses.", e); + } + + return addrs; + } + + /** {@inheritDoc} */ + @Override public void registerAddresses(Collection addrs) throws IgniteSpiException { + // No-op + } + + /** {@inheritDoc} */ + @Override public void unregisterAddresses(Collection addrs) throws IgniteSpiException { + // No-op + } + + /** + * Sets the name of Kubernetes service for Ignite pods' IP addresses lookup. The name of the service must be equal + * to the name set in service's Kubernetes configuration. If this parameter is not changed then the name of the + * service has to be set to 'ignite' in the corresponding Kubernetes configuration. + * + * @param service Kubernetes service name for IP addresses lookup. If it's not set then 'ignite' is used by default. + */ + public void setServiceName(String service) { + this.serviceName = service; + } + + /** + * Sets the namespace the Kubernetes service belongs to. By default, it's supposed that the service is running under + * Kubernetes `default` namespace. + * + * @param namespace The Kubernetes service namespace for IP addresses lookup. + */ + public void setNamespace(String namespace) { + this.namespace = namespace; + } + + /** + * Sets the host name of the Kubernetes API server. By default the following host name is used: + * 'https://kubernetes.default.svc.cluster.local:443'. + * + * @param master The host name of the Kubernetes API server. + */ + public void setMasterUrl(String master) { + this.master = master; + } + + /** + * Specifies the path to the service token file. By default the following account token is used: + * '/var/run/secrets/kubernetes.io/serviceaccount/token'. + * + * @param accountToken The path to the service token file. + */ + public void setAccountToken(String accountToken) { + this.accountToken = accountToken; + } + + /** + * Kubernetes IP finder initialization. + * + * @throws IgniteSpiException In case of error. + */ + private void init() throws IgniteSpiException { + if (initGuard.compareAndSet(false, true)) { + + if (serviceName == null || serviceName.isEmpty() || + namespace == null || namespace.isEmpty() || + master == null || master.isEmpty() || + accountToken == null || accountToken.isEmpty()) { + throw new IgniteSpiException( + "One or more configuration parameters are invalid [setServiceName=" + + serviceName + ", setNamespace=" + namespace + ", setMasterUrl=" + + master + ", setAccountToken=" + accountToken + "]"); + } + + try { + // Preparing the URL and SSL context to be used for connection purposes. + String path = String.format("/api/v1/namespaces/%s/endpoints/%s", namespace, serviceName); + + url = new URL(master + path); + + ctx = SSLContext.getInstance("SSL"); + + ctx.init(null, trustAll, new SecureRandom()); + } + catch (Exception e) { + throw new IgniteSpiException("Failed to connect to Ignite's Kubernetes Service.", e); + } + finally { + initLatch.countDown(); + } + } + else { + try { + U.await(initLatch); + } + catch (IgniteInterruptedCheckedException e) { + throw new IgniteSpiException("Thread has been interrupted.", e); + } + + if (url == null || ctx == null) + throw new IgniteSpiException("IP finder has not been initialized properly."); + } + } + + /** + * Reads content of the service account token file. + * + * @param file The path to the service account token. + * @return Service account token. + */ + private String serviceAccountToken(String file) { + try { + return new String(Files.readAllBytes(Paths.get(file))); + } catch (IOException e) { + throw new IgniteSpiException("Failed to load services account token [setAccountToken= " + file + "]", e); + } + } + + /** + * Object used by Jackson for processing of Kubernetes lookup service's response. + */ + @JsonIgnoreProperties(ignoreUnknown = true) + private static class Address { + /** */ + public String ip; + } + + /** + * Object used by Jackson for processing of Kubernetes lookup service's response. + */ + @JsonIgnoreProperties(ignoreUnknown = true) + private static class Subset { + /** */ + public List

    addresses; + } + + /** + * Object used by Jackson for processing of Kubernetes lookup service's response. + */ + @JsonIgnoreProperties(ignoreUnknown = true) + private static class Endpoints { + /** */ + public List subsets; + } +} diff --git a/modules/kubernetes/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/kubernetes/package-info.java b/modules/kubernetes/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/kubernetes/package-info.java new file mode 100644 index 0000000000000..a572cb2d8cbc1 --- /dev/null +++ b/modules/kubernetes/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/kubernetes/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * + * Contains Kubernetes IP finder implementation. + */ +package org.apache.ignite.spi.discovery.tcp.ipfinder.kubernetes; \ No newline at end of file diff --git a/modules/kubernetes/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/kubernetes/TcpDiscoveryKubernetesIpFinderSelfTest.java b/modules/kubernetes/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/kubernetes/TcpDiscoveryKubernetesIpFinderSelfTest.java new file mode 100644 index 0000000000000..fd3e2a35e432c --- /dev/null +++ b/modules/kubernetes/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/kubernetes/TcpDiscoveryKubernetesIpFinderSelfTest.java @@ -0,0 +1,93 @@ +/* + * 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.ipfinder.kubernetes; + +import org.apache.ignite.spi.IgniteSpiException; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinderAbstractSelfTest; + +/** + * TcpDiscoveryKubernetesIpFinder test. + */ +public class TcpDiscoveryKubernetesIpFinderSelfTest extends + TcpDiscoveryIpFinderAbstractSelfTest { + /** + * Constructor. + * + * @throws Exception If any error occurs. + */ + public TcpDiscoveryKubernetesIpFinderSelfTest() throws Exception { + // No-op. + } + + @Override protected void beforeTest() throws Exception { + // No-op. + } + + /* {@inheritDoc} */ + @Override protected TcpDiscoveryKubernetesIpFinder ipFinder() throws Exception { + // No-op. + return null; + } + + /* {@inheritDoc} */ + @Override public void testIpFinder() throws Exception { + TcpDiscoveryKubernetesIpFinder ipFinder = new TcpDiscoveryKubernetesIpFinder(); + + ipFinder.setAccountToken(null); + + try { + ipFinder.getRegisteredAddresses(); + } + catch (IgniteSpiException e) { + assertTrue(e.getMessage().startsWith("One or more configuration parameters are invalid")); + } + + ipFinder = new TcpDiscoveryKubernetesIpFinder(); + + ipFinder.setMasterUrl(null); + + try { + ipFinder.getRegisteredAddresses(); + } + catch (IgniteSpiException e) { + assertTrue(e.getMessage().startsWith("One or more configuration parameters are invalid")); + } + + ipFinder = new TcpDiscoveryKubernetesIpFinder(); + + ipFinder.setNamespace(null); + + try { + ipFinder.getRegisteredAddresses(); + } + catch (IgniteSpiException e) { + assertTrue(e.getMessage().startsWith("One or more configuration parameters are invalid")); + } + + ipFinder = new TcpDiscoveryKubernetesIpFinder(); + + ipFinder.setServiceName(""); + + try { + ipFinder.getRegisteredAddresses(); + } + catch (IgniteSpiException e) { + assertTrue(e.getMessage().startsWith("One or more configuration parameters are invalid")); + } + } +} \ No newline at end of file diff --git a/modules/kubernetes/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/kubernetes/package-info.java b/modules/kubernetes/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/kubernetes/package-info.java new file mode 100644 index 0000000000000..83ab56ff75163 --- /dev/null +++ b/modules/kubernetes/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/kubernetes/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * + * Contains Kubernetes IP finder internal tests. + */ +package org.apache.ignite.spi.discovery.tcp.ipfinder.kubernetes; \ No newline at end of file diff --git a/modules/kubernetes/src/test/java/org/apache/ignite/testsuites/IgniteKubernetesTestSuite.java b/modules/kubernetes/src/test/java/org/apache/ignite/testsuites/IgniteKubernetesTestSuite.java new file mode 100644 index 0000000000000..540657e8b2601 --- /dev/null +++ b/modules/kubernetes/src/test/java/org/apache/ignite/testsuites/IgniteKubernetesTestSuite.java @@ -0,0 +1,41 @@ +/* + * 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.spi.discovery.tcp.ipfinder.kubernetes.TcpDiscoveryKubernetesIpFinderSelfTest; +import org.apache.ignite.testframework.IgniteTestSuite; + +/** + * Ignite Kubernetes integration test. + */ +public class IgniteKubernetesTestSuite extends TestSuite { + /** + * @return Test suite. + * @throws Exception Thrown in case of the failure. + */ + public static TestSuite suite() throws Exception { + TestSuite suite = new IgniteTestSuite("Kubernetes Integration Test Suite"); + + // Cloud Nodes IP finder. + suite.addTestSuite(TcpDiscoveryKubernetesIpFinderSelfTest.class); + + return suite; + } +} \ No newline at end of file diff --git a/pom.xml b/pom.xml index d83bb1b969b9e..ef8573d3d5390 100644 --- a/pom.xml +++ b/pom.xml @@ -88,6 +88,7 @@ modules/web/ignite-websphere-test modules/cassandra modules/flink + modules/kubernetes From 1db238402f11c67d2b28bfb7ff47955415f00c25 Mon Sep 17 00:00:00 2001 From: Denis Magda Date: Wed, 15 Feb 2017 23:37:26 -0500 Subject: [PATCH 016/357] IGNITE-4159: fixing logging (cherry picked from commit 06908d2) (cherry picked from commit fa27ee3) --- .../kubernetes/TcpDiscoveryKubernetesIpFinder.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/modules/kubernetes/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/kubernetes/TcpDiscoveryKubernetesIpFinder.java b/modules/kubernetes/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/kubernetes/TcpDiscoveryKubernetesIpFinder.java index a5bd24f0affcc..53b6df6a0b711 100644 --- a/modules/kubernetes/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/kubernetes/TcpDiscoveryKubernetesIpFinder.java +++ b/modules/kubernetes/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/kubernetes/TcpDiscoveryKubernetesIpFinder.java @@ -52,7 +52,7 @@ * cluster is required to be containerized as well. Applications and Ignite nodes running outside of Kubernetes will * not be able to reach the containerized counterparts. *

    - * The implementation is based on a dedicated Kubernetes service that has to be created and should be deployed prior + * The implementation is based on a distinct Kubernetes service that has to be created and should be deployed prior * Ignite nodes startup. The service will maintain a list of all endpoints (internal IP addresses) of all containerized * Ignite pods running so far. The name of the service must be equal to {@link #setServiceName(String)} which is * `ignite` by default. @@ -137,7 +137,8 @@ public TcpDiscoveryKubernetesIpFinder() { Collection addrs = new ArrayList<>(); try { - System.out.println("Getting Apache Ignite endpoints from: " + url); + if (log.isDebugEnabled()) + log.debug("Getting Apache Ignite endpoints from: " + url); HttpsURLConnection conn = (HttpsURLConnection)url.openConnection(); @@ -159,7 +160,8 @@ public TcpDiscoveryKubernetesIpFinder() { for (Address address : subset.addresses) { addrs.add(new InetSocketAddress(address.ip, 0)); - System.out.println("Added an address to the list: " + address.ip); + if (log.isDebugEnabled()) + log.debug("Added an address to the list: " + address.ip); } } } From 5dfe16f7e91374008b9f6dfbb899364f5b2e1164 Mon Sep 17 00:00:00 2001 From: Denis Magda Date: Tue, 14 Feb 2017 01:03:30 -0500 Subject: [PATCH 017/357] IGNITE-4159: using logger instead of system.out.println (cherry picked from commit b9bf77c) --- modules/kubernetes/config/example-kube.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/kubernetes/config/example-kube.xml b/modules/kubernetes/config/example-kube.xml index bc04463f71a88..11309d8933b14 100644 --- a/modules/kubernetes/config/example-kube.xml +++ b/modules/kubernetes/config/example-kube.xml @@ -41,4 +41,4 @@ - + \ No newline at end of file From 6e596d1ef426b66abd866d011a8f5cf5c5c25124 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Thu, 6 Apr 2017 14:43:50 +0300 Subject: [PATCH 018/357] IGNITE-4832: Prevent service deployment on client by default when configuration is provided on startup. This closes #1748. (cherry picked from commit b7ab273) --- .../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 adfbc116aa293..d0b2733473333 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 019/357] 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 05f3c747921aed6838804d2f5f2c8d2bd7985337 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Wed, 5 Apr 2017 15:01:02 +0300 Subject: [PATCH 020/357] IGNITE-4917: Fixed failure when accessing BinaryObjectBuilder field value serialized with OptimizedMarshaller . This closes #1736. (cherry picked from commit 443ac9a) --- .../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 b8e3d1b6f972b6d30657b7c85c8b34506dc29b88 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Wed, 5 Apr 2017 15:01:02 +0300 Subject: [PATCH 021/357] IGNITE-4917: Fixed failure when accessing BinaryObjectBuilder field value serialized with OptimizedMarshaller . This closes #1736. (cherry picked from commit 443ac9a) --- .../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 a0392605f39c23fcd20c98d852c4cab749cd059b Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Thu, 6 Apr 2017 14:43:50 +0300 Subject: [PATCH 022/357] IGNITE-4832: Prevent service deployment on client by default when configuration is provided on startup. This closes #1748. (cherry picked from commit b7ab273) --- .../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 adfbc116aa293..d0b2733473333 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 220db882b466c03eadd148b3a19a0bf70d82d4a6 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Mon, 10 Apr 2017 10:28:15 +0300 Subject: [PATCH 023/357] IGNITE-2466 - Use current NIO back pressure mechanism to limit received messages. Mark them process only when backups acknowledged. --- .../managers/communication/GridIoManager.java | 12 +- .../dht/atomic/GridDhtAtomicCache.java | 23 ++- .../util/nio/GridNioBackPressureControl.java | 39 ++++- .../util/nio/GridNioMessageTracker.java | 7 + ...acheAtomicPrimarySyncBackPressureTest.java | 151 ++++++++++++++++++ .../testsuites/IgniteCacheTestSuite4.java | 3 + 6 files changed, 220 insertions(+), 15 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAtomicPrimarySyncBackPressureTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java index bb3add4833ada..dbd5db6730175 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 @@ -738,7 +738,7 @@ private void processP2PMessage( Runnable c = new Runnable() { @Override public void run() { try { - threadProcessingMessage(true); + threadProcessingMessage(true, msgC); GridMessageListener lsnr = listenerGet0(msg.topic()); @@ -752,7 +752,7 @@ private void processP2PMessage( invokeListener(msg.policy(), lsnr, nodeId, obj); } finally { - threadProcessingMessage(false); + threadProcessingMessage(false, null); msgC.run(); } @@ -787,12 +787,12 @@ private void processRegularMessage( Runnable c = new Runnable() { @Override public void run() { try { - threadProcessingMessage(true); + threadProcessingMessage(true, msgC); processRegularMessage0(msg, nodeId); } finally { - threadProcessingMessage(false); + threadProcessingMessage(false, null); msgC.run(); } @@ -1148,12 +1148,12 @@ else if (log.isDebugEnabled()) { Runnable c = new Runnable() { @Override public void run() { try { - threadProcessingMessage(true); + threadProcessingMessage(true, msgC); unwindMessageSet(msgSet0, lsnr); } finally { - threadProcessingMessage(false); + threadProcessingMessage(false, null); } } }; 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 463fc57d2115f..047be8751d7b1 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 @@ -84,6 +84,8 @@ import org.apache.ignite.internal.util.GridUnsafe; import org.apache.ignite.internal.util.future.GridEmbeddedFuture; import org.apache.ignite.internal.util.future.GridFinishedFuture; +import org.apache.ignite.internal.util.nio.GridNioBackPressureControl; +import org.apache.ignite.internal.util.nio.GridNioMessageTracker; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.typedef.C1; import org.apache.ignite.internal.util.typedef.CI1; @@ -92,14 +94,15 @@ import org.apache.ignite.internal.util.typedef.CX1; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.P1; -import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteBiTuple; import org.apache.ignite.lang.IgniteClosure; +import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.lang.IgniteOutClosure; +import org.apache.ignite.lang.IgniteRunnable; import org.apache.ignite.plugin.security.SecurityPermission; import org.apache.ignite.transactions.TransactionIsolation; import org.jetbrains.annotations.Nullable; @@ -110,6 +113,7 @@ import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.CLOCK; import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_ASYNC; import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.PRIMARY_SYNC; import static org.apache.ignite.internal.processors.cache.GridCacheOperation.DELETE; import static org.apache.ignite.internal.processors.cache.GridCacheOperation.TRANSFORM; import static org.apache.ignite.internal.processors.cache.GridCacheOperation.UPDATE; @@ -1904,8 +1908,23 @@ private void updateAllAsyncInternal0( if (req.writeSynchronizationMode() != FULL_ASYNC) req.cleanup(!node.isLocal()); - if (dhtFut != null) + if (dhtFut != null) { + if (req.writeSynchronizationMode() == PRIMARY_SYNC && !dhtFut.isDone()) { + final IgniteRunnable tracker = GridNioBackPressureControl.threadTracker(); + + if (tracker != null && tracker instanceof GridNioMessageTracker) { + ((GridNioMessageTracker)tracker).onMessageReceived(); + + dhtFut.listen(new IgniteInClosure>() { + @Override public void apply(IgniteInternalFuture fut) { + ((GridNioMessageTracker)tracker).onMessageProcessed(); + } + }); + } + } + ctx.mvcc().addAtomicFuture(dhtFut.version(), dhtFut); + } } else // Should remap all keys. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioBackPressureControl.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioBackPressureControl.java index 96a1ab347d402..37d985f7ff100 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioBackPressureControl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioBackPressureControl.java @@ -17,14 +17,17 @@ package org.apache.ignite.internal.util.nio; +import org.apache.ignite.lang.IgniteRunnable; +import org.jetbrains.annotations.Nullable; + /** * Utility class that allows to ignore back-pressure control for threads that are processing messages. */ public class GridNioBackPressureControl { /** Thread local flag indicating that thread is processing message. */ - private static ThreadLocal threadProcMsg = new ThreadLocal() { - @Override protected Boolean initialValue() { - return Boolean.FALSE; + private static ThreadLocal threadProcMsg = new ThreadLocal() { + @Override protected Holder initialValue() { + return new Holder(); } }; @@ -32,13 +35,35 @@ public class GridNioBackPressureControl { * @return Flag indicating whether current thread is processing message. */ public static boolean threadProcessingMessage() { - return threadProcMsg.get(); + return threadProcMsg.get().procMsg; } /** * @param processing Flag indicating whether current thread is processing message. + * @param tracker Thread local back pressure tracker of messages, associated with one connection. + */ + public static void threadProcessingMessage(boolean processing, @Nullable IgniteRunnable tracker) { + Holder holder = threadProcMsg.get(); + + holder.procMsg = processing; + holder.tracker = tracker; + } + + /** + * @return Thread local back pressure tracker of messages, associated with one connection. */ - public static void threadProcessingMessage(boolean processing) { - threadProcMsg.set(processing); + @Nullable public static IgniteRunnable threadTracker() { + return threadProcMsg.get().tracker; + } + + /** + * + */ + private static class Holder { + /** Process message. */ + private boolean procMsg; + + /** Tracker. */ + private IgniteRunnable tracker; } -} \ No newline at end of file +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioMessageTracker.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioMessageTracker.java index e02c7ca7fd6ee..f05ee0c60d045 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioMessageTracker.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioMessageTracker.java @@ -92,6 +92,13 @@ public GridNioMessageTracker(GridNioSession ses, int msgQueueLimit) { } } + /** + * + */ + public void onMessageProcessed() { + run(); + } + /** */ public void onMessageReceived() { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAtomicPrimarySyncBackPressureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAtomicPrimarySyncBackPressureTest.java new file mode 100644 index 0000000000000..49e3e5ca07a15 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAtomicPrimarySyncBackPressureTest.java @@ -0,0 +1,151 @@ +/* + * 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 org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteException; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheMemoryMode; +import org.apache.ignite.cache.CacheWriteSynchronizationMode; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteKernal; +import org.apache.ignite.internal.managers.communication.GridIoMessage; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicDeferredUpdateResponse; +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.communication.tcp.TcpCommunicationSpi; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Checks that back-pressure control restricts uncontrolled growing + * of backup message queue. This means, if queue too big - any reads + * will be stopped until received acks from backup nodes. + */ +public class CacheAtomicPrimarySyncBackPressureTest extends GridCommonAbstractTest { + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + CacheConfiguration ccfg = new CacheConfiguration("cache"); + + ccfg.setBackups(1); + + ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.PRIMARY_SYNC); + ccfg.setMemoryMode(CacheMemoryMode.ONHEAP_TIERED); + ccfg.setAtomicityMode(CacheAtomicityMode.ATOMIC); + + TestCommunicationSpi spi = new TestCommunicationSpi(); + + spi.setMessageQueueLimit(100); + + cfg.setCommunicationSpi(spi); + cfg.setClientMode(gridName.contains("client")); + cfg.setCacheConfiguration(ccfg); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** + * @throws Exception If failed. + */ + public void testClientPut() throws Exception { + Ignite srv1 = startGrid("server1"); + Ignite srv2 = startGrid("server2"); + + final Ignite client = startGrid("client"); + + checkBackPressure(client, srv1, srv2); + } + + /** + * @throws Exception If failed. + */ + public void testServerPut() throws Exception { + Ignite srv1 = startGrid("server1"); + Ignite srv2 = startGrid("server2"); + + final Ignite client = startGrid("server3"); + + checkBackPressure(client, srv1, srv2); + } + + /** + * @param client Producer node. + * @throws InterruptedException If failed. + */ + private void checkBackPressure(Ignite client, final Ignite srv1, final Ignite srv2) throws Exception { + final IgniteCache cache = client.cache("cache"); + + awaitPartitionMapExchange(); + + for (int i = 0; i < 10000; i++) { + cache.put(i, String.valueOf(i)); + + if (i % 100 == 0) { + int size1 = futuresNum(srv1); + int size2 = futuresNum(srv2); + + assert size1 < 150 : size1; + assert size2 < 150 : size2; + } + } + } + + /** + * @param ignite Ignite. + * @return Size of the backup queue. + */ + private int futuresNum(Ignite ignite) { + return ((IgniteKernal)ignite).context().cache().context().mvcc().atomicFutures().size(); + } + + /** + * Delays backup update acks. + */ + private static class TestCommunicationSpi extends TcpCommunicationSpi { + /** {@inheritDoc} */ + @Override public void sendMessage(ClusterNode node, Message msg, + IgniteInClosure ackC) throws IgniteSpiException { + if (((GridIoMessage)msg).message() instanceof GridDhtAtomicDeferredUpdateResponse) + sleep(100); + + super.sendMessage(node, msg, ackC); + } + } + + /** + * @param millis Millis. + */ + private static void sleep(long millis) { + try { + Thread.sleep(millis); + } + catch (InterruptedException e) { + throw new IgniteSpiException(e); + } + } +} 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 5a09a1c5ae797..9fcf31a322f2a 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 @@ -94,6 +94,7 @@ import org.apache.ignite.internal.processors.cache.IgniteSystemCacheOnClientTest; import org.apache.ignite.internal.processors.cache.MarshallerCacheJobRunNodeRestartTest; import org.apache.ignite.internal.processors.cache.distributed.CacheAffinityEarlyTest; +import org.apache.ignite.internal.processors.cache.distributed.CacheAtomicPrimarySyncBackPressureTest; import org.apache.ignite.internal.processors.cache.distributed.CacheGetFutureHangsSelfTest; import org.apache.ignite.internal.processors.cache.distributed.CacheNoValueClassOnServerNodeTest; import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheCreatePutMultiNodeSelfTest; @@ -334,6 +335,8 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(IgniteCacheNearOnlyTxTest.class); + suite.addTestSuite(CacheAtomicPrimarySyncBackPressureTest.class); + return suite; } } \ No newline at end of file From 3d616799efb472227b3b313516e6b40729654631 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Mon, 10 Apr 2017 10:36:11 +0300 Subject: [PATCH 024/357] IGNITE-2466 - Use current NIO back pressure mechanism to limit received messages. Mark them process only when backups acknowledged. (backport from 1.9.2) (cherry picked from commit 220db882b466c03eadd148b3a19a0bf70d82d4a6) --- .../managers/communication/GridIoManager.java | 12 +- .../dht/atomic/GridDhtAtomicCache.java | 23 ++- .../util/nio/GridNioBackPressureControl.java | 39 ++++- .../util/nio/GridNioMessageTracker.java | 7 + ...acheAtomicPrimarySyncBackPressureTest.java | 151 ++++++++++++++++++ .../testsuites/IgniteCacheTestSuite4.java | 6 + 6 files changed, 223 insertions(+), 15 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAtomicPrimarySyncBackPressureTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java index 2eda4b715ee01..a1315dbd96fb7 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 @@ -731,7 +731,7 @@ private void processP2PMessage( Runnable c = new Runnable() { @Override public void run() { try { - threadProcessingMessage(true); + threadProcessingMessage(true, msgC); GridMessageListener lsnr = listenerGet0(msg.topic()); @@ -745,7 +745,7 @@ private void processP2PMessage( invokeListener(msg.policy(), lsnr, nodeId, obj); } finally { - threadProcessingMessage(false); + threadProcessingMessage(false, null); msgC.run(); } @@ -780,12 +780,12 @@ private void processRegularMessage( Runnable c = new Runnable() { @Override public void run() { try { - threadProcessingMessage(true); + threadProcessingMessage(true, msgC); processRegularMessage0(msg, nodeId); } finally { - threadProcessingMessage(false); + threadProcessingMessage(false, null); msgC.run(); } @@ -1141,12 +1141,12 @@ else if (log.isDebugEnabled()) { Runnable c = new Runnable() { @Override public void run() { try { - threadProcessingMessage(true); + threadProcessingMessage(true, msgC); unwindMessageSet(msgSet0, lsnr); } finally { - threadProcessingMessage(false); + threadProcessingMessage(false, null); } } }; 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 c91f881b60378..7d360766dd0fc 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 @@ -84,6 +84,8 @@ import org.apache.ignite.internal.util.GridUnsafe; import org.apache.ignite.internal.util.future.GridEmbeddedFuture; import org.apache.ignite.internal.util.future.GridFinishedFuture; +import org.apache.ignite.internal.util.nio.GridNioBackPressureControl; +import org.apache.ignite.internal.util.nio.GridNioMessageTracker; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.typedef.C1; import org.apache.ignite.internal.util.typedef.CI1; @@ -92,14 +94,15 @@ import org.apache.ignite.internal.util.typedef.CX1; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.P1; -import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteBiTuple; import org.apache.ignite.lang.IgniteClosure; +import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.lang.IgniteOutClosure; +import org.apache.ignite.lang.IgniteRunnable; import org.apache.ignite.plugin.security.SecurityPermission; import org.apache.ignite.transactions.TransactionIsolation; import org.jetbrains.annotations.Nullable; @@ -110,6 +113,7 @@ import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.CLOCK; import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_ASYNC; import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.PRIMARY_SYNC; import static org.apache.ignite.internal.processors.cache.GridCacheOperation.DELETE; import static org.apache.ignite.internal.processors.cache.GridCacheOperation.TRANSFORM; import static org.apache.ignite.internal.processors.cache.GridCacheOperation.UPDATE; @@ -1908,8 +1912,23 @@ private void updateAllAsyncInternal0( if (req.writeSynchronizationMode() != FULL_ASYNC) req.cleanup(!node.isLocal()); - if (dhtFut != null) + if (dhtFut != null) { + if (req.writeSynchronizationMode() == PRIMARY_SYNC && !dhtFut.isDone()) { + final IgniteRunnable tracker = GridNioBackPressureControl.threadTracker(); + + if (tracker != null && tracker instanceof GridNioMessageTracker) { + ((GridNioMessageTracker)tracker).onMessageReceived(); + + dhtFut.listen(new IgniteInClosure>() { + @Override public void apply(IgniteInternalFuture fut) { + ((GridNioMessageTracker)tracker).onMessageProcessed(); + } + }); + } + } + ctx.mvcc().addAtomicFuture(dhtFut.version(), dhtFut); + } } else // Should remap all keys. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioBackPressureControl.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioBackPressureControl.java index 96a1ab347d402..37d985f7ff100 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioBackPressureControl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioBackPressureControl.java @@ -17,14 +17,17 @@ package org.apache.ignite.internal.util.nio; +import org.apache.ignite.lang.IgniteRunnable; +import org.jetbrains.annotations.Nullable; + /** * Utility class that allows to ignore back-pressure control for threads that are processing messages. */ public class GridNioBackPressureControl { /** Thread local flag indicating that thread is processing message. */ - private static ThreadLocal threadProcMsg = new ThreadLocal() { - @Override protected Boolean initialValue() { - return Boolean.FALSE; + private static ThreadLocal threadProcMsg = new ThreadLocal() { + @Override protected Holder initialValue() { + return new Holder(); } }; @@ -32,13 +35,35 @@ public class GridNioBackPressureControl { * @return Flag indicating whether current thread is processing message. */ public static boolean threadProcessingMessage() { - return threadProcMsg.get(); + return threadProcMsg.get().procMsg; } /** * @param processing Flag indicating whether current thread is processing message. + * @param tracker Thread local back pressure tracker of messages, associated with one connection. + */ + public static void threadProcessingMessage(boolean processing, @Nullable IgniteRunnable tracker) { + Holder holder = threadProcMsg.get(); + + holder.procMsg = processing; + holder.tracker = tracker; + } + + /** + * @return Thread local back pressure tracker of messages, associated with one connection. */ - public static void threadProcessingMessage(boolean processing) { - threadProcMsg.set(processing); + @Nullable public static IgniteRunnable threadTracker() { + return threadProcMsg.get().tracker; + } + + /** + * + */ + private static class Holder { + /** Process message. */ + private boolean procMsg; + + /** Tracker. */ + private IgniteRunnable tracker; } -} \ No newline at end of file +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioMessageTracker.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioMessageTracker.java index e02c7ca7fd6ee..f05ee0c60d045 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioMessageTracker.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioMessageTracker.java @@ -92,6 +92,13 @@ public GridNioMessageTracker(GridNioSession ses, int msgQueueLimit) { } } + /** + * + */ + public void onMessageProcessed() { + run(); + } + /** */ public void onMessageReceived() { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAtomicPrimarySyncBackPressureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAtomicPrimarySyncBackPressureTest.java new file mode 100644 index 0000000000000..49e3e5ca07a15 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAtomicPrimarySyncBackPressureTest.java @@ -0,0 +1,151 @@ +/* + * 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 org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteException; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheMemoryMode; +import org.apache.ignite.cache.CacheWriteSynchronizationMode; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteKernal; +import org.apache.ignite.internal.managers.communication.GridIoMessage; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicDeferredUpdateResponse; +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.communication.tcp.TcpCommunicationSpi; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Checks that back-pressure control restricts uncontrolled growing + * of backup message queue. This means, if queue too big - any reads + * will be stopped until received acks from backup nodes. + */ +public class CacheAtomicPrimarySyncBackPressureTest extends GridCommonAbstractTest { + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + CacheConfiguration ccfg = new CacheConfiguration("cache"); + + ccfg.setBackups(1); + + ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.PRIMARY_SYNC); + ccfg.setMemoryMode(CacheMemoryMode.ONHEAP_TIERED); + ccfg.setAtomicityMode(CacheAtomicityMode.ATOMIC); + + TestCommunicationSpi spi = new TestCommunicationSpi(); + + spi.setMessageQueueLimit(100); + + cfg.setCommunicationSpi(spi); + cfg.setClientMode(gridName.contains("client")); + cfg.setCacheConfiguration(ccfg); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** + * @throws Exception If failed. + */ + public void testClientPut() throws Exception { + Ignite srv1 = startGrid("server1"); + Ignite srv2 = startGrid("server2"); + + final Ignite client = startGrid("client"); + + checkBackPressure(client, srv1, srv2); + } + + /** + * @throws Exception If failed. + */ + public void testServerPut() throws Exception { + Ignite srv1 = startGrid("server1"); + Ignite srv2 = startGrid("server2"); + + final Ignite client = startGrid("server3"); + + checkBackPressure(client, srv1, srv2); + } + + /** + * @param client Producer node. + * @throws InterruptedException If failed. + */ + private void checkBackPressure(Ignite client, final Ignite srv1, final Ignite srv2) throws Exception { + final IgniteCache cache = client.cache("cache"); + + awaitPartitionMapExchange(); + + for (int i = 0; i < 10000; i++) { + cache.put(i, String.valueOf(i)); + + if (i % 100 == 0) { + int size1 = futuresNum(srv1); + int size2 = futuresNum(srv2); + + assert size1 < 150 : size1; + assert size2 < 150 : size2; + } + } + } + + /** + * @param ignite Ignite. + * @return Size of the backup queue. + */ + private int futuresNum(Ignite ignite) { + return ((IgniteKernal)ignite).context().cache().context().mvcc().atomicFutures().size(); + } + + /** + * Delays backup update acks. + */ + private static class TestCommunicationSpi extends TcpCommunicationSpi { + /** {@inheritDoc} */ + @Override public void sendMessage(ClusterNode node, Message msg, + IgniteInClosure ackC) throws IgniteSpiException { + if (((GridIoMessage)msg).message() instanceof GridDhtAtomicDeferredUpdateResponse) + sleep(100); + + super.sendMessage(node, msg, ackC); + } + } + + /** + * @param millis Millis. + */ + private static void sleep(long millis) { + try { + Thread.sleep(millis); + } + catch (InterruptedException e) { + throw new IgniteSpiException(e); + } + } +} 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..9fcf31a322f2a 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 @@ -94,6 +94,7 @@ import org.apache.ignite.internal.processors.cache.IgniteSystemCacheOnClientTest; import org.apache.ignite.internal.processors.cache.MarshallerCacheJobRunNodeRestartTest; import org.apache.ignite.internal.processors.cache.distributed.CacheAffinityEarlyTest; +import org.apache.ignite.internal.processors.cache.distributed.CacheAtomicPrimarySyncBackPressureTest; import org.apache.ignite.internal.processors.cache.distributed.CacheGetFutureHangsSelfTest; import org.apache.ignite.internal.processors.cache.distributed.CacheNoValueClassOnServerNodeTest; import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheCreatePutMultiNodeSelfTest; @@ -108,6 +109,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCacheMultiTxLockSelfTest; import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCrossCacheTxSelfTest; import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheNearTxPreloadSelfTest; +import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheNearOnlyTxTest; import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheNearReadCommittedTest; import org.apache.ignite.internal.processors.cache.distributed.replicated.GridReplicatedTxPreloadTest; import org.apache.ignite.internal.processors.cache.integration.IgniteCacheAtomicLoadAllTest; @@ -331,6 +333,10 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(MarshallerCacheJobRunNodeRestartTest.class); + suite.addTestSuite(IgniteCacheNearOnlyTxTest.class); + + suite.addTestSuite(CacheAtomicPrimarySyncBackPressureTest.class); + return suite; } } \ No newline at end of file From 2a88a7a7581465ff0a6f8733550e6d42d7f71a6c Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Mon, 10 Apr 2017 10:54:37 +0300 Subject: [PATCH 025/357] IGNITE-4667 - Throw exception on starting client cache when indexed types cannot be loaded --- .../cache/CacheAffinitySharedManager.java | 52 +-- .../GridCachePartitionExchangeManager.java | 4 +- .../processors/cache/GridCacheProcessor.java | 42 ++- .../dht/atomic/GridDhtAtomicCache.java | 3 +- .../GridDhtPartitionsExchangeFuture.java | 2 +- ...niteClientCacheInitializationFailTest.java | 346 ++++++++++++++++++ .../testsuites/IgniteCacheTestSuite4.java | 2 + 7 files changed, 415 insertions(+), 36 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java index 2642d16c56fc5..84372febb2c91 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java @@ -379,38 +379,48 @@ else if (req.start() && !req.clientStartOnly()) { Integer cacheId = CU.cacheId(req.cacheName()); if (req.start()) { - cctx.cache().prepareCacheStart(req, fut.topologyVersion()); + try { + cctx.cache().prepareCacheStart(req, fut.topologyVersion()); - if (fut.isCacheAdded(cacheId, fut.topologyVersion())) { - if (fut.discoCache().cacheAffinityNodes(req.cacheName()).isEmpty()) - U.quietAndWarn(log, "No server nodes found for cache client: " + req.cacheName()); - } + if (fut.isCacheAdded(cacheId, fut.topologyVersion())) { + if (fut.discoCache().cacheAffinityNodes(req.cacheName()).isEmpty()) + U.quietAndWarn(log, "No server nodes found for cache client: " + req.cacheName()); + } - if (!crd || !lateAffAssign) { - GridCacheContext cacheCtx = cctx.cacheContext(cacheId); + if (!crd || !lateAffAssign) { + GridCacheContext cacheCtx = cctx.cacheContext(cacheId); - if (cacheCtx != null && !cacheCtx.isLocal()) { - boolean clientCacheStarted = - req.clientStartOnly() && req.initiatingNodeId().equals(cctx.localNodeId()); + if (cacheCtx != null && !cacheCtx.isLocal()) { + boolean clientCacheStarted = + req.clientStartOnly() && req.initiatingNodeId().equals(cctx.localNodeId()); - if (clientCacheStarted) - initAffinity(cacheCtx.affinity().affinityCache(), fut, lateAffAssign); - else if (!req.clientStartOnly()) { - assert fut.topologyVersion().equals(cacheCtx.startTopologyVersion()); + if (clientCacheStarted) + initAffinity(cacheCtx.affinity().affinityCache(), fut, lateAffAssign); + else if (!req.clientStartOnly()) { + assert fut.topologyVersion().equals(cacheCtx.startTopologyVersion()); - GridAffinityAssignmentCache aff = cacheCtx.affinity().affinityCache(); + GridAffinityAssignmentCache aff = cacheCtx.affinity().affinityCache(); - assert aff.lastVersion().equals(AffinityTopologyVersion.NONE) : aff.lastVersion(); + assert aff.lastVersion().equals(AffinityTopologyVersion.NONE) : aff.lastVersion(); - List> assignment = aff.calculate(fut.topologyVersion(), - fut.discoveryEvent(), fut.discoCache()); + List> assignment = aff.calculate(fut.topologyVersion(), + fut.discoveryEvent(), fut.discoCache()); - aff.initialize(fut.topologyVersion(), assignment); + aff.initialize(fut.topologyVersion(), assignment); + } } } + else + initStartedCacheOnCoordinator(fut, cacheId); + } + catch (IgniteCheckedException | RuntimeException e) { + U.error(log, "Failed to initialize cache. Will try to rollback cache start routine. " + + "[cacheName=" + req.cacheName() + ']', e); + + cctx.cache().forceCloseCache(fut.topologyVersion(), req, e); + + throw e; } - else - initStartedCacheOnCoordinator(fut, cacheId); } else if (req.stop() || req.close()) { cctx.cache().blockGateway(req); 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 16b07694ed5c0..679395727123a 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 @@ -241,11 +241,11 @@ public class GridCachePartitionExchangeManager extends GridCacheSharedMana fut = affinityReadyFuture(req.cacheFutureTopologyVersion()); if (fut == null || fut.isDone()) - cctx.cache().completeStartFuture(req); + cctx.cache().completeStartFuture(req, null); else { fut.listen(new CI1>() { @Override public void apply(IgniteInternalFuture fut) { - cctx.cache().completeStartFuture(req); + cctx.cache().completeStartFuture(req, null); } }); } 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 9487589aa4c05..5740ad17f0617 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 @@ -1612,6 +1612,8 @@ public void prepareCacheStart(DynamicCacheChangeRequest req, AffinityTopologyVer assert req.start() : req; assert req.cacheType() != null : req; + String cacheName = maskNull(req.cacheName()); + prepareCacheStart( req.startCacheConfiguration(), req.nearCacheConfiguration(), @@ -1622,7 +1624,7 @@ public void prepareCacheStart(DynamicCacheChangeRequest req, AffinityTopologyVer topVer ); - DynamicCacheDescriptor desc = registeredCaches.get(maskNull(req.cacheName())); + DynamicCacheDescriptor desc = registeredCaches.get(cacheName); if (desc != null) desc.onStart(); @@ -1761,8 +1763,8 @@ private void stopGateway(DynamicCacheChangeRequest req) { /** * @param req Stop request. */ - private void prepareCacheStop(DynamicCacheChangeRequest req) { - assert req.stop() || req.close() : req; + private void prepareCacheStop(DynamicCacheChangeRequest req, boolean forceClose) { + assert req.stop() || req.close() || forceClose : req; GridCacheAdapter cache = caches.remove(maskNull(req.cacheName())); @@ -1779,18 +1781,35 @@ private void prepareCacheStop(DynamicCacheChangeRequest req) { } } + /** + * Closes cache even if it's not fully initialized (e.g. fail on cache init stage). + * + * @param topVer Completed topology version. + * @param req Change request. + * @param err Error. + */ + void forceCloseCache( + AffinityTopologyVersion topVer, + DynamicCacheChangeRequest req, + Throwable err + ) { + onExchangeDone(topVer, Collections.singleton(req), err, true); + } + /** * Callback invoked when first exchange future for dynamic cache is completed. * * @param topVer Completed topology version. * @param reqs Change requests. * @param err Error. + * @param forceClose Close cache despite flags in requests. */ @SuppressWarnings("unchecked") public void onExchangeDone( AffinityTopologyVersion topVer, Collection reqs, - Throwable err + Throwable err, + boolean forceClose ) { for (GridCacheAdapter cache : caches.values()) { GridCacheContext cacheCtx = cache.context(); @@ -1805,16 +1824,16 @@ public void onExchangeDone( } } - if (!F.isEmpty(reqs) && err == null) { + if (!F.isEmpty(reqs) && (err == null || forceClose)) { for (DynamicCacheChangeRequest req : reqs) { String masked = maskNull(req.cacheName()); if (req.stop()) { stopGateway(req); - prepareCacheStop(req); + prepareCacheStop(req, forceClose); } - else if (req.close() && req.initiatingNodeId().equals(ctx.localNodeId())) { + else if (req.close() && req.initiatingNodeId().equals(ctx.localNodeId()) || forceClose) { IgniteCacheProxy proxy = jCacheProxies.remove(masked); if (proxy != null) { @@ -1827,20 +1846,21 @@ else if (req.close() && req.initiatingNodeId().equals(ctx.localNodeId())) { else { proxy.context().gate().onStopped(); - prepareCacheStop(req); + prepareCacheStop(req, forceClose); } } } - completeStartFuture(req); + completeStartFuture(req, err); } } } /** * @param req Request to complete future for. + * @param err Error to be passed to futures. */ - public void completeStartFuture(DynamicCacheChangeRequest req) { + public void completeStartFuture(DynamicCacheChangeRequest req, @Nullable Throwable err) { DynamicCacheStartFuture fut = (DynamicCacheStartFuture)pendingFuts.get(maskNull(req.cacheName())); assert req.deploymentId() != null; @@ -1848,7 +1868,7 @@ public void completeStartFuture(DynamicCacheChangeRequest req) { if (fut != null && fut.deploymentId().equals(req.deploymentId()) && F.eq(req.initiatingNodeId(), ctx.localNodeId())) - fut.onDone(); + fut.onDone(err); } /** 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 7d360766dd0fc..5b9b1416d8f84 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 @@ -457,7 +457,8 @@ else if (res.error() != null) { /** {@inheritDoc} */ @Override public void stop() { - deferredUpdateMsgSnd.stop(); + if (deferredUpdateMsgSnd != null) + deferredUpdateMsgSnd.stop(); } /** 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 f11f868cf0e33..9aa0755269b9f 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 @@ -1069,7 +1069,7 @@ private void sendPartitions(ClusterNode oldestNode) { cctx.exchange().onExchangeDone(this, err); - cctx.cache().onExchangeDone(exchId.topologyVersion(), reqs, err); + cctx.cache().onExchangeDone(exchId.topologyVersion(), reqs, err, false); if (super.onDone(res, err) && realExchange) { if (log.isDebugEnabled()) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java new file mode 100644 index 0000000000000..63dd57527183b --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java @@ -0,0 +1,346 @@ +/* + * 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.sql.PreparedStatement; +import java.sql.SQLException; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.Callable; +import javax.cache.Cache; +import javax.cache.CacheException; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.query.QueryCursor; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.cache.query.SqlQuery; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.configuration.NearCacheConfiguration; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.query.GridQueryCancel; +import org.apache.ignite.internal.processors.query.GridQueryFieldsResult; +import org.apache.ignite.internal.processors.query.GridQueryIndexing; +import org.apache.ignite.internal.processors.query.GridQueryProcessor; +import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor; +import org.apache.ignite.internal.processors.query.GridRunningQueryInfo; +import org.apache.ignite.internal.util.GridSpinBusyLock; +import org.apache.ignite.internal.util.lang.GridCloseableIterator; +import org.apache.ignite.lang.IgniteBiTuple; +import org.apache.ignite.lang.IgniteFuture; +import org.apache.ignite.spi.indexing.IndexingQueryFilter; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.jetbrains.annotations.Nullable; + +/** + * Test checks whether cache initialization error on client side + * doesn't causes hangs and doesn't impact other caches. + */ +public class IgniteClientCacheInitializationFailTest extends GridCommonAbstractTest { + /** Failed cache name. */ + private static final String CACHE_NAME = "cache"; + + /** Atomic cache name. */ + private static final String ATOMIC_CACHE_NAME = "atomic-cache"; + + /** Tx cache name. */ + private static final String TX_CACHE_NAME = "tx-cache"; + + /** Near atomic cache name. */ + private static final String NEAR_ATOMIC_CACHE_NAME = "near-atomic-cache"; + + /** Near tx cache name. */ + private static final String NEAR_TX_CACHE_NAME = "near-tx-cache"; + + /** Failed caches. */ + private static final Set FAILED_CACHES; + + static { + Set set = new HashSet<>(); + + set.add(ATOMIC_CACHE_NAME); + set.add(TX_CACHE_NAME); + set.add(NEAR_ATOMIC_CACHE_NAME); + set.add(NEAR_TX_CACHE_NAME); + + FAILED_CACHES = Collections.unmodifiableSet(set); + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + startGrid("server"); + startGrid("client"); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + if (gridName.contains("server")) { + CacheConfiguration ccfg1 = new CacheConfiguration<>(); + + ccfg1.setIndexedTypes(Integer.class, String.class); + ccfg1.setName(ATOMIC_CACHE_NAME); + ccfg1.setAtomicityMode(CacheAtomicityMode.ATOMIC); + + CacheConfiguration ccfg2 = new CacheConfiguration<>(); + + ccfg2.setIndexedTypes(Integer.class, String.class); + ccfg2.setName(TX_CACHE_NAME); + ccfg2.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL); + + cfg.setCacheConfiguration(ccfg1, ccfg2); + } + else { + GridQueryProcessor.idxCls = FailedIndexing.class; + + cfg.setClientMode(true); + } + + return cfg; + } + + /** + * @throws Exception If failed. + */ + public void testAtomicCacheInitialization() throws Exception { + checkCacheInitialization(ATOMIC_CACHE_NAME); + } + + /** + * @throws Exception If failed. + */ + public void testTransactionalCacheInitialization() throws Exception { + checkCacheInitialization(TX_CACHE_NAME); + } + + /** + * @throws Exception If failed. + */ + public void testAtomicNearCacheInitialization() throws Exception { + checkCacheInitialization(NEAR_ATOMIC_CACHE_NAME); + } + + /** + * @throws Exception If failed. + */ + public void testTransactionalNearCacheInitialization() throws Exception { + checkCacheInitialization(NEAR_TX_CACHE_NAME); + } + + /** + * @throws Exception If failed. + */ + private void checkCacheInitialization(final String cacheName) throws Exception { + Ignite client = grid("client"); + + checkFailedCache(client, cacheName); + + checkFineCache(client, CACHE_NAME + 1); + + assertNull(client.cache(cacheName)); + assertNull(client.getOrCreateCache(cacheName)); + + checkFineCache(client, CACHE_NAME + 2); + } + + /** + * @param client Client. + * @param cacheName Cache name. + */ + private void checkFineCache(Ignite client, String cacheName) { + IgniteCache cache = client.getOrCreateCache(cacheName); + + cache.put(1, "1"); + + assertEquals("1", cache.get(1)); + } + + /** + * @param client Client. + */ + @SuppressWarnings("ThrowableNotThrown") + private void checkFailedCache(final Ignite client, final String cacheName) { + GridTestUtils.assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + IgniteCache cache; + + // Start cache with near enabled. + if (NEAR_ATOMIC_CACHE_NAME.equals(cacheName) || NEAR_TX_CACHE_NAME.equals(cacheName)) { + CacheConfiguration ccfg = new CacheConfiguration(cacheName) + .setNearConfiguration(new NearCacheConfiguration()); + + if (NEAR_TX_CACHE_NAME.equals(cacheName)) + ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL); + + cache = client.getOrCreateCache(ccfg); + } + else + cache = client.cache(cacheName); + + + cache.put(1, "1"); + + assertEquals("1", cache.get(1)); + + return null; + } + }, CacheException.class, null); + } + + /** + * To fail on cache start. + */ + private static class FailedIndexing implements GridQueryIndexing { + /** {@inheritDoc} */ + @Override public void start(GridKernalContext ctx, GridSpinBusyLock busyLock) throws IgniteCheckedException { + + } + + /** {@inheritDoc} */ + @Override public void stop() throws IgniteCheckedException { + + } + + /** {@inheritDoc} */ + @Override public QueryCursor> queryTwoStep(GridCacheContext cctx, SqlFieldsQuery qry, + GridQueryCancel cancel) throws IgniteCheckedException { + return null; + } + + /** {@inheritDoc} */ + @Override public QueryCursor> queryTwoStep(GridCacheContext cctx, + SqlQuery qry) throws IgniteCheckedException { + return null; + } + + /** {@inheritDoc} */ + @Override public GridQueryFieldsResult queryLocalSqlFields(@Nullable String spaceName, String qry, + Collection params, IndexingQueryFilter filter, boolean enforceJoinOrder, int timeout, + GridQueryCancel cancel) throws IgniteCheckedException { + return null; + } + + /** {@inheritDoc} */ + @Override public GridCloseableIterator> queryLocalSql(@Nullable String spaceName, + String qry, String alias, Collection params, GridQueryTypeDescriptor type, + IndexingQueryFilter filter) throws IgniteCheckedException { + return null; + } + + /** {@inheritDoc} */ + @Override public GridCloseableIterator> queryLocalText(@Nullable String spaceName, + String qry, GridQueryTypeDescriptor type, IndexingQueryFilter filter) throws IgniteCheckedException { + return null; + } + + /** {@inheritDoc} */ + @Override public void registerCache(GridCacheContext cctx, + CacheConfiguration ccfg) throws IgniteCheckedException { + if (FAILED_CACHES.contains(cctx.name()) && cctx.kernalContext().clientNode()) + throw new IgniteCheckedException("Test query exception " + cctx.name()); + } + + /** {@inheritDoc} */ + @Override public void unregisterCache(CacheConfiguration ccfg) throws IgniteCheckedException { + + } + + /** {@inheritDoc} */ + @Override public boolean registerType(@Nullable String spaceName, + GridQueryTypeDescriptor desc) throws IgniteCheckedException { + return false; + } + + /** {@inheritDoc} */ + @Override public void unregisterType(@Nullable String spaceName, + GridQueryTypeDescriptor type) throws IgniteCheckedException { + + } + + /** {@inheritDoc} */ + @Override public void store(@Nullable String spaceName, GridQueryTypeDescriptor type, CacheObject key, + CacheObject val, byte[] ver, long expirationTime) throws IgniteCheckedException { + + } + + /** {@inheritDoc} */ + @Override public void remove(@Nullable String spaceName, CacheObject key, + CacheObject val) throws IgniteCheckedException { + + } + + /** {@inheritDoc} */ + @Override public void onSwap(@Nullable String spaceName, CacheObject key) throws IgniteCheckedException { + + } + + /** {@inheritDoc} */ + @Override public void onUnswap(@Nullable String spaceName, CacheObject key, + CacheObject val) throws IgniteCheckedException { + + } + + /** {@inheritDoc} */ + @Override public void rebuildIndexes(@Nullable String spaceName, GridQueryTypeDescriptor type) { + + } + + /** {@inheritDoc} */ + @Override public IndexingQueryFilter backupFilter(AffinityTopologyVersion topVer, int[] parts) { + return null; + } + + /** {@inheritDoc} */ + @Override public void onDisconnected(IgniteFuture reconnectFut) { + + } + + /** {@inheritDoc} */ + @Override public PreparedStatement prepareNativeStatement(String schema, String sql) throws SQLException { + return null; + } + + /** {@inheritDoc} */ + @Override public Collection runningQueries(long duration) { + return null; + } + + /** {@inheritDoc} */ + @Override public void cancelQueries(Collection queries) { + + } + + /** {@inheritDoc} */ + @Override public void cancelAllQueries() { + + } + } +} 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 9fcf31a322f2a..914aedb40764a 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 @@ -81,6 +81,7 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheTxPreloadNoWriteTest; import org.apache.ignite.internal.processors.cache.IgniteCacheTxReplicatedPeekModesTest; import org.apache.ignite.internal.processors.cache.IgniteCacheTxStoreValueTest; +import org.apache.ignite.internal.processors.cache.IgniteClientCacheInitializationFailTest; import org.apache.ignite.internal.processors.cache.IgniteDynamicCacheFilterTest; import org.apache.ignite.internal.processors.cache.IgniteDynamicCacheStartNoExchangeTimeoutTest; import org.apache.ignite.internal.processors.cache.IgniteDynamicCacheStartSelfTest; @@ -235,6 +236,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(CacheAffinityEarlyTest.class); suite.addTestSuite(IgniteCacheCreatePutMultiNodeSelfTest.class); suite.addTestSuite(IgniteCacheCreatePutTest.class); + suite.addTestSuite(IgniteClientCacheInitializationFailTest.class); suite.addTestSuite(GridCacheTxLoadFromStoreOnLockSelfTest.class); From ba6227be49c8a395a5632e9841a6acb65ae340b6 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Mon, 10 Apr 2017 11:40:17 +0300 Subject: [PATCH 026/357] IGNITE-2466 - Disable back-pressure for sender data nodes to avoid deadlock. --- .../dht/atomic/GridDhtAtomicCache.java | 5 ++++- .../CacheAtomicPrimarySyncBackPressureTest.java | 16 ++++++++++++++++ 2 files changed, 20 insertions(+), 1 deletion(-) 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 5b9b1416d8f84..8a8221b428f13 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 @@ -1914,7 +1914,10 @@ private void updateAllAsyncInternal0( req.cleanup(!node.isLocal()); if (dhtFut != null) { - if (req.writeSynchronizationMode() == PRIMARY_SYNC && !dhtFut.isDone()) { + if (req.writeSynchronizationMode() == PRIMARY_SYNC + // To avoid deadlock disable back-pressure for sender data node. + && !ctx.discovery().cacheAffinityNode(ctx.discovery().node(nodeId), ctx.name()) + && !dhtFut.isDone()) { final IgniteRunnable tracker = GridNioBackPressureControl.threadTracker(); if (tracker != null && tracker instanceof GridNioMessageTracker) { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAtomicPrimarySyncBackPressureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAtomicPrimarySyncBackPressureTest.java index 49e3e5ca07a15..30436f559f4ec 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAtomicPrimarySyncBackPressureTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAtomicPrimarySyncBackPressureTest.java @@ -30,6 +30,7 @@ import org.apache.ignite.internal.managers.communication.GridIoMessage; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicDeferredUpdateResponse; import org.apache.ignite.lang.IgniteInClosure; +import org.apache.ignite.lang.IgnitePredicate; import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.spi.IgniteSpiException; import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; @@ -53,6 +54,8 @@ public class CacheAtomicPrimarySyncBackPressureTest extends GridCommonAbstractTe ccfg.setMemoryMode(CacheMemoryMode.ONHEAP_TIERED); ccfg.setAtomicityMode(CacheAtomicityMode.ATOMIC); + ccfg.setNodeFilter(new NodeFilter()); + TestCommunicationSpi spi = new TestCommunicationSpi(); spi.setMessageQueueLimit(100); @@ -148,4 +151,17 @@ private static void sleep(long millis) { throw new IgniteSpiException(e); } } + + /** + * Filters out server node producer. + */ + private static class NodeFilter implements IgnitePredicate { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Override public boolean apply(ClusterNode node) { + return !("server3".equals(node.attribute("org.apache.ignite.ignite.name"))); + } + } } From bb3ff120e6995431d10439243d8b163712de0e0e Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Mon, 10 Apr 2017 11:40:17 +0300 Subject: [PATCH 027/357] IGNITE-2466 - Disable back-pressure for sender data nodes to avoid deadlock. (cherry picked from commit ba6227b) --- .../dht/atomic/GridDhtAtomicCache.java | 5 ++++- .../CacheAtomicPrimarySyncBackPressureTest.java | 16 ++++++++++++++++ 2 files changed, 20 insertions(+), 1 deletion(-) 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 047be8751d7b1..4159359a2f780 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 @@ -1909,7 +1909,10 @@ private void updateAllAsyncInternal0( req.cleanup(!node.isLocal()); if (dhtFut != null) { - if (req.writeSynchronizationMode() == PRIMARY_SYNC && !dhtFut.isDone()) { + if (req.writeSynchronizationMode() == PRIMARY_SYNC + // To avoid deadlock disable back-pressure for sender data node. + && !ctx.discovery().cacheAffinityNode(ctx.discovery().node(nodeId), ctx.name()) + && !dhtFut.isDone()) { final IgniteRunnable tracker = GridNioBackPressureControl.threadTracker(); if (tracker != null && tracker instanceof GridNioMessageTracker) { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAtomicPrimarySyncBackPressureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAtomicPrimarySyncBackPressureTest.java index 49e3e5ca07a15..30436f559f4ec 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAtomicPrimarySyncBackPressureTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAtomicPrimarySyncBackPressureTest.java @@ -30,6 +30,7 @@ import org.apache.ignite.internal.managers.communication.GridIoMessage; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicDeferredUpdateResponse; import org.apache.ignite.lang.IgniteInClosure; +import org.apache.ignite.lang.IgnitePredicate; import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.spi.IgniteSpiException; import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; @@ -53,6 +54,8 @@ public class CacheAtomicPrimarySyncBackPressureTest extends GridCommonAbstractTe ccfg.setMemoryMode(CacheMemoryMode.ONHEAP_TIERED); ccfg.setAtomicityMode(CacheAtomicityMode.ATOMIC); + ccfg.setNodeFilter(new NodeFilter()); + TestCommunicationSpi spi = new TestCommunicationSpi(); spi.setMessageQueueLimit(100); @@ -148,4 +151,17 @@ private static void sleep(long millis) { throw new IgniteSpiException(e); } } + + /** + * Filters out server node producer. + */ + private static class NodeFilter implements IgnitePredicate { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Override public boolean apply(ClusterNode node) { + return !("server3".equals(node.attribute("org.apache.ignite.ignite.name"))); + } + } } From 315ff38eeef96f12954d6ff39c84d58b2b959667 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Thu, 6 Apr 2017 14:43:50 +0300 Subject: [PATCH 028/357] IGNITE-4879: Fixed System pool starvation while partition evicting. --- .../cache/distributed/dht/GridDhtLocalPartition.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 3ce1dd83ae61e..a0ccc28c33b0c 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 @@ -294,8 +294,8 @@ void onRemoved(GridDhtCacheEntry entry) { // Make sure to remove exactly this entry. map.removeEntry(entry); - // Attempt to evict. - tryEvict(); + // Attempt to evict in async way as multiple-threads can compete for same partition. + tryEvictAsync(true); } /** From 01ceeb13420b68edf12b0262fe0991e84c085dd8 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Thu, 6 Apr 2017 14:43:50 +0300 Subject: [PATCH 029/357] IGNITE-4863: Disallow change RootLogger log-level if it can have negative effect on other loggers. This closes #1687. --- .../ignite/logger/log4j/Log4JLogger.java | 64 +++++- .../log4j/GridLog4jInitializationTest.java | 212 ++++++++++++++++++ .../log4j/GridLog4jInitializedTest.java | 55 ----- .../log4j/GridLog4jNotInitializedTest.java | 46 ---- .../testsuites/IgniteLog4jTestSuite.java | 6 +- 5 files changed, 268 insertions(+), 115 deletions(-) create mode 100644 modules/log4j/src/test/java/org/apache/ignite/logger/log4j/GridLog4jInitializationTest.java delete mode 100644 modules/log4j/src/test/java/org/apache/ignite/logger/log4j/GridLog4jInitializedTest.java delete mode 100644 modules/log4j/src/test/java/org/apache/ignite/logger/log4j/GridLog4jNotInitializedTest.java diff --git a/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4JLogger.java b/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4JLogger.java index d5b0f0227bd85..f6ed83025af55 100644 --- a/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4JLogger.java +++ b/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4JLogger.java @@ -35,10 +35,12 @@ import org.apache.ignite.lang.IgniteClosure; import org.apache.ignite.logger.LoggerNodeIdAware; import org.apache.log4j.Appender; +import org.apache.log4j.AppenderSkeleton; import org.apache.log4j.Category; import org.apache.log4j.ConsoleAppender; import org.apache.log4j.FileAppender; import org.apache.log4j.Level; +import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.log4j.PatternLayout; import org.apache.log4j.varia.LevelRangeFilter; @@ -78,6 +80,9 @@ * injection. */ public class Log4JLogger implements IgniteLogger, LoggerNodeIdAware, Log4jFileAware { + /** */ + public static final String CONSOLE_ERR_APPENDER_NAME = "CONSOLE_ERR"; + /** Appenders. */ private static Collection fileAppenders = new GridConcurrentHashSet<>(); @@ -308,7 +313,7 @@ private void addConsoleAppenderIfNeeded(@Nullable Level logLevel, Appender appender = (Appender)appenders.nextElement(); if (appender instanceof ConsoleAppender) { - if ("CONSOLE_ERR".equals(appender.getName())) { + if (CONSOLE_ERR_APPENDER_NAME.equals(appender.getName())) { // Treat CONSOLE_ERR appender as a system one and don't count it. errAppender = (ConsoleAppender)appender; @@ -347,34 +352,62 @@ private void addConsoleAppenderIfNeeded(@Nullable Level logLevel, if (errAppender.getThreshold() == Level.ERROR) errAppender.setThreshold(Level.WARN); } - else - // No error console appender => create console appender with no level limit. - rootCategory.addAppender(createConsoleAppender(Level.OFF)); + else { + // No error console appender => create console appender with. + final AppenderSkeleton consoleAppender = createConsoleAppender(Level.OFF); + + consoleAppender.setThreshold(Level.INFO); - if (logLevel != null) + rootCategory.addAppender(consoleAppender); + } + + // Won't raise LogLevel if there is other loggers configured. As LogLevel can be inherited. + if (logLevel != null && !logLevel.isGreaterOrEqual(impl.getEffectiveLevel())) { impl.setLevel(logLevel); - } - // If still don't have appenders, disable logging. - if (!isConfigured()) + impl.warn("RootLogger logging level has been dropped by Apache Ignite.\n"+ + "Set lower log level or configure ConsoleAppender manually or disable ConsoleAppender automatic creation."); + } + } + else if (!isConfigured() && !hasOtherLoggers()) { + // If still don't have appenders and other loggers configured, disable logging. impl.setLevel(Level.OFF); + } quiet0 = quiet; inited = true; } } + /** + * Checks if there is other loggers configured. + * + * @return {@code True} if other logger found. + */ + private boolean hasOtherLoggers() { + final Enumeration loggers = LogManager.getCurrentLoggers(); + + while (loggers.hasMoreElements()) { + Logger c = (Logger)loggers.nextElement(); + + if (c != impl && c.getAllAppenders().hasMoreElements()) + return true; + } + + return false; + } + /** * Creates console appender with some reasonable default logging settings. * * @param maxLevel Max logging level. * @return New console appender. */ - private Appender createConsoleAppender(Level maxLevel) { + private AppenderSkeleton createConsoleAppender(Level maxLevel) { String fmt = "[%d{ABSOLUTE}][%-5p][%t][%c{1}] %m%n"; // Configure output that should go to System.out - Appender app = new ConsoleAppender(new PatternLayout(fmt), ConsoleAppender.SYSTEM_OUT); + AppenderSkeleton app = new ConsoleAppender(new PatternLayout(fmt), ConsoleAppender.SYSTEM_OUT); LevelRangeFilter lvlFilter = new LevelRangeFilter(); @@ -532,4 +565,15 @@ public static Collection logFiles() { } } } + + /** + * For test purposes only. + */ + static void reset(){ + inited = false; + + quiet0 = false; + + fileAppenders.clear(); + } } diff --git a/modules/log4j/src/test/java/org/apache/ignite/logger/log4j/GridLog4jInitializationTest.java b/modules/log4j/src/test/java/org/apache/ignite/logger/log4j/GridLog4jInitializationTest.java new file mode 100644 index 0000000000000..2a98490744db5 --- /dev/null +++ b/modules/log4j/src/test/java/org/apache/ignite/logger/log4j/GridLog4jInitializationTest.java @@ -0,0 +1,212 @@ +/* + * 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.logger.log4j; + +import junit.framework.TestCase; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.IgniteSystemProperties; +import org.apache.ignite.testframework.junits.common.GridCommonTest; +import org.apache.log4j.BasicConfigurator; +import org.apache.log4j.Level; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.log4j.varia.NullAppender; + +/** + * Log4j not initialized test. + */ +@GridCommonTest(group = "Logger") +public class GridLog4jInitializationTest extends TestCase { + /** */ + private static final boolean VERBOSE = true; + + /** {@inheritDoc} */ + @Override public void setUp() throws Exception { + super.setUp(); + + resetLogger(); + } + + /** {@inheritDoc} */ + @Override public void tearDown() throws Exception { + super.tearDown(); + + resetLogger(); + } + + /** */ + private void resetLogger() { + Log4JLogger.reset(); + + LogManager.resetConfiguration(); + + System.clearProperty(IgniteSystemProperties.IGNITE_QUIET); + } + + /** */ + public void testLogNotInitialized() { + IgniteLogger log = new Log4JLogger().getLogger(GridLog4jInitializationTest.class); + + if (VERBOSE) + printLoggerResults(log); + + assertTrue(log instanceof Log4JLogger); + + assertEquals(Level.OFF, LogManager.getRootLogger().getEffectiveLevel()); + } + + /** */ + public void testLogInitialized() { + BasicConfigurator.configure(); + + IgniteLogger log = new Log4JLogger().getLogger(GridLog4jInitializationTest.class); + + if (VERBOSE) + printLoggerResults(log); + + assertTrue(log instanceof Log4JLogger); + + assertEquals(Level.DEBUG, LogManager.getRootLogger().getEffectiveLevel()); + } + + /** */ + public void testNoAppendersConfigured() { + LogManager.getRootLogger().setLevel(Level.WARN); + + final Logger logger = LogManager.getLogger(GridLog4jInitializationTest.class); + + assertEquals(Level.WARN, logger.getEffectiveLevel()); + + IgniteLogger log = new Log4JLogger().getLogger(GridLog4jInitializationTest.class); + + if (VERBOSE) + printLoggerResults(log); + + assertEquals(Level.OFF, logger.getEffectiveLevel()); + } + + /** */ + public void testAutoAddConsoleAppender() { + System.setProperty(IgniteSystemProperties.IGNITE_QUIET, String.valueOf(false)); + + LogManager.getRootLogger().setLevel(Level.WARN); + + final Logger logger = LogManager.getLogger(GridLog4jInitializationTest.class); + + assertEquals(Level.WARN, logger.getEffectiveLevel()); + + IgniteLogger log = new Log4JLogger().getLogger(GridLog4jInitializationTest.class); + + if (VERBOSE) + printLoggerResults(log); + + assertEquals(Level.INFO, logger.getEffectiveLevel()); // LogLevel is allowed to be dropped. + } + + /** */ + public void testAutoAddConsoleAppender2() { + System.setProperty(IgniteSystemProperties.IGNITE_QUIET, String.valueOf(false)); + + LogManager.getRootLogger().setLevel(Level.DEBUG); + + final Logger logger = LogManager.getLogger(GridLog4jInitializationTest.class); + + assertEquals(Level.DEBUG, logger.getEffectiveLevel()); + + IgniteLogger log = new Log4JLogger().getLogger(GridLog4jInitializationTest.class); + + if (VERBOSE) + printLoggerResults(log); + + assertEquals(Level.DEBUG, logger.getEffectiveLevel()); // LogLevel should not change. + } + + /** */ + public void testOtherLoggerConfigured() { + LogManager.getRootLogger().setLevel(Level.DEBUG); + + final Logger logger = LogManager.getLogger(GridLog4jInitializationTest.class); + + logger.addAppender(new NullAppender()); + + assertEquals(Level.DEBUG, logger.getEffectiveLevel()); + + IgniteLogger log = new Log4JLogger().getLogger(GridLog4jInitializationTest.class); + + if (VERBOSE) + printLoggerResults(log); + + assertEquals(Level.DEBUG, logger.getEffectiveLevel()); // LogLevel should not be OFF. + } + + /** */ + public void testAutoAddConsoleAppenderWithOtherLoggerConfigured() { + System.setProperty(IgniteSystemProperties.IGNITE_QUIET, String.valueOf(false)); + + LogManager.getRootLogger().setLevel(Level.DEBUG); + + final Logger logger = LogManager.getLogger(GridLog4jInitializationTest.class); + + logger.addAppender(new NullAppender()); + + assertEquals(Level.DEBUG, logger.getEffectiveLevel()); + + IgniteLogger log = new Log4JLogger().getLogger(GridLog4jInitializationTest.class); + + if (VERBOSE) + printLoggerResults(log); + + assertEquals(Level.DEBUG, logger.getEffectiveLevel()); // LogLevel should not be raised. + } + + /** */ + public void testAutoAddConsoleAppenderWithOtherLoggerConfigured2() { + System.setProperty(IgniteSystemProperties.IGNITE_QUIET, String.valueOf(false)); + + LogManager.getRootLogger().setLevel(Level.WARN); + + final Logger logger = LogManager.getLogger(GridLog4jInitializationTest.class); + + logger.addAppender(new NullAppender()); + + assertEquals(Level.WARN, logger.getEffectiveLevel()); + + IgniteLogger log = new Log4JLogger().getLogger(GridLog4jInitializationTest.class); + + if (VERBOSE) + printLoggerResults(log); + + assertEquals(Level.INFO, logger.getEffectiveLevel()); // LogLevel is allowed to be dropped. + } + + /** */ + private void printLoggerResults(IgniteLogger log) { + if (log.isDebugEnabled()) + log.debug("This is 'debug' message."); + else + System.out.println("DEBUG level is not enabled."); + + if (log.isInfoEnabled()) + log.info("This is 'info' message."); + else + System.out.println("INFO level is not enabled."); + + log.warning("This is 'warning' message."); + log.error("This is 'error' message."); + } +} \ No newline at end of file diff --git a/modules/log4j/src/test/java/org/apache/ignite/logger/log4j/GridLog4jInitializedTest.java b/modules/log4j/src/test/java/org/apache/ignite/logger/log4j/GridLog4jInitializedTest.java deleted file mode 100644 index 94907f098b0c3..0000000000000 --- a/modules/log4j/src/test/java/org/apache/ignite/logger/log4j/GridLog4jInitializedTest.java +++ /dev/null @@ -1,55 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.logger.log4j; - -import junit.framework.TestCase; -import org.apache.ignite.IgniteLogger; -import org.apache.ignite.testframework.junits.common.GridCommonTest; -import org.apache.log4j.BasicConfigurator; - -/** - * Log4j initialized test. - */ -@GridCommonTest(group = "Logger") -public class GridLog4jInitializedTest extends TestCase { - - /** - * @throws Exception If failed. - */ - @Override protected void setUp() throws Exception { - BasicConfigurator.configure(); - } - - /** */ - public void testLogInitialize() { - IgniteLogger log = new Log4JLogger(); - - assert log.isInfoEnabled() == true; - - if (log.isDebugEnabled()) - log.debug("This is 'debug' message."); - - log.info("This is 'info' message."); - log.warning("This is 'warning' message."); - log.warning("This is 'warning' message.", new Exception("It's a test warning exception")); - log.error("This is 'error' message."); - log.error("This is 'error' message.", new Exception("It's a test error exception")); - - assert log.getLogger(GridLog4jInitializedTest.class.getName()) instanceof Log4JLogger; - } -} \ No newline at end of file diff --git a/modules/log4j/src/test/java/org/apache/ignite/logger/log4j/GridLog4jNotInitializedTest.java b/modules/log4j/src/test/java/org/apache/ignite/logger/log4j/GridLog4jNotInitializedTest.java deleted file mode 100644 index 390fdcb0a66bc..0000000000000 --- a/modules/log4j/src/test/java/org/apache/ignite/logger/log4j/GridLog4jNotInitializedTest.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.logger.log4j; - -import junit.framework.TestCase; -import org.apache.ignite.IgniteLogger; -import org.apache.ignite.testframework.junits.common.GridCommonTest; - -/** - * Log4j not initialized test. - */ -@GridCommonTest(group = "Logger") -public class GridLog4jNotInitializedTest extends TestCase { - /** */ - public void testLogInitialize() { - IgniteLogger log = new Log4JLogger().getLogger(GridLog4jNotInitializedTest.class); - - if (log.isDebugEnabled()) - log.debug("This is 'debug' message."); - else - System.out.println("DEBUG level is not enabled."); - - if (log.isInfoEnabled()) - log.info("This is 'info' message."); - else - System.out.println("INFO level is not enabled."); - - log.warning("This is 'warning' message."); - log.error("This is 'error' message."); - } -} \ No newline at end of file diff --git a/modules/log4j/src/test/java/org/apache/ignite/testsuites/IgniteLog4jTestSuite.java b/modules/log4j/src/test/java/org/apache/ignite/testsuites/IgniteLog4jTestSuite.java index f5f13d9e7ffb3..e20d32ce06c21 100644 --- a/modules/log4j/src/test/java/org/apache/ignite/testsuites/IgniteLog4jTestSuite.java +++ b/modules/log4j/src/test/java/org/apache/ignite/testsuites/IgniteLog4jTestSuite.java @@ -19,8 +19,7 @@ import junit.framework.TestSuite; import org.apache.ignite.logger.log4j.GridLog4jCorrectFileNameTest; -import org.apache.ignite.logger.log4j.GridLog4jInitializedTest; -import org.apache.ignite.logger.log4j.GridLog4jNotInitializedTest; +import org.apache.ignite.logger.log4j.GridLog4jInitializationTest; /** * Log4j logging tests. @@ -33,8 +32,7 @@ public class IgniteLog4jTestSuite extends TestSuite { public static TestSuite suite() throws Exception { TestSuite suite = new TestSuite("Log4j Logging Test Suite"); - suite.addTest(new TestSuite(GridLog4jInitializedTest.class)); - suite.addTest(new TestSuite(GridLog4jNotInitializedTest.class)); + suite.addTest(new TestSuite(GridLog4jInitializationTest.class)); suite.addTest(new TestSuite(GridLog4jCorrectFileNameTest.class)); return suite; From 0f7ef74216fab64f5d1d2b6d432b552b7fe40d2f Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Wed, 12 Apr 2017 13:01:25 +0300 Subject: [PATCH 030/357] IGNITE-4907: Fixed excessive service instances can be started with dynamic deployment. This closes #1766. --- .../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 d0b2733473333..37bffc3130a73 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 465084da5b00dcfc056d338f5d0a24875ca2af08 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Wed, 12 Apr 2017 13:01:25 +0300 Subject: [PATCH 031/357] 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 d0b2733473333..37bffc3130a73 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 a20c307df1dd000309a273ef93231fdc41a2a81c Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Thu, 13 Apr 2017 09:31:17 +0300 Subject: [PATCH 032/357] IGNITE-4891 - Fix. Key is deserialized during transactional get() even if withKeepBinary is set (Backport from master) --- .../transactions/IgniteTxLocalAdapter.java | 5 +- .../cache/CacheKeepBinaryTransactionTest.java | 121 ++++++++++++++++++ .../testsuites/IgniteCacheTestSuite5.java | 2 + 3 files changed, 126 insertions(+), 2 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheKeepBinaryTransactionTest.java 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 ea816fe6ce5f0..7f161859da101 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 @@ -1823,8 +1823,9 @@ private IgniteInternalFuture> checkMissed( for (Iterator it = missed.keySet().iterator(); it.hasNext(); ) { KeyCacheObject cacheKey = it.next(); - K keyVal = - (K)(keepCacheObjects ? cacheKey : cacheKey.value(cacheCtx.cacheObjectContext(), false)); + K keyVal = (K)(keepCacheObjects ? cacheKey + : cacheCtx.cacheObjectContext() + .unwrapBinaryIfNeeded(cacheKey, !deserializeBinary, false)); if (retMap.containsKey(keyVal)) it.remove(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheKeepBinaryTransactionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheKeepBinaryTransactionTest.java new file mode 100644 index 0000000000000..27d7c7f004330 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheKeepBinaryTransactionTest.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.internal.processors.cache; + +import org.apache.ignite.IgniteBinary; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.binary.BinaryObject; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.configuration.TransactionConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.binary.BinaryMarshaller; +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; + +/** + * Test that no deserialization happens with binary objects and keepBinary set flag. + */ +public class CacheKeepBinaryTransactionTest extends GridCommonAbstractTest { + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + TransactionConfiguration txCfg = new TransactionConfiguration(); + txCfg.setDefaultTxConcurrency(TransactionConcurrency.OPTIMISTIC); + txCfg.setDefaultTxIsolation(TransactionIsolation.REPEATABLE_READ); + + cfg.setTransactionConfiguration(txCfg); + + cfg.setMarshaller(new BinaryMarshaller()); + + CacheConfiguration ccfg = new CacheConfiguration("tx-cache"); + ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL); + + cfg.setCacheConfiguration(ccfg); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + startGrid(0); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + stopAllGrids(); + } + + /** + * @throws Exception If failed. + */ + public void testBinaryGet() throws Exception { + IgniteEx ignite = grid(0); + IgniteCache cache = ignite.cache("tx-cache").withKeepBinary(); + + try (Transaction tx = ignite.transactions().txStart()) { + BinaryObject key = ignite.binary().builder("test1") + .setField("id", 1).hashCode(1).build(); + + assertNull(cache.get(key)); + } + } + + /** + * @throws Exception If failed. + */ + public void testBinaryContains() throws Exception { + IgniteEx ignite = grid(0); + IgniteCache cache = ignite.cache("tx-cache").withKeepBinary(); + + try (Transaction tx = ignite.transactions().txStart()) { + BinaryObject key = ignite.binary().builder("test2") + .setField("id", 1).hashCode(1).build(); + + assertFalse(cache.containsKey(key)); + } + } + + /** + * @throws Exception If failed. + */ + public void testBinaryPutGetContains() throws Exception { + IgniteEx ignite = grid(0); + IgniteCache cache = ignite.cache("tx-cache").withKeepBinary(); + + try (Transaction tx = ignite.transactions().txStart()) { + IgniteBinary binary = ignite.binary(); + + BinaryObject key = binary.builder("test-key").setField("id", 1).hashCode(1).build(); + BinaryObject val = binary.builder("test-val").setField("id", 22).build(); + + cache.put(key, val); + + assertTrue(cache.containsKey(key)); + assertEquals(val, cache.get(key)); + } + } +} 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 7f0e23c29aa19..421676654e7d5 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 @@ -18,6 +18,7 @@ package org.apache.ignite.testsuites; import junit.framework.TestSuite; +import org.apache.ignite.internal.processors.cache.CacheKeepBinaryTransactionTest; import org.apache.ignite.internal.processors.cache.CacheNearReaderUpdateTest; import org.apache.ignite.internal.processors.cache.CacheSerializableTransactionsTest; import org.apache.ignite.internal.processors.cache.GridCacheSwapSpaceSpiConsistencySelfTest; @@ -50,6 +51,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(IgniteCacheWriteBehindNoUpdateSelfTest.class); suite.addTestSuite(IgniteCachePutStackOverflowSelfTest.class); suite.addTestSuite(GridCacheSwapSpaceSpiConsistencySelfTest.class); + suite.addTestSuite(CacheKeepBinaryTransactionTest.class); suite.addTestSuite(CacheLateAffinityAssignmentTest.class); suite.addTestSuite(CacheLateAffinityAssignmentFairAffinityTest.class); From 630558dfeb373f237057e394e8f2f63230d59dab Mon Sep 17 00:00:00 2001 From: vladisav Date: Thu, 13 Apr 2017 13:24:42 +0300 Subject: [PATCH 033/357] ignite-4173 IgniteSemaphore with failoverSafe enabled doesn't release permits in case permits owner node left topology Backport from master. (cherry picked from commit 76485fc) --- .../datastructures/GridCacheSemaphoreImpl.java | 12 ++++++++++++ .../SemaphoreFailoverSafeReleasePermitsTest.java | 9 ++++----- 2 files changed, 16 insertions(+), 5 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSemaphoreImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSemaphoreImpl.java index a11c79db6282c..e3cfd778da4e1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSemaphoreImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSemaphoreImpl.java @@ -532,6 +532,18 @@ private void initializeSemaphore() throws IgniteCheckedException { /** {@inheritDoc} */ @Override public void onNodeRemoved(UUID nodeId) { + try { + initializeSemaphore(); + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to recover from failover because distributed semaphore cannot be initialized " + + "(Ignore if this node is failing also)." ); + + // Degrade gracefully, no exception is thrown + // because other semaphores might also attempt to recover from failover. + return; + } + int numPermits = sync.getPermitsForNode(nodeId); if (numPermits > 0) { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/SemaphoreFailoverSafeReleasePermitsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/SemaphoreFailoverSafeReleasePermitsTest.java index 241253d3de8bd..9a588e7c419f7 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/SemaphoreFailoverSafeReleasePermitsTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/SemaphoreFailoverSafeReleasePermitsTest.java @@ -92,6 +92,9 @@ private void doTest() throws Exception { IgniteSemaphore sem = ignite.semaphore("sem", 1, true, true); + // Initialize second semaphore before the first one is broken. + IgniteSemaphore sem2 = grid(1).semaphore("sem", 1, true, true); + assertEquals(1, sem.availablePermits()); sem.acquire(1); @@ -102,11 +105,7 @@ private void doTest() throws Exception { awaitPartitionMapExchange(); - ignite = grid(1); - - sem = ignite.semaphore("sem", 1, true, true); - - assertTrue(sem.tryAcquire(1, 5000, TimeUnit.MILLISECONDS)); + assertTrue(sem2.tryAcquire(1, 5000, TimeUnit.MILLISECONDS)); } finally { stopAllGrids(); From 870b752c095ed3776e91a65b99763142b9f2ebc0 Mon Sep 17 00:00:00 2001 From: Vladisav Jelisavcic Date: Tue, 11 Apr 2017 14:09:12 +0300 Subject: [PATCH 034/357] ignite-1977 - fixed IgniteSemaphore fault tolerance. Backport from master. (cherry picked from commit 902bf42) --- .../GridCacheSemaphoreImpl.java | 74 ++++++++++++++++--- .../GridCacheSemaphoreState.java | 22 ++++++ ...bstractDataStructuresFailoverSelfTest.java | 21 ++++-- 3 files changed, 102 insertions(+), 15 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSemaphoreImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSemaphoreImpl.java index e3cfd778da4e1..c7426300db5e8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSemaphoreImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSemaphoreImpl.java @@ -232,6 +232,10 @@ final int nonfairTryAcquireShared(int acquires) { /** {@inheritDoc} */ @Override protected final boolean tryReleaseShared(int releases) { + // Fail-fast path. + if(broken) + return true; + // Check if some other node updated the state. // This method is called with release==0 only when trying to wake through update. if (releases == 0) @@ -295,6 +299,13 @@ protected boolean compareAndSetGlobalState(final int expVal, final int newVal, f throw new IgniteCheckedException("Failed to find semaphore with given name: " + name); + // Abort if state is already broken. + if (val.isBroken()) { + tx.rollback(); + + return true; + } + boolean retVal = val.getCount() == expVal; if (retVal) { @@ -349,11 +360,13 @@ protected boolean compareAndSetGlobalState(final int expVal, final int newVal, f /** * This method is used for releasing the permits acquired by failing node. + * In case the semaphore is broken, no permits are released and semaphore is set (globally) to broken state. * * @param nodeId ID of the failing node. + * @param broken Flag indicating that this semaphore is broken. * @return True if this is the call that succeeded to change the global state. */ - protected boolean releaseFailedNode(final UUID nodeId) { + protected boolean releaseFailedNode(final UUID nodeId, final boolean broken) { try { return CU.outTx( retryTopologySafe(new Callable() { @@ -369,6 +382,25 @@ protected boolean releaseFailedNode(final UUID nodeId) { throw new IgniteCheckedException("Failed to find semaphore with given name: " + name); + // Quit early if semaphore is already broken. + if( val.isBroken()) { + tx.rollback(); + + return false; + } + + // Mark semaphore as broken. No permits are released, + // since semaphore is useless from now on. + if (broken) { + val.setBroken(true); + + semView.put(key, val); + + tx.commit(); + + return true; + } + Map map = val.getWaiters(); if (!map.containsKey(nodeId)) { @@ -473,7 +505,11 @@ private void initializeSemaphore() throws IgniteCheckedException { tx.commit(); - return new Sync(count, waiters, failoverSafe); + Sync sync = new Sync(count, waiters, failoverSafe); + + sync.setBroken(val.isBroken()); + + return sync; } } }), @@ -520,6 +556,9 @@ private void initializeSemaphore() throws IgniteCheckedException { if (sync == null) return; + // Update broken flag. + sync.setBroken(val.isBroken()); + // Update permission count. sync.setPermits(val.getCount()); @@ -547,10 +586,13 @@ private void initializeSemaphore() throws IgniteCheckedException { int numPermits = sync.getPermitsForNode(nodeId); if (numPermits > 0) { - if (sync.failoverSafe) - // Release permits acquired by threads on failing node. - sync.releaseFailedNode(nodeId); - else { + // Semaphore is broken if reaches this point in non-failover safe mode. + boolean broken = !sync.failoverSafe; + + // Release permits acquired by threads on failing node. + sync.releaseFailedNode(nodeId, broken); + + if (broken) { // Interrupt every waiting thread if this semaphore is not failover safe. sync.setBroken(true); @@ -626,8 +668,11 @@ private void initializeSemaphore() throws IgniteCheckedException { sync.acquireSharedInterruptibly(permits); - if (isBroken()) + if (isBroken()) { + Thread.interrupted(); // Clear interrupt flag. + throw new InterruptedException(); + } } catch (IgniteCheckedException e) { throw U.convertException(e); @@ -743,8 +788,11 @@ private void initializeSemaphore() throws IgniteCheckedException { boolean result = sync.nonfairTryAcquireShared(1) >= 0; - if (isBroken()) + if (isBroken()) { + Thread.interrupted(); // Clear interrupt flag. + throw new InterruptedException(); + } return result; } @@ -768,8 +816,11 @@ private void initializeSemaphore() throws IgniteCheckedException { boolean result = sync.tryAcquireSharedNanos(1, unit.toNanos(timeout)); - if (isBroken()) + if (isBroken()) { + Thread.interrupted(); // Clear interrupt flag. + throw new InterruptedException(); + } return result; } @@ -837,8 +888,11 @@ private void initializeSemaphore() throws IgniteCheckedException { boolean result = sync.tryAcquireSharedNanos(permits, unit.toNanos(timeout)); - if (isBroken()) + if (isBroken()) { + Thread.interrupted(); + throw new InterruptedException(); + } return result; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSemaphoreState.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSemaphoreState.java index 50cdf102e0331..cdff9c51d7a35 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSemaphoreState.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSemaphoreState.java @@ -46,6 +46,9 @@ public class GridCacheSemaphoreState implements GridCacheInternal, Externalizabl /** FailoverSafe flag. */ private boolean failoverSafe; + /** Flag indicating that semaphore is no longer safe to use. */ + private boolean broken; + /** * Constructor. * @@ -101,6 +104,21 @@ public boolean isFailoverSafe() { return failoverSafe; } + /** + * @return broken flag. + */ + public boolean isBroken() { + return broken; + } + + /** + * + * @param broken Flag indicating that this semaphore should be no longer used. + */ + public void setBroken(boolean broken) { + this.broken = broken; + } + /** {@inheritDoc} */ @Override public Object clone() throws CloneNotSupportedException { return super.clone(); @@ -120,6 +138,8 @@ public boolean isFailoverSafe() { out.writeInt(e.getValue()); } } + + out.writeBoolean(broken); } /** {@inheritDoc} */ @@ -135,6 +155,8 @@ public boolean isFailoverSafe() { for (int i = 0; i < size; i++) waiters.put(U.readUuid(in), in.readInt()); } + + broken = in.readBoolean(); } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java index 6ba65abf5dec9..9295770796381 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java @@ -529,8 +529,6 @@ public void testSemaphoreMultipleTopologyChangeNonFailoverSafe() throws Exceptio * @throws Exception If failed. */ private void doTestSemaphore(ConstantTopologyChangeWorker topWorker, final boolean failoverSafe) throws Exception { - fail("https://issues.apache.org/jira/browse/IGNITE-1977"); - final int permits = topWorker instanceof MultipleTopologyChangeWorker || topWorker instanceof PartitionedMultipleTopologyChangeWorker ? TOP_CHANGE_THREAD_CNT * 3 : TOP_CHANGE_CNT; @@ -547,9 +545,14 @@ private void doTestSemaphore(ConstantTopologyChangeWorker topWorker, final boole break; } catch (IgniteInterruptedException e) { - // Exception may happen in non failover safe mode. + // Exception may happen in non failover safe mode. if (failoverSafe) throw e; + else { + // In non-failoverSafe mode semaphore is not safe to be reused, + // and should always be discarded after exception is caught. + break; + } } } @@ -568,6 +571,11 @@ private void doTestSemaphore(ConstantTopologyChangeWorker topWorker, final boole // Exception may happen in non failover safe mode. if (failoverSafe) throw e; + else { + // In non-failoverSafe mode semaphore is not safe to be reused, + // and should always be discarded after exception is caught. + break; + } } } @@ -580,8 +588,11 @@ private void doTestSemaphore(ConstantTopologyChangeWorker topWorker, final boole fut.get(); - for (Ignite g : G.allGrids()) - assertEquals(permits, g.semaphore(STRUCTURE_NAME, permits, false, false).availablePermits()); + // Semaphore is left in proper state only if failoverSafe mode is used. + if (failoverSafe) { + for (Ignite g : G.allGrids()) + assertEquals(permits, g.semaphore(STRUCTURE_NAME, permits, false, false).availablePermits()); + } } } From 405ce563fb7c35627c6e1bb0b68f423ba089c6f2 Mon Sep 17 00:00:00 2001 From: Dmitriy Shabalin Date: Fri, 14 Apr 2017 17:55:38 +0700 Subject: [PATCH 035/357] IGNITE-4068 Added common primitive for buttons group. Refactored existing button groups. (cherry picked from commit e5200c2) --- .../frontend/app/helpers/jade/mixins.pug | 12 ++++--- .../app/modules/sql/sql.controller.js | 6 ---- .../app/primitives/btn-group/index.pug | 35 +++++++++++++++++++ .../views/configuration/domains.tpl.pug | 1 - .../frontend/views/sql/sql.tpl.pug | 12 +++---- 5 files changed, 46 insertions(+), 20 deletions(-) create mode 100644 modules/web-console/frontend/app/primitives/btn-group/index.pug diff --git a/modules/web-console/frontend/app/helpers/jade/mixins.pug b/modules/web-console/frontend/app/helpers/jade/mixins.pug index db175a2146ef1..b8cbfe2100b7d 100644 --- a/modules/web-console/frontend/app/helpers/jade/mixins.pug +++ b/modules/web-console/frontend/app/helpers/jade/mixins.pug @@ -15,6 +15,7 @@ limitations under the License. include ./form +include ../../primitives/btn-group/index //- Mixin for advanced options toggle. mixin advanced-options-toggle(click, cond, showMessage, hideMessage) @@ -49,7 +50,7 @@ mixin main-table(title, rows, focusId, click, rowTemplate, searchField) //- Mixin with save, remove, clone and undo buttons. mixin save-remove-clone-undo-buttons(objectName) - -var removeTip = '"Remove current ' + objectName + '"' + -var removeTip = 'Remove current ' + objectName -var cloneTip = '"Clone current ' + objectName + '"' -var undoTip = '"Undo all changes for current ' + objectName + '"' @@ -60,10 +61,11 @@ mixin save-remove-clone-undo-buttons(objectName) a.btn.btn-primary(id='save-item' ng-disabled='!ui.inputForm.$dirty' ng-click='ui.inputForm.$dirty && saveItem()' bs-tooltip='' data-title=`{{saveBtnTipText(ui.inputForm.$dirty, '${objectName}')}}` data-placement='bottom' data-trigger='hover') Save .panel-tip-container(ng-show='backupItem._id') a.btn.btn-primary(id='clone-item' ng-click='cloneItem()' bs-tooltip=cloneTip data-placement='bottom' data-trigger='hover') Clone - .btn-group.panel-tip-container(ng-show='backupItem._id') - button.btn.btn-primary(id='remove-item' ng-click='removeItem()' bs-tooltip=removeTip data-placement='bottom' data-trigger='hover') Remove - button.btn.dropdown-toggle.btn-primary(id='remove-item-dropdown' data-toggle='dropdown' data-container='body' bs-dropdown='[{ text: "Remove All", click: "removeAllItems()" }]' data-placement='bottom-right') - span.caret + + -var actions = [{ text: "Remove", click: "removeItem()" }, { text: "Remove All", click: "removeAllItems()" }] + + +btn-group(actions, removeTip)(ng-show='backupItem._id') + .panel-tip-container(ng-show='backupItem') i.btn.btn-primary.fa.fa-undo(id='undo-item' ng-disabled='!ui.inputForm.$dirty' ng-click='ui.inputForm.$dirty && resetAll()' bs-tooltip=undoTip data-placement='bottom' data-trigger='hover') diff --git a/modules/web-console/frontend/app/modules/sql/sql.controller.js b/modules/web-console/frontend/app/modules/sql/sql.controller.js index 075bd559ee1e2..b1cd3d079c89c 100644 --- a/modules/web-console/frontend/app/modules/sql/sql.controller.js +++ b/modules/web-console/frontend/app/modules/sql/sql.controller.js @@ -271,12 +271,6 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', {value: 3600000, label: 'hours', short: 'h'} ]; - $scope.exportDropdown = [ - { text: 'Export all', click: 'exportCsvAll(paragraph)' } - // { 'text': 'Export all to CSV', 'click': 'exportCsvAll(paragraph)' }, - // { 'text': 'Export all to PDF', 'click': 'exportPdfAll(paragraph)' } - ]; - $scope.metadata = []; $scope.metaFilter = ''; diff --git a/modules/web-console/frontend/app/primitives/btn-group/index.pug b/modules/web-console/frontend/app/primitives/btn-group/index.pug new file mode 100644 index 0000000000000..c047b255bc3cd --- /dev/null +++ b/modules/web-console/frontend/app/primitives/btn-group/index.pug @@ -0,0 +1,35 @@ +//- + 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. + +mixin btn-group(actions, tip) + .btn-group.panel-tip-container&attributes(attributes) + button.btn.btn-primary( + ng-click=`${actions[0].click}` + + bs-tooltip='' + data-title=tip + + data-trigger='hover' + data-placement='bottom' + ) #{actions[0].text} + button.btn.dropdown-toggle.btn-primary( + bs-dropdown=`${JSON.stringify(actions)}` + + data-toggle='dropdown' + data-container='body' + data-placement='bottom-right' + ) + span.caret diff --git a/modules/web-console/frontend/views/configuration/domains.tpl.pug b/modules/web-console/frontend/views/configuration/domains.tpl.pug index 33528c7c8dd68..a8f9b5ce19c1a 100644 --- a/modules/web-console/frontend/views/configuration/domains.tpl.pug +++ b/modules/web-console/frontend/views/configuration/domains.tpl.pug @@ -55,7 +55,6 @@ include /app/helpers/jade/mixins .panel-tip-container(bs-tooltip='' data-title='Import domain models from database' data-placement='bottom') button.btn.btn-primary(ng-click='showImportDomainModal()') Import from database +save-remove-clone-undo-buttons('domain model') - .btn-group.panel-tip-container.pull-right(bs-tooltip='' data-title='Import domain models from demo database' data-placement='bottom') hr .bs-affix-fix div(bs-collapse='' data-allow-multiple='true' ng-model='ui.activePanels') diff --git a/modules/web-console/frontend/views/sql/sql.tpl.pug b/modules/web-console/frontend/views/sql/sql.tpl.pug index dcfc53137f616..6b14555a68ebf 100644 --- a/modules/web-console/frontend/views/sql/sql.tpl.pug +++ b/modules/web-console/frontend/views/sql/sql.tpl.pug @@ -134,10 +134,8 @@ mixin table-result-heading-query +result-toolbar .col-xs-4 .pull-right - .btn-group(ng-disabled='paragraph.loading') - button.btn.btn-primary(ng-click='exportCsv(paragraph)' bs-tooltip data-title='{{actionTooltip(paragraph, "export", false)}}') Export - button.btn.btn-primary.dropdown-toggle(id='export-item-dropdown' data-toggle='dropdown' data-container='body' bs-dropdown='exportDropdown' data-placement='bottom-right') - span.caret + -var actions = [{ text: "Export", click: 'exportCsv(paragraph)' }, { text: 'Export all', click: 'exportCsvAll(paragraph)' }] + +btn-group(actions, '{{ actionTooltip(paragraph, "export", false) }}')(ng-disabled='paragraph.loading') mixin table-result-heading-scan .total.row @@ -151,10 +149,8 @@ mixin table-result-heading-scan +result-toolbar .col-xs-4 .pull-right - .btn-group(ng-disabled='paragraph.loading') - button.btn.btn-primary(ng-click='exportCsv(paragraph)' bs-tooltip data-title='{{actionTooltip(paragraph, "export", false)}}') Export - button.btn.btn-primary.dropdown-toggle(id='export-item-dropdown' data-toggle='dropdown' data-container='body' bs-dropdown='exportDropdown' data-placement='bottom-right') - span.caret + -var actions = [{ text: "Export", click: 'exportCsv(paragraph)' }, { text: 'Export all', click: 'exportCsvAll(paragraph)' }] + +btn-group(actions, '{{ actionTooltip(paragraph, "export", false) }}')(ng-disabled='paragraph.loading') mixin table-result-body .grid(ui-grid='paragraph.gridOptions' ui-grid-resize-columns ui-grid-exporter) From 60cf48dc175fa288cd74d1189f0e992c9a16dc99 Mon Sep 17 00:00:00 2001 From: Vasiliy Sisko Date: Fri, 14 Apr 2017 18:00:47 +0700 Subject: [PATCH 036/357] IGNITE-4886 Catch all errors. (cherry picked from commit 7e8d9e8) --- .../internal/visor/service/VisorServiceDescriptor.java | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceDescriptor.java index 83ec77d5c22e0..26f5c10dd57b8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceDescriptor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceDescriptor.java @@ -68,7 +68,14 @@ public VisorServiceDescriptor() { */ public VisorServiceDescriptor(ServiceDescriptor srvc) { name = srvc.name(); - srvcCls = VisorTaskUtils.compactClass(srvc.serviceClass()); + + try { + srvcCls = VisorTaskUtils.compactClass(srvc.serviceClass()); + } + catch (Throwable e) { + srvcCls = e.getClass().getName() + ": " + e.getMessage(); + } + totalCnt = srvc.totalCount(); maxPerNodeCnt = srvc.maxPerNodeCount(); cacheName = srvc.cacheName(); From 81c3ed4c0511841f7056677db6063b4eb8d2def0 Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Fri, 14 Apr 2017 18:18:23 +0700 Subject: [PATCH 037/357] IGNITE-4896 Rewored GridClientNodeBean serialization. (cherry picked from commit a025268) --- .../client/message/GridClientNodeBean.java | 2 +- .../top/GridTopologyCommandHandler.java | 4 + ...ConfigurationCustomSerializerSelfTest.java | 147 ++++++++++++++++++ .../IgniteBinaryObjectsTestSuite.java | 2 + 4 files changed, 154 insertions(+), 1 deletion(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryConfigurationCustomSerializerSelfTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientNodeBean.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientNodeBean.java index ca8260865d9dc..f1848dfc6d0e9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientNodeBean.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientNodeBean.java @@ -250,7 +250,7 @@ public void setTcpPort(int tcpPort) { U.writeUuid(out, nodeId); - out.writeObject(consistentId); + out.writeObject(String.valueOf(consistentId)); out.writeObject(metrics); } 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..536ec88815471 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 @@ -52,7 +52,9 @@ import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.spi.IgnitePortProtocol; +import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_BINARY_CONFIGURATION; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_CACHE; +import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_NODE_CONSISTENT_ID; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_REST_TCP_ADDRS; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_REST_TCP_HOST_NAMES; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_REST_TCP_PORT; @@ -291,6 +293,8 @@ private GridClientNodeBean createNodeBean(ClusterNode node, boolean mtr, boolean attrs.remove(ATTR_TX_CONFIG); attrs.remove(ATTR_SECURITY_SUBJECT); attrs.remove(ATTR_SECURITY_CREDENTIALS); + attrs.remove(ATTR_BINARY_CONFIGURATION); + attrs.remove(ATTR_NODE_CONSISTENT_ID); for (Iterator> i = attrs.entrySet().iterator(); i.hasNext();) { Map.Entry e = i.next(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryConfigurationCustomSerializerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryConfigurationCustomSerializerSelfTest.java new file mode 100644 index 0000000000000..1da2967612c21 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryConfigurationCustomSerializerSelfTest.java @@ -0,0 +1,147 @@ +/* + * 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.Serializable; +import java.util.Collections; +import java.util.UUID; +import org.apache.ignite.binary.BinaryObjectException; +import org.apache.ignite.binary.BinaryReader; +import org.apache.ignite.binary.BinarySerializer; +import org.apache.ignite.binary.BinaryTypeConfiguration; +import org.apache.ignite.binary.BinaryWriter; +import org.apache.ignite.configuration.BinaryConfiguration; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.ConnectorConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.client.GridClient; +import org.apache.ignite.internal.client.GridClientConfiguration; +import org.apache.ignite.internal.client.GridClientFactory; +import org.apache.ignite.internal.client.GridClientProtocol; +import org.apache.ignite.internal.client.balancer.GridClientRoundRobinBalancer; +import org.apache.ignite.internal.visor.VisorTaskArgument; +import org.apache.ignite.internal.visor.node.VisorNodePingTask; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Tests that node will start with custom binary serializer and thin client will connect to such node. + */ +public class BinaryConfigurationCustomSerializerSelfTest extends GridCommonAbstractTest { + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setConnectorConfiguration(new ConnectorConfiguration()); + + cfg.setMarshaller(new BinaryMarshaller()); + + BinaryConfiguration binaryCfg = new BinaryConfiguration(); + + BinaryTypeConfiguration btc = new BinaryTypeConfiguration("org.MyClass"); + + btc.setIdMapper(BinaryContext.defaultIdMapper()); + btc.setEnum(false); + + // Set custom serializer that is unknown for Optimized marshaller. + btc.setSerializer(new MyBinarySerializer()); + + binaryCfg.setTypeConfigurations(Collections.singletonList(btc)); + + cfg.setBinaryConfiguration(binaryCfg); + + // Set custom consistent ID that unknown for Optimized marshaller. + cfg.setConsistentId(new MyConsistentId("test")); + + cfg.setCacheConfiguration(new CacheConfiguration("TEST")); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + startGrids(2); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** + * Test that thin client will be able to connect to node with custom binary serializer and custom consistent ID. + * + * @throws Exception If failed. + */ + public void testThinClientConnected() throws Exception { + UUID nid = ignite(0).cluster().localNode().id(); + + GridClientConfiguration clnCfg = new GridClientConfiguration(); + + clnCfg.setProtocol(GridClientProtocol.TCP); + clnCfg.setServers(Collections.singleton("127.0.0.1:11211")); + clnCfg.setBalancer(new GridClientRoundRobinBalancer()); + + // Start client. + GridClient client = GridClientFactory.start(clnCfg); + + // Execute some task. + client.compute().execute(VisorNodePingTask.class.getName(), new VisorTaskArgument<>(nid, nid, false)); + + GridClientFactory.stop(client.id(), false); + } + + /** + * Custom consistent ID. + */ + private static class MyConsistentId implements Serializable { + /** */ + private static final long serialVersionUID = 0L; + + /** Actual ID. */ + private String id; + + /** + * @param id Actual ID. + */ + MyConsistentId(String id) { + this.id = id; + } + + /** + * @return Consistent ID. + */ + public String getId() { + return id; + } + } + + /** + * Custom BinarySerializer. + */ + private static class MyBinarySerializer implements BinarySerializer { + /** {@inheritDoc} */ + @Override public void writeBinary(Object obj, BinaryWriter writer) throws BinaryObjectException { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void readBinary(Object obj, BinaryReader reader) throws BinaryObjectException { + // No-op. + } + } +} 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 3496dbf381ac8..4aa22278d1d9d 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 @@ -22,6 +22,7 @@ import org.apache.ignite.internal.binary.BinaryBasicIdMapperSelfTest; import org.apache.ignite.internal.binary.BinaryBasicNameMapperSelfTest; import org.apache.ignite.internal.binary.BinaryConfigurationConsistencySelfTest; +import org.apache.ignite.internal.binary.BinaryConfigurationCustomSerializerSelfTest; import org.apache.ignite.internal.binary.BinaryEnumsSelfTest; import org.apache.ignite.internal.binary.BinaryFieldIdentityResolverSelfTest; import org.apache.ignite.internal.binary.BinaryFieldsHeapSelfTest; @@ -100,6 +101,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(BinaryIdentityResolverConfigurationSelfTest.class); suite.addTestSuite(BinaryConfigurationConsistencySelfTest.class); + suite.addTestSuite(BinaryConfigurationCustomSerializerSelfTest.class); suite.addTestSuite(GridBinaryMarshallerCtxDisabledSelfTest.class); suite.addTestSuite(BinaryObjectBuilderDefaultMappersSelfTest.class); suite.addTestSuite(BinaryObjectBuilderSimpleNameLowerCaseMappersSelfTest.class); From 4a1415ad01ff9fde30d5c7c02e6d938f1515178d Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Wed, 12 Apr 2017 13:01:25 +0300 Subject: [PATCH 038/357] 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 e206b9f1fd3dbf927f940d558144a4796479ed5d Mon Sep 17 00:00:00 2001 From: vsisko Date: Fri, 14 Apr 2017 18:32:30 +0700 Subject: [PATCH 039/357] IGNITE-4871 Added Kubernetes IP finder to Cluster configuration screen. (cherry picked from commit f978ff2) --- modules/web-console/backend/app/mongo.js | 8 +++- .../frontend/app/data/pom-dependencies.json | 1 + .../app/helpers/jade/form/form-field-text.pug | 40 +++++++++++-------- .../frontend/app/helpers/jade/mixins.pug | 8 +++- .../generator/ConfigurationGenerator.js | 12 ++++++ .../generator/defaults/Cluster.service.js | 6 +++ .../states/configuration/clusters/general.pug | 3 ++ .../clusters/general/discovery/kubernetes.pug | 37 +++++++++++++++++ .../summary/summary.controller.js | 3 ++ .../configuration/summary/summary.worker.js | 22 ++++++++++ .../controllers/clusters-controller.js | 3 +- 11 files changed, 124 insertions(+), 19 deletions(-) create mode 100644 modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/kubernetes.pug diff --git a/modules/web-console/backend/app/mongo.js b/modules/web-console/backend/app/mongo.js index 2d252b9b8eccc..f4a62b1fe2558 100644 --- a/modules/web-console/backend/app/mongo.js +++ b/modules/web-console/backend/app/mongo.js @@ -407,7 +407,7 @@ module.exports.factory = function(passportMongo, settings, pluginMongo, mongoose authenticator: String, forceServerMode: Boolean, clientReconnectDisabled: Boolean, - kind: {type: String, enum: ['Vm', 'Multicast', 'S3', 'Cloud', 'GoogleStorage', 'Jdbc', 'SharedFs', 'ZooKeeper']}, + kind: {type: String, enum: ['Vm', 'Multicast', 'S3', 'Cloud', 'GoogleStorage', 'Jdbc', 'SharedFs', 'ZooKeeper', 'Kubernetes']}, Vm: { addresses: [String] }, @@ -528,6 +528,12 @@ module.exports.factory = function(passportMongo, settings, pluginMongo, mongoose basePath: String, serviceName: String, allowDuplicateRegistrations: Boolean + }, + Kubernetes: { + serviceName: String, + namespace: String, + masterUrl: String, + accountToken: String } }, atomicConfiguration: { diff --git a/modules/web-console/frontend/app/data/pom-dependencies.json b/modules/web-console/frontend/app/data/pom-dependencies.json index 7d2bed0de8172..3e0543bb64861 100644 --- a/modules/web-console/frontend/app/data/pom-dependencies.json +++ b/modules/web-console/frontend/app/data/pom-dependencies.json @@ -3,6 +3,7 @@ "S3": {"artifactId": "ignite-aws"}, "GoogleStorage": {"artifactId": "ignite-gce"}, "ZooKeeper": {"artifactId": "ignite-zookeeper"}, + "Kubernetes": {"artifactId": "ignite-kubernetes"}, "Log4j": {"artifactId": "ignite-log4j"}, "Log4j2": {"artifactId": "ignite-log4j2"}, diff --git a/modules/web-console/frontend/app/helpers/jade/form/form-field-text.pug b/modules/web-console/frontend/app/helpers/jade/form/form-field-text.pug index 76ea6e609d6e1..63feaf85f13a3 100644 --- a/modules/web-console/frontend/app/helpers/jade/form/form-field-text.pug +++ b/modules/web-console/frontend/app/helpers/jade/form/form-field-text.pug @@ -14,12 +14,12 @@ See the License for the specific language governing permissions and limitations under the License. -mixin ignite-form-field-input(name, model, disabled, required, placeholder) +mixin ignite-field-input(type, name, model, disabled, required, placeholder) input.form-control( id=`{{ ${name} }}Input` name=`{{ ${name} }}` placeholder=placeholder - type='text' + type=type data-ng-model=model @@ -30,21 +30,11 @@ mixin ignite-form-field-input(name, model, disabled, required, placeholder) data-ignite-form-panel-field='' )&attributes(attributes ? attributes.attributes ? attributes.attributes : attributes: {}) -mixin ignite-form-field-url-input(name, model, disabled, required, placeholder) - input.form-control( - id=`{{ ${name} }}Input` - name=`{{ ${name} }}` - placeholder=placeholder - type='url' - - data-ng-model=model - - data-ng-required=required && `${required}` - data-ng-disabled=disabled && `${disabled}` - data-ng-focus='tableReset()' +mixin ignite-form-field-input(name, model, disabled, required, placeholder) + +ignite-field-input('text', name, model, disabled, required, placeholder) - data-ignite-form-panel-field='' - )&attributes(attributes ? attributes.attributes ? attributes.attributes : attributes: {}) +mixin ignite-form-field-url-input(name, model, disabled, required, placeholder) + +ignite-field-input('url', name, model, disabled, required, placeholder) mixin ignite-form-field-text(label, model, name, disabled, required, placeholder, tip) -var errLbl = label.substring(0, label.length - 1) @@ -62,3 +52,21 @@ mixin ignite-form-field-text(label, model, name, disabled, required, placeholder .input-tip +ignite-form-field-input(name, model, disabled, required, placeholder)(attributes=attributes) + +mixin ignite-form-field-url(label, model, name, required, placeholder, tip) + -var errLbl = label.substring(0, label.length - 1) + + .ignite-form-field + +ignite-form-field__label(label, name, required) + .ignite-form-field__control + if tip + i.tipField.icon-help(bs-tooltip='' data-title=tip) + + if block + block + + +form-field-feedback(name, 'required', errLbl + ' could not be empty!') + +form-field-feedback(name, 'url', errLbl + ' should be a valid URL!') + + .input-tip + +ignite-form-field-url-input(name, model, false, required, placeholder)(attributes=attributes) diff --git a/modules/web-console/frontend/app/helpers/jade/mixins.pug b/modules/web-console/frontend/app/helpers/jade/mixins.pug index b8cbfe2100b7d..472d30414b31e 100644 --- a/modules/web-console/frontend/app/helpers/jade/mixins.pug +++ b/modules/web-console/frontend/app/helpers/jade/mixins.pug @@ -199,6 +199,12 @@ mixin text(lbl, model, name, required, placeholder, tip) if block block +//- Mixin for text field. +mixin url(lbl, model, name, required, placeholder, tip) + +ignite-form-field-url(lbl, model, name, required, placeholder, tip) + if block + block + //- Mixin for password field. mixin password(lbl, model, name, required, placeholder, tip) +ignite-form-field-password(lbl, model, name, false, required, placeholder, tip) @@ -350,7 +356,7 @@ mixin table-java-package-field(name, model, items, valid, save, newItem) ignite-on-escape=onEscape ) -//- Mixin for table java package field. +//- Mixin for table url field. mixin table-url-field(name, model, items, valid, save, newItem) -var resetOnEnter = newItem ? '(stopblur = true) && (group.add = [{}])' : '(field.edit = false)' -var onEnter = `${valid} && (${save}); ${valid} && ${resetOnEnter};` diff --git a/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js b/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js index 670b8284f14c1..7cd910da43f93 100644 --- a/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js @@ -320,6 +320,18 @@ export default class IgniteConfigurationGenerator { .boolProperty('allowDuplicateRegistrations'); break; + + case 'Kubernetes': + ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.kubernetes.TcpDiscoveryKubernetesIpFinder', + 'ipFinder', cluster.discovery.Kubernetes, clusterDflts.discovery.Kubernetes); + + ipFinder.stringProperty('serviceName') + .stringProperty('namespace') + .stringProperty('masterUrl') + .pathProperty('accountToken'); + + break; + default: // No-op. } diff --git a/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.service.js b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.service.js index 6333ef96afcd1..b2e91c8836fe8 100644 --- a/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.service.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.service.js @@ -75,6 +75,12 @@ const DFLT_CLUSTER = { Forever: { retryIntervalMs: 1000 } + }, + Kubernetes: { + serviceName: 'ignite', + namespace: 'default', + masterUrl: 'https://kubernetes.default.svc.cluster.local:443', + accountToken: '/var/run/secrets/kubernetes.io/serviceaccount/token' } }, atomics: { diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general.pug b/modules/web-console/frontend/app/modules/states/configuration/clusters/general.pug index dfc49d61ef4c2..62cc23db0045a 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/general.pug +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general.pug @@ -52,6 +52,7 @@ include /app/helpers/jade/mixins
  • JDBC - JDBC based IP finder that use database to store node IP address
  • \
  • Shared filesystem - Shared filesystem based IP finder that use file to store node IP address
  • \
  • Apache ZooKeeper - Apache ZooKeeper based IP finder when you use ZooKeeper to coordinate your distributed environment
  • \ +
  • Kubernetes - IP finder for automatic lookup of Ignite nodes running in Kubernetes environment
  • \ ') .settings-row .panel-details @@ -71,6 +72,8 @@ include /app/helpers/jade/mixins include ./general/discovery/vm div(ng-if=`${modelDiscoveryKind} === 'ZooKeeper'`) include ./general/discovery/zookeeper + div(ng-if=`${modelDiscoveryKind} === 'Kubernetes'`) + include ./general/discovery/kubernetes .col-sm-6 -var model = 'backupItem' +preview-xml-java(model, 'clusterCaches', 'caches') diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/kubernetes.pug b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/kubernetes.pug new file mode 100644 index 0000000000000..c4e639816cd58 --- /dev/null +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/kubernetes.pug @@ -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. + +include /app/helpers/jade/mixins + +-var discoveryKind = 'Kubernetes' +-var model = 'backupItem.discovery.Kubernetes' + +div + .details-row + +text('Service name:', `${model}.serviceName`, `'${discoveryKind}ServiceName'`, 'false', 'ignite', + "The name of Kubernetes service for Ignite pods' IP addresses lookup.
    \ + The name of the service must be equal to the name set in service's Kubernetes configuration.
    \ + If this parameter is not changed then the name of the service has to be set to 'ignite' in the corresponding Kubernetes configuration.") + .details-row + +text('Namespace:', `${model}.namespace`, `'${discoveryKind}Namespace'`, 'false', 'default', + "The namespace the Kubernetes service belongs to.
    \ + By default, it's supposed that the service is running under Kubernetes `default` namespace.") + .details-row + +url('Kubernetes server:', `${model}.masterUrl`, `'${discoveryKind}MasterUrl'`, 'false', 'https://kubernetes.default.svc.cluster.local:443', + 'The host name of the Kubernetes API server') + .details-row + +text('Service token file:', `${model}.accountToken`, `'${discoveryKind}AccountToken'`, 'false', '/var/run/secrets/kubernetes.io/serviceaccount/token', + 'The path to the service token file') diff --git a/modules/web-console/frontend/app/modules/states/configuration/summary/summary.controller.js b/modules/web-console/frontend/app/modules/states/configuration/summary/summary.controller.js index 25203b30d1a27..0089528140f53 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/summary/summary.controller.js +++ b/modules/web-console/frontend/app/modules/states/configuration/summary/summary.controller.js @@ -264,6 +264,9 @@ export default [ if (cluster.discovery.kind === 'Jdbc' && cluster.discovery.Jdbc.dialect) $scope.dialects[cluster.discovery.Jdbc.dialect] = true; + if (cluster.discovery.kind === 'Kubernetes') + resourcesFolder.children.push({ type: 'file', name: 'ignite-service.yaml' }); + _.forEach(cluster.caches, (cache) => { if (cache.cacheStoreFactory) { const store = cache.cacheStoreFactory[cache.cacheStoreFactory.kind]; diff --git a/modules/web-console/frontend/app/modules/states/configuration/summary/summary.worker.js b/modules/web-console/frontend/app/modules/states/configuration/summary/summary.worker.js index 7ea1d5ad5beb0..5933f6caa6c53 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/summary/summary.worker.js +++ b/modules/web-console/frontend/app/modules/states/configuration/summary/summary.worker.js @@ -38,6 +38,25 @@ const generator = IgniteConfigurationGenerator; const escapeFileName = (name) => name.replace(/[\\\/*\"\[\],\.:;|=<>?]/g, '-').replace(/ /g, '_'); +const kubernetesConfig = (cluster) => { + if (!cluster.discovery.Kubernetes) + cluster.discovery.Kubernetes = { serviceName: 'ignite' }; + + return `apiVersion: v1\n\ +kind: Service\n\ +metadata:\n\ + # Name of Ignite Service used by Kubernetes IP finder for IP addresses lookup.\n\ + name: ${ cluster.discovery.Kubernetes.serviceName || 'ignite' }\n\ +spec:\n\ + clusterIP: None # custom value.\n\ + ports:\n\ + - port: 9042 # custom value.\n\ + selector:\n\ + # Must be equal to one of the labels set in Ignite pods'\n\ + # deployement configuration.\n\ + app: ${ cluster.discovery.Kubernetes.serviceName || 'ignite' }`; +}; + // eslint-disable-next-line no-undef onmessage = function(e) { const {cluster, data, demo} = e.data; @@ -67,6 +86,9 @@ onmessage = function(e) { const metaPath = `${resourcesPath}/META-INF`; + if (cluster.discovery.kind === 'Kubernetes') + zip.file(`${metaPath}/ignite-service.yaml`, kubernetesConfig(cluster)); + zip.file(`${metaPath}/${serverXml}`, spring.igniteConfiguration(cfg).asString()); zip.file(`${metaPath}/${clientXml}`, spring.igniteConfiguration(clientCfg, clientNearCaches).asString()); diff --git a/modules/web-console/frontend/controllers/clusters-controller.js b/modules/web-console/frontend/controllers/clusters-controller.js index c8392cf7ef0aa..e936955e05054 100644 --- a/modules/web-console/frontend/controllers/clusters-controller.js +++ b/modules/web-console/frontend/controllers/clusters-controller.js @@ -228,7 +228,8 @@ export default ['clustersController', [ {value: 'GoogleStorage', label: 'Google cloud storage'}, {value: 'Jdbc', label: 'JDBC'}, {value: 'SharedFs', label: 'Shared filesystem'}, - {value: 'ZooKeeper', label: 'Apache ZooKeeper'} + {value: 'ZooKeeper', label: 'Apache ZooKeeper'}, + {value: 'Kubernetes', label: 'Kubernetes'} ]; $scope.swapSpaceSpis = [ From 4518f51f4c1c29517d518f5e44f5d70d17c51170 Mon Sep 17 00:00:00 2001 From: sboikov Date: Fri, 14 Apr 2017 16:38:46 +0300 Subject: [PATCH 040/357] Added IgniteGetFromComputeBenchmark. --- .../cache/IgniteGetFromComputeBenchmark.java | 167 ++++++++++++++++++ 1 file changed, 167 insertions(+) create mode 100644 modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetFromComputeBenchmark.java diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetFromComputeBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetFromComputeBenchmark.java new file mode 100644 index 0000000000000..66aec82653d2f --- /dev/null +++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetFromComputeBenchmark.java @@ -0,0 +1,167 @@ +/* + * 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.yardstick.cache; + +import java.util.Map; +import java.util.Set; +import java.util.TreeSet; +import javax.cache.processor.MutableEntry; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteCompute; +import org.apache.ignite.IgniteDataStreamer; +import org.apache.ignite.cache.CacheEntryProcessor; +import org.apache.ignite.lang.IgniteCallable; +import org.apache.ignite.lang.IgniteFuture; +import org.apache.ignite.resources.IgniteInstanceResource; +import org.apache.ignite.yardstick.cache.model.SampleValue; +import org.yardstickframework.BenchmarkConfiguration; + +import static org.yardstickframework.BenchmarkUtils.println; + +/** + * Benchmark created to verify that slow EntryProcessor does not affect 'get' performance. + */ +public class IgniteGetFromComputeBenchmark extends IgniteCacheAbstractBenchmark { + /** */ + private static final String CACHE_NAME = "atomic-offheap"; + + /** */ + private IgniteCompute compute; + + /** */ + private IgniteCache asyncCache; + + /** */ + private ThreadLocal invokeFut = new ThreadLocal<>(); + + /** {@inheritDoc} */ + @Override public void setUp(BenchmarkConfiguration cfg) throws Exception { + super.setUp(cfg); + + if (args.preloadAmount() > args.range()) + throw new IllegalArgumentException("Preloading amount (\"-pa\", \"--preloadAmount\") " + + "must by less then the range (\"-r\", \"--range\")."); + + String cacheName = cache().getName(); + + println(cfg, "Loading data for cache: " + cacheName); + + long start = System.nanoTime(); + + try (IgniteDataStreamer dataLdr = ignite().dataStreamer(cacheName)) { + for (int i = 0; i < args.preloadAmount(); i++) { + dataLdr.addData(i, new SampleValue(i)); + + if (i % 100000 == 0) { + if (Thread.currentThread().isInterrupted()) + break; + + println("Loaded entries: " + i); + } + } + } + + println(cfg, "Finished populating data [time=" + ((System.nanoTime() - start) / 1_000_000) + "ms, " + + "amount=" + args.preloadAmount() + ']'); + + compute = ignite().compute(); + + asyncCache = cache().withAsync(); + } + + /** {@inheritDoc} */ + @Override public boolean test(Map ctx) throws Exception { + IgniteFuture fut = invokeFut.get(); + + if (fut == null || fut.isDone()) { + Set keys = new TreeSet<>(); + + for (int i = 0; i < 3; i++) + keys.add(nextRandom(args.range())); + + asyncCache.invokeAll(keys, new SlowEntryProcessor(0)); + + invokeFut.set(asyncCache.future()); + } + + int key = nextRandom(args.range()); + + compute.affinityCall(CACHE_NAME, key, new GetClosure(key)); + + return true; + } + + /** {@inheritDoc} */ + @Override protected IgniteCache cache() { + return ignite().cache(CACHE_NAME); + } + + /** + * + */ + public static class GetClosure implements IgniteCallable { + /** */ + @IgniteInstanceResource + private Ignite ignite; + + /** */ + private final int key; + + /** + * @param key Key. + */ + public GetClosure(int key) { + this.key = key; + } + + /** {@inheritDoc} */ + @Override public Object call() throws Exception { + return ignite.cache(CACHE_NAME).get(key); + } + } + + /** + * + */ + public static class SlowEntryProcessor implements CacheEntryProcessor { + /** */ + private Object val; + + /** + * @param val Value. + */ + public SlowEntryProcessor(Object val) { + this.val = val; + } + + /** {@inheritDoc} */ + @Override public Object process(MutableEntry entry, Object... args) { + try { + Thread.sleep(10); + } + catch (InterruptedException ignore) { + // No-op. + } + + entry.setValue(val); + + return null; + } + } +} From b22738080101536a8af1ed60e70d693897e9bc7c Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Fri, 14 Apr 2017 17:54:02 +0300 Subject: [PATCH 041/357] ignite-4173 Fix test. Permits must be released on node fail. (cherry picked from commit 1f867c6) --- .../GridCacheAbstractDataStructuresFailoverSelfTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java index 9295770796381..4ab7b6780d2aa 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java @@ -418,7 +418,7 @@ public void testSemaphoreFailoverSafe() throws Exception { stopGrid(NEW_GRID_NAME); - assertEquals(10, semaphore.availablePermits()); + assertEquals(20, semaphore.availablePermits()); } } From 1985496ea98f4d7112a0b99727297427f343e9b2 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Mon, 17 Apr 2017 10:20:41 +0300 Subject: [PATCH 042/357] Add missed license headers. --- modules/kubernetes/config/Dockerfile | 17 +++++++++++++++++ .../kubernetes/config/ignite-deployment.yaml | 17 +++++++++++++++++ modules/kubernetes/config/ignite-service.yaml | 17 +++++++++++++++++ 3 files changed, 51 insertions(+) diff --git a/modules/kubernetes/config/Dockerfile b/modules/kubernetes/config/Dockerfile index 4e08ce80e2147..2274535b5ccb2 100644 --- a/modules/kubernetes/config/Dockerfile +++ b/modules/kubernetes/config/Dockerfile @@ -1,3 +1,20 @@ +# +# 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. +# + # Use Java 8 image as default one. FROM java:8 diff --git a/modules/kubernetes/config/ignite-deployment.yaml b/modules/kubernetes/config/ignite-deployment.yaml index ed5c102a666a6..1e30f075e63bb 100644 --- a/modules/kubernetes/config/ignite-deployment.yaml +++ b/modules/kubernetes/config/ignite-deployment.yaml @@ -1,3 +1,20 @@ +# +# 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. +# + # An example of a Kubernetes configuration for Ignite pods deployment. apiVersion: extensions/v1beta1 kind: Deployment diff --git a/modules/kubernetes/config/ignite-service.yaml b/modules/kubernetes/config/ignite-service.yaml index 07b751624cbbf..7a7343ab522b3 100644 --- a/modules/kubernetes/config/ignite-service.yaml +++ b/modules/kubernetes/config/ignite-service.yaml @@ -1,3 +1,20 @@ +# +# 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. +# + # An example of a Kubernetes configuration for Ignite lookup service deployment. apiVersion: v1 kind: Service From ae794ab45b22abe1e570d5d29661396ea7c0f951 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Mon, 17 Apr 2017 13:40:28 +0300 Subject: [PATCH 043/357] IGNITE-4159 Fix version. --- modules/kubernetes/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/kubernetes/pom.xml b/modules/kubernetes/pom.xml index 5d4e5f0c8b509..7a632c617934f 100644 --- a/modules/kubernetes/pom.xml +++ b/modules/kubernetes/pom.xml @@ -31,7 +31,7 @@ ignite-kubernetes - 2.0.0-SNAPSHOT + 1.9.0-SNAPSHOT http://ignite.apache.org From bf1049741f7a64728bd433f78262ba273f969848 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Mon, 17 Apr 2017 19:00:30 +0300 Subject: [PATCH 044/357] 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 86c49514c64225059a31912fdccadc6dc68a8397 Mon Sep 17 00:00:00 2001 From: sboikov Date: Tue, 18 Apr 2017 11:15:44 +0300 Subject: [PATCH 045/357] ignite-4932 For offheap_tiered cache first try read value without cache entry creation --- .../JettyRestProcessorAbstractSelfTest.java | 19 + .../processors/cache/GridCacheAdapter.java | 142 +++++-- .../cache/GridCacheConcurrentMap.java | 2 - .../cache/GridCacheConcurrentMapImpl.java | 7 +- .../processors/cache/GridCacheContext.java | 15 + .../cache/GridCacheEventManager.java | 32 ++ .../processors/cache/GridCacheMapEntry.java | 12 +- .../cache/GridCacheMapEntryFactory.java | 4 +- .../cache/GridCacheSwapManager.java | 36 ++ .../cache/GridNoStorageCacheMap.java | 6 +- .../cache/IgniteCacheExpiryPolicy.java | 5 + .../GridDistributedCacheEntry.java | 6 +- .../GridCachePartitionedConcurrentMap.java | 4 +- .../distributed/dht/GridDhtCacheAdapter.java | 9 +- .../distributed/dht/GridDhtCacheEntry.java | 6 +- .../dht/GridDhtLocalPartition.java | 4 +- .../dht/GridDhtOffHeapCacheEntry.java | 6 +- .../dht/GridPartitionedGetFuture.java | 166 +++++--- .../dht/GridPartitionedSingleGetFuture.java | 153 ++++--- .../dht/atomic/GridDhtAtomicCache.java | 237 ++++++----- .../dht/atomic/GridDhtAtomicCacheEntry.java | 6 +- .../GridDhtAtomicOffHeapCacheEntry.java | 6 +- .../dht/colocated/GridDhtColocatedCache.java | 232 ++++++----- .../colocated/GridDhtColocatedCacheEntry.java | 6 +- .../GridDhtColocatedOffHeapCacheEntry.java | 6 +- .../colocated/GridDhtDetachedCacheEntry.java | 8 +- .../near/GridNearCacheAdapter.java | 7 +- .../distributed/near/GridNearCacheEntry.java | 6 +- .../near/GridNearOffHeapCacheEntry.java | 6 +- .../cache/local/GridLocalCache.java | 5 +- .../cache/local/GridLocalCacheEntry.java | 6 +- .../local/atomic/GridLocalAtomicCache.java | 195 +++++---- .../transactions/IgniteTxLocalAdapter.java | 8 +- .../cache/CacheRebalancingSelfTest.java | 17 +- .../cache/IgniteCacheNoSyncForGetTest.java | 393 ++++++++++++++++++ .../IgniteCacheExpiryPolicyAbstractTest.java | 2 +- .../hashmap/GridHashMapLoadTest.java | 3 +- .../testsuites/IgniteCacheTestSuite2.java | 7 +- .../cache/ttl/CacheTtlAbstractSelfTest.java | 2 +- 39 files changed, 1262 insertions(+), 530 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNoSyncForGetTest.java diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java index 8d3ab74a0dee0..9d8a93f86b135 100644 --- a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java @@ -56,6 +56,7 @@ import org.apache.ignite.internal.processors.cache.query.GridCacheSqlMetadata; import org.apache.ignite.internal.processors.rest.handlers.GridRestCommandHandler; import org.apache.ignite.internal.processors.rest.protocols.http.jetty.GridJettyObjectMapper; +import org.apache.ignite.internal.util.lang.GridAbsPredicate; import org.apache.ignite.internal.util.lang.GridTuple3; import org.apache.ignite.internal.util.typedef.C1; import org.apache.ignite.internal.util.typedef.F; @@ -702,6 +703,8 @@ public void testPutWithExpiration() throws Exception { Thread.sleep(2100); + waitExpired("putKey"); + assertNull(jcache().get("putKey")); } @@ -733,6 +736,8 @@ public void testAddWithExpiration() throws Exception { Thread.sleep(2100); + waitExpired("addKey"); + assertNull(jcache().get("addKey")); } @@ -865,6 +870,8 @@ public void testReplaceWithExpiration() throws Exception { // Use larger value to avoid false positives. Thread.sleep(2100); + waitExpired("replaceKey"); + assertNull(jcache().get("replaceKey")); } @@ -2270,6 +2277,18 @@ private static String concat(Object[] vals, String delim) { } } + /** + * @param key Key. + * @throws Exception Exception. + */ + private void waitExpired(final Object key) throws Exception { + GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + return jcache().get(key) == null; + } + }, 5000); + } + /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(gridName); 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 8c3c5d16bd514..aaea4f08eacb0 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 @@ -973,8 +973,8 @@ public GridCacheEntryEx entryEx(KeyCacheObject key, AffinityTopologyVersion topV cur = map.putEntryIfObsoleteOrAbsent( topVer, key, - null, - create, touch); + create, + touch); } return cur; @@ -1908,61 +1908,105 @@ protected final IgniteInternalFuture> getAllAsync0( Map misses = null; + boolean offheapRead = ctx.offheapRead(expiry, readerArgs != null); + for (KeyCacheObject key : keys) { while (true) { - GridCacheEntryEx entry = needEntry ? entryEx(key) : peekEx(key); - - if (entry == null) { - if (!skipVals && ctx.config().isStatisticsEnabled()) - ctx.cache().metrics0().onRead(false); - - break; - } - try { - EntryGetResult res; + EntryGetResult res = null; boolean evt = !skipVals; boolean updateMetrics = !skipVals; - if (storeEnabled) { - res = entry.innerGetAndReserveForLoad(ctx.isSwapOrOffheapEnabled(), - updateMetrics, - evt, - subjId, - taskName, - expiry, - !deserializeBinary, - readerArgs); + GridCacheEntryEx entry = null; + + boolean skipEntry = false; - assert res != null; + if (offheapRead) { + GridCacheSwapEntry swapEntry = ctx.swap().readSwapEntry(key); - if (res.value() == null) { - if (misses == null) - misses = new HashMap<>(); + if (swapEntry != null) { + long expireTime = swapEntry.expireTime(); - misses.put(key, res); + if (expireTime != 0) { + if (expireTime > U.currentTimeMillis()) { + res = new EntryGetWithTtlResult(swapEntry.value(), + swapEntry.version(), + false, + expireTime, + swapEntry.ttl()); + } + } + else + res = new EntryGetResult(swapEntry.value(), swapEntry.version(), false); + } + + if (res != null) { + skipEntry = true; - res = null; + if (evt) { + ctx.events().readEvent(key, + null, + swapEntry.value(), + subjId, + taskName, + !deserializeBinary); + } + + if (updateMetrics && ctx.cache().configuration().isStatisticsEnabled()) + ctx.cache().metrics0().onRead(true); } } - else { - res = entry.innerGetVersioned( - null, - null, - ctx.isSwapOrOffheapEnabled(), - /*unmarshal*/true, - updateMetrics, - evt, - subjId, - null, - taskName, - expiry, - !deserializeBinary, - readerArgs); - - if (res == null) - ctx.evicts().touch(entry, topVer); + + if (!skipEntry) { + entry = needEntry ? entryEx(key) : peekEx(key); + + if (entry == null) { + if (!skipVals && ctx.config().isStatisticsEnabled()) + ctx.cache().metrics0().onRead(false); + + break; + } + + if (storeEnabled) { + res = entry.innerGetAndReserveForLoad(ctx.isSwapOrOffheapEnabled(), + updateMetrics, + evt, + subjId, + taskName, + expiry, + !deserializeBinary, + readerArgs); + + assert res != null; + + if (res.value() == null) { + if (misses == null) + misses = new HashMap<>(); + + misses.put(key, res); + + res = null; + } + } + else { + res = entry.innerGetVersioned( + null, + null, + ctx.isSwapOrOffheapEnabled(), + /*unmarshal*/true, + updateMetrics, + evt, + subjId, + null, + taskName, + expiry, + !deserializeBinary, + readerArgs); + + if (res == null) + ctx.evicts().touch(entry, topVer); + } } if (res != null) { @@ -1975,7 +2019,7 @@ protected final IgniteInternalFuture> getAllAsync0( true, needVer); - if (tx == null || (!tx.implicit() && tx.isolation() == READ_COMMITTED)) + if (entry != null && (tx == null || (!tx.implicit() && tx.isolation() == READ_COMMITTED))) ctx.evicts().touch(entry, topVer); if (keysSize == 1) @@ -5773,6 +5817,10 @@ protected abstract static class CacheExpiryPolicy implements IgniteCacheExpiryPo return CU.toTtl(expiryPlc.getExpiryForAccess()); } + @Override public boolean hasAccessTtl() { + return CU.toTtl(expiryPlc.getExpiryForAccess()) != CU.TTL_NOT_CHANGED; + } + @Override public long forCreate() { return CU.toTtl(expiryPlc.getExpiryForCreation()); } @@ -5801,6 +5849,10 @@ protected abstract static class CacheExpiryPolicy implements IgniteCacheExpiryPo return accessTtl; } + @Override public boolean hasAccessTtl() { + return accessTtl != CU.TTL_NOT_CHANGED; + } + /** {@inheritDoc} */ @Override public long forUpdate() { return CU.TTL_NOT_CHANGED; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMap.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMap.java index e733114f4d95b..0b6d5ab4d71e1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMap.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMap.java @@ -38,7 +38,6 @@ public interface GridCacheConcurrentMap { /** * @param topVer Topology version. * @param key Key. - * @param val Value. * @param create Create flag. * @return Triple where the first element is current entry associated with the key, * the second is created entry and the third is doomed (all may be null). @@ -46,7 +45,6 @@ public interface GridCacheConcurrentMap { @Nullable public GridCacheMapEntry putEntryIfObsoleteOrAbsent( AffinityTopologyVersion topVer, KeyCacheObject key, - @Nullable CacheObject val, boolean create, boolean touch); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMapImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMapImpl.java index 48dae76fca730..3830d476bb4ab 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMapImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMapImpl.java @@ -110,8 +110,7 @@ public GridCacheConcurrentMapImpl( /** {@inheritDoc} */ @Nullable @Override public GridCacheMapEntry putEntryIfObsoleteOrAbsent(final AffinityTopologyVersion topVer, - KeyCacheObject key, @Nullable final CacheObject val, final boolean create, final boolean touch) { - + KeyCacheObject key, final boolean create, final boolean touch) { GridCacheMapEntry cur = null; GridCacheMapEntry created = null; GridCacheMapEntry created0 = null; @@ -127,7 +126,7 @@ public GridCacheConcurrentMapImpl( if (entry == null) { if (create) { if (created0 == null) - created0 = factory.create(ctx, topVer, key, key.hashCode(), val); + created0 = factory.create(ctx, topVer, key, key.hashCode()); cur = created = created0; @@ -142,7 +141,7 @@ public GridCacheConcurrentMapImpl( if (create) { if (created0 == null) - created0 = factory.create(ctx, topVer, key, key.hashCode(), val); + created0 = factory.create(ctx, topVer, key, key.hashCode()); cur = created = created0; 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 3b44b5096f0b0..184f467d261c8 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 @@ -2057,6 +2057,21 @@ public Collection cacheKeysView(Collection keys) { }); } + /** + * Checks if it is possible to directly read offheap memory without entry creation (this + * is optimization to avoid unnecessary blocking synchronization on cache entry). + * + * @param expiryPlc Expiry policy for read operation. + * @param readers {@code True} if need update entry readers. + * @return {@code True} if it is possible directly read offheap instead of using {@link GridCacheEntryEx#innerGet}. + */ + public boolean offheapRead(IgniteCacheExpiryPolicy expiryPlc, boolean readers) { + return offheapTiered() && + isSwapOrOffheapEnabled() && + (expiryPlc == null || !expiryPlc.hasAccessTtl()) && + !readers; + } + /** * @param part Partition. * @param affNodes Affinity nodes. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java index 1c1873814e675..7a417d5bbca9f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java @@ -31,6 +31,7 @@ import org.apache.ignite.lang.IgniteUuid; import org.jetbrains.annotations.Nullable; +import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_READ; import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_PART_UNLOADED; import static org.apache.ignite.events.EventType.EVT_CACHE_STARTED; import static org.apache.ignite.events.EventType.EVT_CACHE_STOPPED; @@ -61,6 +62,37 @@ public void removeListener(GridLocalEventListener lsnr) { cctx.gridEvents().removeLocalEventListener(lsnr); } + /** + * @param key Key for event. + * @param tx Possible surrounding transaction. + * @param val Read value. + * @param subjId Subject ID. + * @param taskName Task name. + * @param keepBinary Keep binary flag. + */ + public void readEvent(KeyCacheObject key, + @Nullable IgniteInternalTx tx, + @Nullable CacheObject val, + @Nullable UUID subjId, + @Nullable String taskName, + boolean keepBinary) { + if (isRecordable(EVT_CACHE_OBJECT_READ)) { + addEvent(cctx.affinity().partition(key), + key, + tx, + null, + EVT_CACHE_OBJECT_READ, + val, + val != null, + val, + val != null, + subjId, + null, + taskName, + keepBinary); + } + } + /** * @param part Partition. * @param key Key for the event. 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 2237e227b60e3..d6d81de9d19a1 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 @@ -172,13 +172,11 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme * @param cctx Cache context. * @param key Cache key. * @param hash Key hash value. - * @param val Entry value. */ protected GridCacheMapEntry( GridCacheContext cctx, KeyCacheObject key, - int hash, - CacheObject val + int hash ) { if (log == null) log = U.logger(cctx.kernalContext(), logRef, GridCacheMapEntry.class); @@ -191,12 +189,6 @@ protected GridCacheMapEntry( this.hash = hash; this.cctx = cctx; - val = cctx.kernalContext().cacheObjects().prepareForCache(val, cctx); - - synchronized (this) { - value(val); - } - ver = cctx.versions().next(); startVer = ver.order(); @@ -388,7 +380,7 @@ protected CacheObject valueBytesUnlocked() { /** * @return {@code True} if start version. */ - public boolean isStartVersion() { + private boolean isStartVersion() { return ver.nodeOrder() == cctx.localNode().order() && ver.order() == startVer; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntryFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntryFactory.java index 4ee938532c58c..0b49f791da5ca 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntryFactory.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntryFactory.java @@ -28,14 +28,12 @@ public interface GridCacheMapEntryFactory { * @param topVer Topology version. * @param key Cache key. * @param hash Key hash value. - * @param val Entry value. * @return New cache entry. */ public GridCacheMapEntry create( GridCacheContext ctx, AffinityTopologyVersion topVer, KeyCacheObject key, - int hash, - CacheObject val + int hash ); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java index 159b3b85c9fbc..67c00dc99bd64 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java @@ -25,6 +25,7 @@ import java.util.Collections; import java.util.Iterator; import java.util.LinkedHashMap; +import java.util.List; import java.util.Map; import java.util.NoSuchElementException; import java.util.Set; @@ -162,6 +163,8 @@ public void unwindOffheapEvicts() { Collection> evicts = offheapEvicts.get(); if (evicts != null) { + List entries = null; + GridCacheVersion obsoleteVer = cctx.versions().next(); for (IgniteBiTuple t : evicts) { @@ -179,6 +182,12 @@ public void unwindOffheapEvicts() { try { if (entry.onOffheapEvict(vb, evictVer, obsoleteVer)) cctx.cache().removeEntry(entry); + else { + if (entries == null) + entries = new ArrayList<>(); + + entries.add(entry); + } break; } @@ -196,6 +205,11 @@ public void unwindOffheapEvicts() { } offheapEvicts.set(null); + + if (entries != null) { + for (int i = 0; i < entries.size(); i++) + cctx.evicts().touch(entries.get(i), AffinityTopologyVersion.NONE); + } } } @@ -818,6 +832,28 @@ public boolean containsKey(KeyCacheObject key, int part) throws IgniteCheckedExc readOffheap, readSwap, valOnly); } + /** + * @param key Key. + * @return Read value. + * @throws IgniteCheckedException If failed. + */ + @Nullable public GridCacheSwapEntry readSwapEntry(KeyCacheObject key) throws IgniteCheckedException { + assert offheapEnabled || swapEnabled; + + GridCacheSwapEntry entry = read(key, + key.valueBytes(cctx.cacheObjectContext()), + cctx.affinity().partition(key), + false, + true, + true, + false); + + assert entry == null || entry.value() != null : entry; + assert entry == null || entry.version() != null : entry; + + return entry; + } + /** * @param entry Entry to read. * @return Read value address. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridNoStorageCacheMap.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridNoStorageCacheMap.java index 253288220d030..17bdfeea63a6d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridNoStorageCacheMap.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridNoStorageCacheMap.java @@ -46,11 +46,11 @@ public GridNoStorageCacheMap(GridCacheContext ctx) { /** {@inheritDoc} */ @Override public GridCacheMapEntry putEntryIfObsoleteOrAbsent(AffinityTopologyVersion topVer, KeyCacheObject key, - @Nullable CacheObject val, boolean create, boolean touch) { + boolean create, boolean touch) { if (create) return ctx.useOffheapEntry() ? - new GridDhtOffHeapCacheEntry(ctx, topVer, key, key.hashCode(), val) : - new GridDhtCacheEntry(ctx, topVer, key, key.hashCode(), val); + new GridDhtOffHeapCacheEntry(ctx, topVer, key, key.hashCode()) : + new GridDhtCacheEntry(ctx, topVer, key, key.hashCode()); else return null; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheExpiryPolicy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheExpiryPolicy.java index f82c5f0289132..96f1c6fa693a2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheExpiryPolicy.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheExpiryPolicy.java @@ -45,6 +45,11 @@ public interface IgniteCacheExpiryPolicy { */ public long forAccess(); + /** + * @return {@code True} if expiry policy change ttl on entry read. + */ + public boolean hasAccessTtl(); + /** * Callback for ttl update on entry access. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java index f518934500144..28cd2029245c5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java @@ -50,15 +50,13 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry { * @param ctx Cache context. * @param key Cache key. * @param hash Key hash value. - * @param val Entry value. */ public GridDistributedCacheEntry( GridCacheContext ctx, KeyCacheObject key, - int hash, - CacheObject val + int hash ) { - super(ctx, key, hash, val); + super(ctx, key, hash); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedConcurrentMap.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedConcurrentMap.java index cfbe9bbe782af..d59bf2dc5d41a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedConcurrentMap.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedConcurrentMap.java @@ -80,13 +80,13 @@ public GridCachePartitionedConcurrentMap(GridCacheContext ctx) { /** {@inheritDoc} */ @Override public GridCacheMapEntry putEntryIfObsoleteOrAbsent(AffinityTopologyVersion topVer, KeyCacheObject key, - @Nullable CacheObject val, boolean create, boolean touch) { + boolean create, boolean touch) { GridDhtLocalPartition part = localPartition(key, topVer, create); if (part == null) return null; - return part.putEntryIfObsoleteOrAbsent(topVer, key, val, create, touch); + return part.putEntryIfObsoleteOrAbsent(topVer, key, create, touch); } /** {@inheritDoc} */ 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 a8cc599d50f2c..f8932867552bb 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 @@ -247,13 +247,12 @@ protected GridCacheMapEntryFactory entryFactory() { GridCacheContext ctx, AffinityTopologyVersion topVer, KeyCacheObject key, - int hash, - CacheObject val + int hash ) { if (ctx.useOffheapEntry()) - return new GridDhtOffHeapCacheEntry(ctx, topVer, key, hash, val); + return new GridDhtOffHeapCacheEntry(ctx, topVer, key, hash); - return new GridDhtCacheEntry(ctx, topVer, key, hash, val); + return new GridDhtCacheEntry(ctx, topVer, key, hash); } }; } @@ -441,7 +440,7 @@ public GridDhtCacheEntry entryExx(KeyCacheObject key, * @return Cache entry. */ protected GridDistributedCacheEntry createEntry(KeyCacheObject key) { - return new GridDhtDetachedCacheEntry(ctx, key, key.hashCode(), null, null, 0); + return new GridDhtDetachedCacheEntry(ctx, key, key.hashCode()); } /** {@inheritDoc} */ 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 39571ff4720e6..44720500de105 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 @@ -77,16 +77,14 @@ public class GridDhtCacheEntry extends GridDistributedCacheEntry { * @param topVer Topology version at the time of creation (if negative, then latest topology is assumed). * @param key Cache key. * @param hash Key hash value. - * @param val Entry value. */ public GridDhtCacheEntry( GridCacheContext ctx, AffinityTopologyVersion topVer, KeyCacheObject key, - int hash, - CacheObject val + int hash ) { - super(ctx, key, hash, val); + super(ctx, key, hash); // Record this entry with partition. int p = cctx.affinity().partition(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 9f8498ad357f7..4475d48b2ac73 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 @@ -281,8 +281,8 @@ public boolean valid() { /** {@inheritDoc} */ @Override public GridCacheMapEntry putEntryIfObsoleteOrAbsent( AffinityTopologyVersion topVer, KeyCacheObject key, - @Nullable CacheObject val, boolean create, boolean touch) { - return map.putEntryIfObsoleteOrAbsent(topVer, key, val, create, touch); + boolean create, boolean touch) { + return map.putEntryIfObsoleteOrAbsent(topVer, key, create, touch); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtOffHeapCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtOffHeapCacheEntry.java index 75ee45cb4798a..043b78f13dc08 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtOffHeapCacheEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtOffHeapCacheEntry.java @@ -35,16 +35,14 @@ public class GridDhtOffHeapCacheEntry extends GridDhtCacheEntry { * @param topVer Topology version at the time of creation (if negative, then latest topology is assumed). * @param key Cache key. * @param hash Key hash value. - * @param val Entry value. */ public GridDhtOffHeapCacheEntry( GridCacheContext ctx, AffinityTopologyVersion topVer, KeyCacheObject key, - int hash, - CacheObject val + int hash ) { - super(ctx, topVer, key, hash, val); + super(ctx, topVer, key, hash); } /** {@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 519239aecb532..6f3f776e57db8 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 @@ -39,6 +39,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheEntryInfo; import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException; import org.apache.ignite.internal.processors.cache.GridCacheMessage; +import org.apache.ignite.internal.processors.cache.GridCacheSwapEntry; import org.apache.ignite.internal.processors.cache.IgniteCacheExpiryPolicy; import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetRequest; @@ -437,81 +438,114 @@ private boolean localGet(KeyCacheObject key, int part, Map locVals) { GridDhtCacheAdapter cache = cache(); - while (true) { - GridCacheEntryEx entry; + boolean offheapRead = cctx.offheapRead(expiryPlc, false); + boolean evt = !skipVals; + while (true) { try { - entry = cache.context().isSwapOrOffheapEnabled() ? cache.entryEx(key) : cache.peekEx(key); - - // If our DHT cache do has value, then we peek it. - if (entry != null) { - boolean isNew = entry.isNewLocked(); - - EntryGetResult getRes = null; - CacheObject v = null; - GridCacheVersion ver = null; - - if (needVer) { - getRes = entry.innerGetVersioned( - null, - null, - /*swap*/true, - /*unmarshal*/true, - /**update-metrics*/false, - /*event*/!skipVals, - subjId, - null, - taskName, - expiryPlc, - !deserializeBinary, - null); - - if (getRes != null) { - v = getRes.value(); - ver = getRes.version(); + boolean skipEntry = false; + + EntryGetResult getRes = null; + CacheObject v = null; + GridCacheVersion ver = null; + + if (offheapRead) { + GridCacheSwapEntry swapEntry = cctx.swap().readSwapEntry(key); + + if (swapEntry != null) { + long expireTime = swapEntry.expireTime(); + + if (expireTime == 0 || expireTime > U.currentTimeMillis()) { + skipEntry = true; + + v = swapEntry.value(); + + if (needVer) + ver = swapEntry.version(); + + if (evt) { + cctx.events().readEvent(key, + null, + swapEntry.value(), + subjId, + taskName, + !deserializeBinary); + } } } - else { - v = entry.innerGet( - null, - null, - /*swap*/true, - /*read-through*/false, - /**update-metrics*/false, - /*event*/!skipVals, - /*temporary*/false, - subjId, - null, - taskName, - expiryPlc, - !deserializeBinary); - } + } + + if (!skipEntry) { + GridCacheEntryEx entry = + cache.context().isSwapOrOffheapEnabled() ? cache.entryEx(key) : cache.peekEx(key); + + // If our DHT cache do has value, then we peek it. + if (entry != null) { + boolean isNew = entry.isNewLocked(); + + if (needVer) { + getRes = entry.innerGetVersioned( + null, + null, + /*swap*/true, + /*unmarshal*/true, + /*update-metrics*/false, + /*event*/evt, + subjId, + null, + taskName, + expiryPlc, + !deserializeBinary, + null); + + if (getRes != null) { + v = getRes.value(); + ver = getRes.version(); + } + } + else { + v = entry.innerGet( + null, + null, + /*swap*/true, + /*read-through*/false, + /*update-metrics*/false, + /*event*/evt, + /*temporary*/false, + subjId, + null, + taskName, + expiryPlc, + !deserializeBinary); + } - cache.context().evicts().touch(entry, topVer); + cache.context().evicts().touch(entry, topVer); - // Entry was not in memory or in swap, so we remove it from cache. - if (v == null) { - if (isNew && entry.markObsoleteIfEmpty(ver)) - cache.removeEntry(entry); - } - else { - cctx.addResult(locVals, - key, - v, - skipVals, - keepCacheObjects, - deserializeBinary, - true, - getRes, - ver, - 0, - 0, - needVer); - - return true; + // Entry was not in memory or in swap, so we remove it from cache. + if (v == null) { + if (isNew && entry.markObsoleteIfEmpty(ver)) + cache.removeEntry(entry); + } } } + if (v != null) { + cctx.addResult(locVals, + key, + v, + skipVals, + keepCacheObjects, + deserializeBinary, + true, + getRes, + ver, + 0, + 0, + needVer); + + return true; + } + boolean topStable = cctx.isReplicated() || topVer.equals(cctx.topology().topologyVersion()); // Entry not found, do not continue search if topology did not change and there is no store. 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 a3f6b72376e8f..85ccd6b86d334 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 @@ -38,6 +38,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException; import org.apache.ignite.internal.processors.cache.GridCacheFuture; import org.apache.ignite.internal.processors.cache.GridCacheMessage; +import org.apache.ignite.internal.processors.cache.GridCacheSwapEntry; import org.apache.ignite.internal.processors.cache.IgniteCacheExpiryPolicy; import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.distributed.near.CacheVersionedValue; @@ -360,74 +361,108 @@ private boolean localGet(AffinityTopologyVersion topVer, int part) { GridDhtCacheAdapter colocated = cctx.dht(); - while (true) { - GridCacheEntryEx entry; + boolean offheapRead = cctx.offheapRead(expiryPlc, false); + boolean evt = !skipVals; + while (true) { try { - entry = colocated.context().isSwapOrOffheapEnabled() ? colocated.entryEx(key) : - colocated.peekEx(key); - - // If our DHT cache do has value, then we peek it. - if (entry != null) { - boolean isNew = entry.isNewLocked(); - - CacheObject v = null; - GridCacheVersion ver = null; - - if (needVer) { - EntryGetResult res = entry.innerGetVersioned( - null, - null, - /*swap*/true, - /*unmarshal*/true, - /**update-metrics*/false, - /*event*/!skipVals, - subjId, - null, - taskName, - expiryPlc, - true, - null); - - if (res != null) { - v = res.value(); - ver = res.version(); + CacheObject v = null; + GridCacheVersion ver = null; + + boolean skipEntry = false; + + if (offheapRead) { + GridCacheSwapEntry swapEntry = cctx.swap().readSwapEntry(key); + + if (swapEntry != null) { + long expireTime = swapEntry.expireTime(); + + if (expireTime == 0 || expireTime > U.currentTimeMillis()) { + skipEntry = true; + + v = swapEntry.value(); + + if (needVer) + ver = swapEntry.version(); + + if (evt) { + cctx.events().readEvent(key, + null, + swapEntry.value(), + subjId, + taskName, + !deserializeBinary); + } } + else + skipEntry = false; } - else { - v = entry.innerGet( - null, - null, - /*swap*/true, - /*read-through*/false, - /**update-metrics*/false, - /*event*/!skipVals, - /*temporary*/false, - subjId, - null, - taskName, - expiryPlc, - true); - } + } - colocated.context().evicts().touch(entry, topVer); + if (!skipEntry) { + GridCacheEntryEx entry = colocated.context().isSwapOrOffheapEnabled() ? colocated.entryEx(key) : + colocated.peekEx(key); + + // If our DHT cache do has value, then we peek it. + if (entry != null) { + boolean isNew = entry.isNewLocked(); + + if (needVer) { + EntryGetResult res = entry.innerGetVersioned( + null, + null, + /*swap*/true, + /*unmarshal*/true, + /*update-metrics*/false, + /*event*/evt, + subjId, + null, + taskName, + expiryPlc, + true, + null); + + if (res != null) { + v = res.value(); + ver = res.version(); + } + } + else { + v = entry.innerGet( + null, + null, + /*swap*/true, + /*read-through*/false, + /*update-metrics*/false, + /*event*/evt, + /*temporary*/false, + subjId, + null, + taskName, + expiryPlc, + true); + } + + colocated.context().evicts().touch(entry, topVer); - // Entry was not in memory or in swap, so we remove it from cache. - if (v == null) { - if (isNew && entry.markObsoleteIfEmpty(ver)) - colocated.removeEntry(entry); + if (v == null) { + // Entry was not in memory or in swap, so we remove it from cache. + if (isNew && entry.markObsoleteIfEmpty(ver)) + colocated.removeEntry(entry); + } } - else { - if (!skipVals && cctx.config().isStatisticsEnabled()) - cctx.cache().metrics0().onRead(true); + } - if (!skipVals) - setResult(v, ver); - else - setSkipValueResult(true, ver); + if (v != null) { + if (!skipVals && cctx.config().isStatisticsEnabled()) + cctx.cache().metrics0().onRead(true); - return true; - } + if (!skipVals) + setResult(v, ver); + else + setSkipValueResult(true, ver); + + return true; } boolean topStable = cctx.isReplicated() || topVer.equals(cctx.topology().topologyVersion()); 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 4159359a2f780..5497980efe9d6 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 @@ -57,6 +57,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheMapEntryFactory; import org.apache.ignite.internal.processors.cache.GridCacheOperation; import org.apache.ignite.internal.processors.cache.GridCacheReturn; +import org.apache.ignite.internal.processors.cache.GridCacheSwapEntry; import org.apache.ignite.internal.processors.cache.GridCacheUpdateAtomicResult; import org.apache.ignite.internal.processors.cache.GridDeferredAckMessageSender; import org.apache.ignite.internal.processors.cache.IgniteCacheExpiryPolicy; @@ -195,13 +196,12 @@ public GridDhtAtomicCache(GridCacheContext ctx, GridCacheConcurrentMap map GridCacheContext ctx, AffinityTopologyVersion topVer, KeyCacheObject key, - int hash, - CacheObject val + int hash ) { if (ctx.useOffheapEntry()) - return new GridDhtAtomicOffHeapCacheEntry(ctx, topVer, key, hash, val); + return new GridDhtAtomicOffHeapCacheEntry(ctx, topVer, key, hash); - return new GridDhtAtomicCacheEntry(ctx, topVer, key, hash, val); + return new GridDhtAtomicCacheEntry(ctx, topVer, key, hash); } }; } @@ -1566,114 +1566,165 @@ private IgniteInternalFuture> getAllAsync0(@Nullable Collection locVals = U.newHashMap(keys.size()); - - boolean success = true; - - // Optimistically expect that all keys are available locally (avoid creation of get future). - for (KeyCacheObject key : keys) { - GridCacheEntryEx entry = null; - - while (true) { - try { - entry = ctx.isSwapOrOffheapEnabled() ? entryEx(key) : peekEx(key); - - // If our DHT cache do has value, then we peek it. - if (entry != null) { - boolean isNew = entry.isNewLocked(); - - EntryGetResult getRes = null; - CacheObject v = null; - GridCacheVersion ver = null; - - if (needVer) { - getRes = entry.innerGetVersioned( - null, - null, - /*swap*/true, - /*unmarshal*/true, - /**update-metrics*/false, - /*event*/!skipVals, - subjId, - null, - taskName, - expiry, + try { + Map locVals = U.newHashMap(keys.size()); + + boolean success = true; + boolean offheapRead = ctx.offheapRead(expiry, false); + + // Optimistically expect that all keys are available locally (avoid creation of get future). + for (KeyCacheObject key : keys) { + if (offheapRead) { + GridCacheSwapEntry swapEntry = ctx.swap().readSwapEntry(key); + + if (swapEntry != null) { + long expireTime = swapEntry.expireTime(); + + if (expireTime == 0 || expireTime > U.currentTimeMillis()) { + ctx.addResult(locVals, + key, + swapEntry.value(), + skipVals, + false, + deserializeBinary, true, - null); - - if (getRes != null) { - v = getRes.value(); - ver = getRes.version(); - } - } - else { - v = entry.innerGet(null, - null, - /*swap*/true, - /*read-through*/false, - /**update-metrics*/false, - /*event*/!skipVals, - /*temporary*/false, - subjId, null, - taskName, - expiry, - !deserializeBinary); - } - - // Entry was not in memory or in swap, so we remove it from cache. - if (v == null) { - GridCacheVersion obsoleteVer = context().versions().next(); - - if (isNew && entry.markObsoleteIfEmpty(obsoleteVer)) - removeEntry(entry); - - success = false; + swapEntry.version(), + 0, + 0, + needVer); + + if (evt) { + ctx.events().readEvent(key, + null, + swapEntry.value(), + subjId, + taskName, + !deserializeBinary); + } } else - ctx.addResult(locVals, key, v, skipVals, false, deserializeBinary, true, - getRes, ver, 0, 0, needVer); + success = false; } else success = false; - - break; // While. } - catch (GridCacheEntryRemovedException ignored) { - // No-op, retry. - } - catch (GridDhtInvalidPartitionException ignored) { - success = false; + else { + GridCacheEntryEx entry = null; + + while (true) { + try { + entry = ctx.isSwapOrOffheapEnabled() ? entryEx(key) : peekEx(key); + + // If our DHT cache do has value, then we peek it. + if (entry != null) { + boolean isNew = entry.isNewLocked(); + + EntryGetResult getRes = null; + CacheObject v = null; + GridCacheVersion ver = null; + + if (needVer) { + getRes = entry.innerGetVersioned( + null, + null, + /*swap*/true, + /*unmarshal*/true, + /*update-metrics*/false, + /*event*/evt, + subjId, + null, + taskName, + expiry, + true, + null); + + if (getRes != null) { + v = getRes.value(); + ver = getRes.version(); + } + } + else { + v = entry.innerGet(null, + null, + /*swap*/true, + /*read-through*/false, + /*update-metrics*/false, + /*event*/evt, + /*temporary*/false, + subjId, + null, + taskName, + expiry, + !deserializeBinary); + } + + // Entry was not in memory or in swap, so we remove it from cache. + if (v == null) { + if (isNew && entry.markObsoleteIfEmpty(context().versions().next())) + removeEntry(entry); + + success = false; + } + else { + ctx.addResult(locVals, + key, + v, + skipVals, + false, + deserializeBinary, + true, + getRes, + ver, + 0, + 0, + needVer); + } + } + else + success = false; - break; // While. - } - catch (IgniteCheckedException e) { - return new GridFinishedFuture<>(e); - } - finally { - if (entry != null) - ctx.evicts().touch(entry, topVer); + break; // While. + } + catch (GridCacheEntryRemovedException ignored) { + // No-op, retry. + } + catch (GridDhtInvalidPartitionException ignored) { + success = false; + + break; // While. + } + finally { + if (entry != null) + ctx.evicts().touch(entry, topVer); + } + } + + if (!success) + break; + else if (!skipVals && ctx.config().isStatisticsEnabled()) + metrics0().onRead(true); } } - if (!success) - break; - else if (!skipVals && ctx.config().isStatisticsEnabled()) - metrics0().onRead(true); - } + if (success) { + sendTtlUpdateRequest(expiry); - if (success) { - sendTtlUpdateRequest(expiry); + return new GridFinishedFuture<>(locVals); + } - return new GridFinishedFuture<>(locVals); + if (expiry != null) + expiry.reset(); + } + catch (IgniteCheckedException e) { + return new GridFinishedFuture<>(e); } } - if (expiry != null) - expiry.reset(); - // Either reload or not all values are available locally. GridPartitionedGetFuture fut = new GridPartitionedGetFuture<>(ctx, keys, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCacheEntry.java index 3f014d54f6ef5..15278d7ffe85f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCacheEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCacheEntry.java @@ -34,16 +34,14 @@ public class GridDhtAtomicCacheEntry extends GridDhtCacheEntry { * @param topVer Topology version at the time of creation (if negative, then latest topology is assumed). * @param key Cache key. * @param hash Key hash value. - * @param val Entry value. */ public GridDhtAtomicCacheEntry( GridCacheContext ctx, AffinityTopologyVersion topVer, KeyCacheObject key, - int hash, - CacheObject val + int hash ) { - super(ctx, topVer, key, hash, val); + super(ctx, topVer, key, hash); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicOffHeapCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicOffHeapCacheEntry.java index c86d477b1924b..d00493df78c5c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicOffHeapCacheEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicOffHeapCacheEntry.java @@ -35,16 +35,14 @@ public class GridDhtAtomicOffHeapCacheEntry extends GridDhtAtomicCacheEntry { * @param topVer Topology version at the time of creation (if negative, then latest topology is assumed). * @param key Cache key. * @param hash Key hash value. - * @param val Entry value. */ public GridDhtAtomicOffHeapCacheEntry( GridCacheContext ctx, AffinityTopologyVersion topVer, KeyCacheObject key, - int hash, - CacheObject val + int hash ) { - super(ctx, topVer, key, hash, val); + super(ctx, topVer, key, hash); } /** {@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 e1e0ec2bf087e..c09ad9b03ec2d 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 @@ -42,6 +42,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheMapEntryFactory; import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate; import org.apache.ignite.internal.processors.cache.GridCacheReturn; +import org.apache.ignite.internal.processors.cache.GridCacheSwapEntry; import org.apache.ignite.internal.processors.cache.IgniteCacheExpiryPolicy; import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.distributed.GridDistributedCacheEntry; @@ -123,13 +124,12 @@ public GridDhtColocatedCache(GridCacheContext ctx, GridCacheConcurrentMap GridCacheContext ctx, AffinityTopologyVersion topVer, KeyCacheObject key, - int hash, - CacheObject val + int hash ) { if (ctx.useOffheapEntry()) - return new GridDhtColocatedOffHeapCacheEntry(ctx, topVer, key, hash, val); + return new GridDhtColocatedOffHeapCacheEntry(ctx, topVer, key, hash); - return new GridDhtColocatedCacheEntry(ctx, topVer, key, hash, val); + return new GridDhtColocatedCacheEntry(ctx, topVer, key, hash); } }; } @@ -452,121 +452,165 @@ public final IgniteInternalFuture> loadAsync( // Optimisation: try to resolve value locally and escape 'get future' creation. if (!forcePrimary) { - Map locVals = null; + try { + Map locVals = null; - boolean success = true; + boolean success = true; + boolean offheapRead = ctx.offheapRead(expiryPlc, false); + boolean evt = !skipVals; - // Optimistically expect that all keys are available locally (avoid creation of get future). - for (KeyCacheObject key : keys) { - GridCacheEntryEx entry = null; - - while (true) { - try { - entry = ctx.isSwapOrOffheapEnabled() ? entryEx(key) : peekEx(key); - - // If our DHT cache do has value, then we peek it. - if (entry != null) { - boolean isNew = entry.isNewLocked(); - - EntryGetResult getRes = null; - CacheObject v = null; - GridCacheVersion ver = null; - - if (needVer) { - getRes = entry.innerGetVersioned( - null, - null, - /*swap*/true, - /*unmarshal*/true, - /**update-metrics*/false, - /*event*/!skipVals, - subjId, - null, - taskName, - expiryPlc, - !deserializeBinary, - null); - - if (getRes != null) { - v = getRes.value(); - ver = getRes.version(); - } - } - else { - v = entry.innerGet( - null, - null, - /*swap*/true, - /*read-through*/false, - /**update-metrics*/false, - /*event*/!skipVals, - /*temporary*/false, - subjId, - null, - taskName, - expiryPlc, - !deserializeBinary); - } - - // Entry was not in memory or in swap, so we remove it from cache. - if (v == null) { - GridCacheVersion obsoleteVer = context().versions().next(); + // Optimistically expect that all keys are available locally (avoid creation of get future). + for (KeyCacheObject key : keys) { + if (offheapRead) { + GridCacheSwapEntry swapEntry = ctx.swap().readSwapEntry(key); - if (isNew && entry.markObsoleteIfEmpty(obsoleteVer)) - removeEntry(entry); + if (swapEntry != null) { + long expireTime = swapEntry.expireTime(); - success = false; - } - else { + if (expireTime == 0 || expireTime > U.currentTimeMillis()) { if (locVals == null) locVals = U.newHashMap(keys.size()); ctx.addResult(locVals, key, - v, + swapEntry.value(), skipVals, - keepCacheObj, + false, deserializeBinary, true, - getRes, - ver, + null, + swapEntry.version(), 0, 0, needVer); + + if (evt) { + ctx.events().readEvent(key, + null, + swapEntry.value(), + subjId, + taskName, + !deserializeBinary); + } } + else + success = false; } else success = false; - - break; // While. - } - catch (GridCacheEntryRemovedException ignored) { - // No-op, retry. } - catch (GridDhtInvalidPartitionException ignored) { - success = false; + else { + GridCacheEntryEx entry = null; + + while (true) { + try { + entry = ctx.isSwapOrOffheapEnabled() ? entryEx(key) : peekEx(key); + + // If our DHT cache do has value, then we peek it. + if (entry != null) { + boolean isNew = entry.isNewLocked(); + + EntryGetResult getRes = null; + CacheObject v = null; + GridCacheVersion ver = null; + + if (needVer) { + getRes = entry.innerGetVersioned( + null, + null, + /*swap*/true, + /*unmarshal*/true, + /*update-metrics*/false, + /*event*/evt, + subjId, + null, + taskName, + expiryPlc, + !deserializeBinary, + null); + + if (getRes != null) { + v = getRes.value(); + ver = getRes.version(); + } + } + else { + v = entry.innerGet( + null, + null, + /*swap*/true, + /*read-through*/false, + /*update-metrics*/false, + /*event*/evt, + /*temporary*/false, + subjId, + null, + taskName, + expiryPlc, + !deserializeBinary); + } + + // Entry was not in memory or in swap, so we remove it from cache. + if (v == null) { + GridCacheVersion obsoleteVer = context().versions().next(); + + if (isNew && entry.markObsoleteIfEmpty(obsoleteVer)) + removeEntry(entry); + + success = false; + } + else { + if (locVals == null) + locVals = U.newHashMap(keys.size()); + + ctx.addResult(locVals, + key, + v, + skipVals, + keepCacheObj, + deserializeBinary, + true, + getRes, + ver, + 0, + 0, + needVer); + } + } + else + success = false; - break; // While. - } - catch (IgniteCheckedException e) { - return new GridFinishedFuture<>(e); - } - finally { - if (entry != null) - context().evicts().touch(entry, topVer); + break; // While. + } + catch (GridCacheEntryRemovedException ignored) { + // No-op, retry. + } + catch (GridDhtInvalidPartitionException ignored) { + success = false; + + break; // While. + } + finally { + if (entry != null) + context().evicts().touch(entry, topVer); + } + } } - } - if (!success) - break; - else if (!skipVals && ctx.config().isStatisticsEnabled()) - ctx.cache().metrics0().onRead(true); - } + if (!success) + break; + else if (!skipVals && ctx.config().isStatisticsEnabled()) + ctx.cache().metrics0().onRead(true); + } - if (success) { - sendTtlUpdateRequest(expiryPlc); + if (success) { + sendTtlUpdateRequest(expiryPlc); - return new GridFinishedFuture<>(locVals); + return new GridFinishedFuture<>(locVals); + } + } + catch (IgniteCheckedException e) { + return new GridFinishedFuture<>(e); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCacheEntry.java index cc71e115519b8..db837a74239ac 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCacheEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCacheEntry.java @@ -33,16 +33,14 @@ public class GridDhtColocatedCacheEntry extends GridDhtCacheEntry { * @param topVer Topology version at the time of creation (if negative, then latest topology is assumed). * @param key Cache key. * @param hash Key hash value. - * @param val Entry value. */ public GridDhtColocatedCacheEntry( GridCacheContext ctx, AffinityTopologyVersion topVer, KeyCacheObject key, - int hash, - CacheObject val + int hash ) { - super(ctx, topVer, key, hash, val); + super(ctx, topVer, key, hash); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedOffHeapCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedOffHeapCacheEntry.java index 8080ca391ebdf..a6869d7d19880 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedOffHeapCacheEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedOffHeapCacheEntry.java @@ -35,16 +35,14 @@ public class GridDhtColocatedOffHeapCacheEntry extends GridDhtColocatedCacheEntr * @param topVer Topology version at the time of creation (if negative, then latest topology is assumed). * @param key Cache key. * @param hash Key hash value. - * @param val Entry value. */ public GridDhtColocatedOffHeapCacheEntry( GridCacheContext ctx, AffinityTopologyVersion topVer, KeyCacheObject key, - int hash, - CacheObject val + int hash ) { - super(ctx, topVer, key, hash, val); + super(ctx, topVer, key, hash); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java index 2e055604b2687..3ae57f2b36d37 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java @@ -35,13 +35,9 @@ public class GridDhtDetachedCacheEntry extends GridDistributedCacheEntry { * @param ctx Cache context. * @param key Cache key. * @param hash Key hash value. - * @param val Entry value. - * @param next Next entry in the linked list. - * @param hdrId Header ID. */ - public GridDhtDetachedCacheEntry(GridCacheContext ctx, KeyCacheObject key, int hash, CacheObject val, - GridCacheMapEntry next, int hdrId) { - super(ctx, key, hash, val); + public GridDhtDetachedCacheEntry(GridCacheContext ctx, KeyCacheObject key, int hash) { + super(ctx, key, hash); } /** 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..8cefe0c9287ba 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 @@ -96,15 +96,14 @@ protected GridNearCacheAdapter(GridCacheContext ctx) { GridCacheContext ctx, AffinityTopologyVersion topVer, KeyCacheObject key, - int hash, - CacheObject val + int hash ) { // Can't hold any locks here - this method is invoked when // holding write-lock on the whole cache map. if (ctx.useOffheapEntry()) - return new GridNearOffHeapCacheEntry(ctx, key, hash, val); + return new GridNearOffHeapCacheEntry(ctx, key, hash); - return new GridNearCacheEntry(ctx, key, hash, val); + return new GridNearCacheEntry(ctx, key, hash); } }; } 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 d022805a76bf2..5d449435f09bd 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 @@ -65,15 +65,13 @@ public class GridNearCacheEntry extends GridDistributedCacheEntry { * @param ctx Cache context. * @param key Cache key. * @param hash Key hash value. - * @param val Entry value. */ public GridNearCacheEntry( GridCacheContext ctx, KeyCacheObject key, - int hash, - CacheObject val + int hash ) { - super(ctx, key, hash, val); + super(ctx, key, hash); part = ctx.affinity().partition(key); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOffHeapCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOffHeapCacheEntry.java index 1558f4c9a18a7..f48256dcf1d90 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOffHeapCacheEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOffHeapCacheEntry.java @@ -32,15 +32,13 @@ public class GridNearOffHeapCacheEntry extends GridNearCacheEntry { * @param ctx Cache context. * @param key Cache key. * @param hash Key hash value. - * @param val Entry value. */ public GridNearOffHeapCacheEntry( GridCacheContext ctx, KeyCacheObject key, - int hash, - CacheObject val + int hash ) { - super(ctx, key, hash, val); + super(ctx, key, hash); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java index 5b44d759f9080..e88742ca5b4ff 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java @@ -86,10 +86,9 @@ public GridLocalCache(GridCacheContext ctx) { GridCacheContext ctx, AffinityTopologyVersion topVer, KeyCacheObject key, - int hash, - CacheObject val + int hash ) { - return new GridLocalCacheEntry(ctx, key, hash, val); + return new GridLocalCacheEntry(ctx, key, hash); } }; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCacheEntry.java index bc61333514ff8..6572672bf09fb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCacheEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCacheEntry.java @@ -42,15 +42,13 @@ public class GridLocalCacheEntry extends GridCacheMapEntry { * @param ctx Cache registry. * @param key Cache key. * @param hash Key hash value. - * @param val Entry value. */ GridLocalCacheEntry( GridCacheContext ctx, KeyCacheObject key, - int hash, - CacheObject val + int hash ) { - super(ctx, key, hash, val); + super(ctx, key, hash); } /** {@inheritDoc} */ 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 f86df2f29d684..b181e16220e37 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 @@ -22,7 +22,6 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; -import java.util.HashMap; import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; @@ -53,6 +52,7 @@ import org.apache.ignite.internal.processors.cache.GridCachePreloader; import org.apache.ignite.internal.processors.cache.GridCachePreloaderAdapter; import org.apache.ignite.internal.processors.cache.GridCacheReturn; +import org.apache.ignite.internal.processors.cache.GridCacheSwapEntry; import org.apache.ignite.internal.processors.cache.IgniteCacheExpiryPolicy; import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.local.GridLocalCache; @@ -389,7 +389,7 @@ private Map getAllInternal(@Nullable Collection keys, UUID subjId = ctx.subjectIdPerCall(null, opCtx); - Map vals = new HashMap<>(keys.size(), 1.0f); + Map vals = U.newHashMap(keys.size()); if (keyCheck) validateCacheKeys(keys); @@ -397,97 +397,144 @@ private Map getAllInternal(@Nullable Collection keys, final IgniteCacheExpiryPolicy expiry = expiryPolicy(opCtx != null ? opCtx.expiry() : null); boolean success = true; + final boolean offheapRead = ctx.offheapRead(expiry, false); + final boolean evt = !skipVals; for (K key : keys) { if (key == null) throw new NullPointerException("Null key."); - GridCacheEntryEx entry = null; - KeyCacheObject cacheKey = ctx.toCacheKeyObject(key); - while (true) { - try { - entry = swapOrOffheap ? entryEx(cacheKey) : peekEx(cacheKey); + boolean skipEntry = false; - if (entry != null) { - CacheObject v; + if (offheapRead) { + GridCacheSwapEntry swapEntry = ctx.swap().readSwapEntry(cacheKey); - if (needVer) { - EntryGetResult res = entry.innerGetVersioned( - null, - null, - /*swap*/swapOrOffheap, - /*unmarshal*/true, - /**update-metrics*/false, - /*event*/!skipVals, - subjId, - null, - taskName, - expiry, - !deserializeBinary, - null); - - if (res != null) { - ctx.addResult( - vals, - cacheKey, - res, - skipVals, - false, - deserializeBinary, - true, - needVer); - } - else - success = false; - } - else { - v = entry.innerGet( - null, + if (swapEntry != null) { + long expireTime = swapEntry.expireTime(); + + if (expireTime == 0 || expireTime > U.currentTimeMillis()) { + skipEntry = true; + + ctx.addResult(vals, + cacheKey, + swapEntry.value(), + skipVals, + false, + deserializeBinary, + true, + null, + swapEntry.version(), + 0, + 0, + needVer); + + if (configuration().isStatisticsEnabled() && !skipVals) + metrics0().onRead(true); + + if (evt) { + ctx.events().readEvent(cacheKey, null, - /*swap*/swapOrOffheap, - /*read-through*/false, - /**update-metrics*/true, - /**event*/!skipVals, - /**temporary*/false, + swapEntry.value(), subjId, - null, taskName, - expiry, !deserializeBinary); + } + } + } + else + success = false; + } + else + skipEntry = false; + + if (!skipEntry) { + GridCacheEntryEx entry = null; - if (v != null) { - ctx.addResult(vals, - cacheKey, - v, - skipVals, - false, - deserializeBinary, - true, + CacheObject v; + + while (true) { + try { + entry = swapOrOffheap ? entryEx(cacheKey) : peekEx(cacheKey); + + if (entry != null) { + if (needVer) { + EntryGetResult res = entry.innerGetVersioned( + null, null, - 0, - 0); + /*swap*/swapOrOffheap, + /*unmarshal*/true, + /*update-metrics*/false, + /*event*/!skipVals, + subjId, + null, + taskName, + expiry, + !deserializeBinary, + null); + + if (res != null) { + ctx.addResult( + vals, + cacheKey, + res, + skipVals, + false, + deserializeBinary, + true, + needVer); + } + else + success = false; + } + else { + v = entry.innerGet( + null, + null, + /*swap*/swapOrOffheap, + /*read-through*/false, + /*update-metrics*/true, + /*event*/!skipVals, + /*temporary*/false, + subjId, + null, + taskName, + expiry, + !deserializeBinary); + + if (v != null) { + ctx.addResult(vals, + cacheKey, + v, + skipVals, + false, + deserializeBinary, + true, + null, + 0, + 0); + } + else + success = false; } - else - success = false; } - } - else { - if (!storeEnabled && configuration().isStatisticsEnabled() && !skipVals) - metrics0().onRead(false); + else { + if (!storeEnabled && configuration().isStatisticsEnabled() && !skipVals) + metrics0().onRead(false); - success = false; - } + success = false; + } - break; // While. - } - catch (GridCacheEntryRemovedException ignored) { - // No-op, retry. - } - finally { - if (entry != null) - ctx.evicts().touch(entry, ctx.affinity().affinityTopologyVersion()); + break; // While. + } + catch (GridCacheEntryRemovedException ignored) { + // No-op, retry. + } + finally { + if (entry != null) + ctx.evicts().touch(entry, ctx.affinity().affinityTopologyVersion()); + } } if (!success && storeEnabled) 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 bd806aa1d5c17..76979156b0c49 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 @@ -2672,17 +2672,17 @@ private boolean enlistWriteEntry(GridCacheContext cacheCtx, /** * @param cctx Cache context. * @param key Key. - * @param val Value. + * @param val0 Value. * @param filter Filter. * @return {@code True} if filter passed. */ private boolean isAll(GridCacheContext cctx, KeyCacheObject key, - CacheObject val, + final CacheObject val0, CacheEntryPredicate[] filter) { - GridCacheEntryEx e = new GridDhtDetachedCacheEntry(cctx, key, 0, val, null, 0) { + GridCacheEntryEx e = new GridDhtDetachedCacheEntry(cctx, key, 0) { @Nullable @Override public CacheObject peekVisibleValue() { - return rawGet(); + return val0; } }; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRebalancingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRebalancingSelfTest.java index 8d1f67af2c33f..eb565b63b5eed 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRebalancingSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRebalancingSelfTest.java @@ -40,11 +40,19 @@ public class CacheRebalancingSelfTest extends GridCommonAbstractTest { return cfg; } + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + + super.afterTest(); + } + /** * @throws Exception If failed. */ public void testRebalanceFuture() throws Exception { IgniteEx ignite0 = startGrid(0); + startGrid(1); IgniteCache cache = ignite0.cache(null); @@ -63,13 +71,12 @@ public void testRebalanceFuture() throws Exception { } /** - * @param future Future. + * @param fut Future. * @return Internal future. */ - private static IgniteInternalFuture internalFuture(IgniteFuture future) { - assert future instanceof IgniteFutureImpl; + private static IgniteInternalFuture internalFuture(IgniteFuture fut) { + assert fut instanceof IgniteFutureImpl; - return ((IgniteFutureImpl)future).internalFuture(); + return ((IgniteFutureImpl) fut).internalFuture(); } - } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNoSyncForGetTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNoSyncForGetTest.java new file mode 100644 index 0000000000000..3e624a3cf0391 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNoSyncForGetTest.java @@ -0,0 +1,393 @@ +/* + * 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.util.Collection; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import javax.cache.expiry.Duration; +import javax.cache.expiry.ModifiedExpiryPolicy; +import javax.cache.processor.EntryProcessorException; +import javax.cache.processor.MutableEntry; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheEntry; +import org.apache.ignite.cache.CacheEntryProcessor; +import org.apache.ignite.cache.CacheMemoryMode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteCallable; +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.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.CacheMemoryMode.OFFHEAP_TIERED; +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; + +/** + * + */ +@SuppressWarnings("unchecked") +public class IgniteCacheNoSyncForGetTest extends GridCommonAbstractTest { + /** IP finder. */ + private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + + /** */ + private static volatile CountDownLatch processorStartLatch; + + /** */ + private static volatile CountDownLatch hangLatch; + + /** */ + private boolean client; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER); + + cfg.setClientMode(client); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + startGrid(0); + + client = true; + + startGrid(1); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + + super.afterTestsStopped(); + } + + /** + * @throws Exception If failed. + */ + public void testAtomicGetOffheap() throws Exception { + boolean getAll[] = {true, false}; + boolean cfgExpiryPlc[] = {true, false}; + boolean withExpiryPlc[] = {true, false}; + + for (boolean getAll0 : getAll) { + for (boolean expiryPlc0 : cfgExpiryPlc) + for (boolean withExpiryPlc0 : withExpiryPlc) + doGet(ATOMIC, OFFHEAP_TIERED, getAll0, expiryPlc0, withExpiryPlc0); + } + } + + /** + * @throws Exception If failed. + */ + public void testTxGetOffheap() throws Exception { + boolean getAll[] = {true, false}; + boolean cfgExpiryPlc[] = {true, false}; + boolean withExpiryPlc[] = {true, false}; + + for (boolean getAll0 : getAll) { + for (boolean expiryPlc0 : cfgExpiryPlc) + for (boolean withExpiryPlc0 : withExpiryPlc) + doGet(TRANSACTIONAL, OFFHEAP_TIERED, getAll0, expiryPlc0, withExpiryPlc0); + } + } + + /** + * @param atomicityMode Cache atomicity mode. + * @param memoryMode Cache memory mode. + * @param getAll Test getAll flag. + * @param cfgExpiryPlc Configured expiry policy flag. + * @param withExpiryPlc Custom expiry policy flag. + * @throws Exception If failed. + */ + private void doGet(CacheAtomicityMode atomicityMode, + CacheMemoryMode memoryMode, + final boolean getAll, + final boolean cfgExpiryPlc, + final boolean withExpiryPlc) throws Exception { + log.info("Test get [getAll=" + getAll + ", cfgExpiryPlc=" + cfgExpiryPlc + ']'); + + Ignite srv = ignite(0); + + Ignite client = ignite(1); + + final IgniteCache cache = client.createCache(cacheConfiguration(atomicityMode, memoryMode, cfgExpiryPlc)); + + final Map data = new HashMap<>(); + + data.put(1, 1); + data.put(2, 2); + + try { + // Get from compute closure. + { + cache.putAll(data); + + hangLatch = new CountDownLatch(1); + processorStartLatch = new CountDownLatch(1); + + IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { + @Override public Void call() throws Exception { + if (getAll) + cache.invokeAll(data.keySet(), new HangEntryProcessor()); + else + cache.invoke(1, new HangEntryProcessor()); + + return null; + } + }); + + try { + boolean wait = processorStartLatch.await(30, TimeUnit.SECONDS); + + assertTrue(wait); + + if (getAll) { + assertEquals(data, client.compute().affinityCall(cache.getName(), 1, + new GetAllClosure(data.keySet(), cache.getName(), withExpiryPlc))); + } + else { + assertEquals(1, client.compute().affinityCall(cache.getName(), 1, + new GetClosure(1, cache.getName(), withExpiryPlc))); + } + + hangLatch.countDown(); + + fut.get(); + } + finally { + hangLatch.countDown(); + } + } + + // Local get. + { + cache.putAll(data); + + hangLatch = new CountDownLatch(1); + processorStartLatch = new CountDownLatch(1); + + IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { + @Override public Void call() throws Exception { + if (getAll) + cache.invokeAll(data.keySet(), new HangEntryProcessor()); + else + cache.invoke(1, new HangEntryProcessor()); + + return null; + } + }); + + try { + boolean wait = processorStartLatch.await(30, TimeUnit.SECONDS); + + assertTrue(wait); + + IgniteCache srvCache = srv.cache(cache.getName()); + + if (withExpiryPlc) + srvCache = srvCache.withExpiryPolicy(ModifiedExpiryPolicy.factoryOf(Duration.FIVE_MINUTES).create()); + + if (getAll) { + assertEquals(data, srvCache.getAll(data.keySet())); + assertEquals(data.size(), srvCache.getEntries(data.keySet()).size()); + } + else { + assertEquals(1, srvCache.get(1)); + assertEquals(1, srvCache.getEntry(1).getValue()); + } + + hangLatch.countDown(); + + fut.get(); + } + finally { + hangLatch.countDown(); + } + } + } + finally { + client.destroyCache(cache.getName()); + } + } + + /** + * @param atomicityMode Atomicity mode. + * @param memoryMode Memory mode. + * @param expiryPlc Expiry policy flag. + * @return Cache configuration. + */ + private CacheConfiguration cacheConfiguration(CacheAtomicityMode atomicityMode, + CacheMemoryMode memoryMode, + boolean expiryPlc) { + CacheConfiguration ccfg = new CacheConfiguration(); + + ccfg.setAtomicityMode(atomicityMode); + ccfg.setMemoryMode(memoryMode); + ccfg.setWriteSynchronizationMode(FULL_SYNC); + ccfg.setName("testCache"); + + if (expiryPlc) + ccfg.setExpiryPolicyFactory(ModifiedExpiryPolicy.factoryOf(Duration.FIVE_MINUTES)); + + return ccfg; + } + + /** + * + */ + static class HangEntryProcessor implements CacheEntryProcessor { + /** {@inheritDoc} */ + @Override public Object process(MutableEntry entry, Object... arguments) { + assert processorStartLatch != null; + assert hangLatch != null; + + try { + processorStartLatch.countDown(); + + if (!hangLatch.await(60, TimeUnit.SECONDS)) + throw new RuntimeException("Failed to wait for latch"); + } + catch (Exception e) { + System.out.println("Unexpected error: " + e); + + throw new EntryProcessorException(e); + } + + entry.setValue(U.currentTimeMillis()); + + return null; + } + } + + /** + * + */ + public static class GetClosure implements IgniteCallable { + /** */ + @IgniteInstanceResource + private Ignite ignite; + + /** */ + private final int key; + + /** */ + private final String cacheName; + + /** */ + private final boolean withExpiryPlc; + + /** + * @param key Key. + * @param cacheName Cache name. + * @param withExpiryPlc Custom expiry policy flag. + */ + GetClosure(int key, String cacheName, boolean withExpiryPlc) { + this.key = key; + this.cacheName = cacheName; + this.withExpiryPlc = withExpiryPlc; + } + + /** {@inheritDoc} */ + @Override public Object call() throws Exception { + IgniteCache cache = ignite.cache(cacheName); + + if (withExpiryPlc) + cache = cache.withExpiryPolicy(ModifiedExpiryPolicy.factoryOf(Duration.FIVE_MINUTES).create()); + + Object val = cache.get(key); + + CacheEntry e = cache.getEntry(key); + + assertEquals(val, e.getValue()); + + return val; + } + } + + /** + * + */ + public static class GetAllClosure implements IgniteCallable { + /** */ + @IgniteInstanceResource + private Ignite ignite; + + /** */ + private final Set keys; + + /** */ + private final String cacheName; + + /** */ + private final boolean withExpiryPlc; + + /** + * @param keys Keys. + * @param cacheName Cache name. + * @param withExpiryPlc Custom expiry policy flag. + */ + GetAllClosure(Set keys, String cacheName, boolean withExpiryPlc) { + this.keys = keys; + this.cacheName = cacheName; + this.withExpiryPlc = withExpiryPlc; + } + + /** {@inheritDoc} */ + @Override public Object call() throws Exception { + IgniteCache cache = ignite.cache(cacheName); + + if (withExpiryPlc) + cache = cache.withExpiryPolicy(ModifiedExpiryPolicy.factoryOf(Duration.FIVE_MINUTES).create()); + + Map vals = cache.getAll(keys); + + Collection entries = cache.getEntries(keys); + + assertEquals(vals.size(), entries.size()); + + for (CacheEntry entry : entries) { + Object val = vals.get(entry.getKey()); + + assertEquals(val, entry.getValue()); + } + + return vals; + } + } +} 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 8f08ea95bad7b..423c1ba9a6d39 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 @@ -1026,7 +1026,7 @@ public void testNearExpiresOnClient() throws Exception { if(cacheMode() != PARTITIONED) return; - factory = CreatedExpiryPolicy.factoryOf(new Duration(TimeUnit.SECONDS,1)); + factory = CreatedExpiryPolicy.factoryOf(new Duration(TimeUnit.SECONDS, 2)); nearCache = true; diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridHashMapLoadTest.java b/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridHashMapLoadTest.java index 5c12f8496ee99..0e8d66bc1034f 100644 --- a/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridHashMapLoadTest.java +++ b/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridHashMapLoadTest.java @@ -79,10 +79,9 @@ public void testMapEntry() throws Exception { while (true) { Integer key = i++; - Integer val = i++; map.put(key, new GridCacheMapEntry(ctx, ctx.toCacheKeyObject(key), - key.hashCode(), ctx.toCacheObject(val)) { + key.hashCode()) { @Override public boolean tmLock(IgniteInternalTx tx, long timeout, @Nullable GridCacheVersion serOrder, diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java index e37a8a1d52096..6d0745dea8a74 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java @@ -38,6 +38,7 @@ import org.apache.ignite.internal.processors.cache.IgniteAtomicCacheEntryProcessorNodeJoinTest; import org.apache.ignite.internal.processors.cache.IgniteCacheEntryProcessorNodeJoinTest; import org.apache.ignite.internal.processors.cache.IgniteCacheIncrementTxTest; +import org.apache.ignite.internal.processors.cache.IgniteCacheNoSyncForGetTest; import org.apache.ignite.internal.processors.cache.IgniteCachePartitionMapUpdateTest; import org.apache.ignite.internal.processors.cache.IgniteDynamicCacheAndNodeStop; import org.apache.ignite.internal.processors.cache.OffheapCacheOnClientsTest; @@ -114,11 +115,11 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedTxSingleThreadedSelfTest; import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedTxTimeoutSelfTest; import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheRendezvousAffinityClientSelfTest; -import org.apache.ignite.internal.processors.cache.distributed.near.GridPartitionedBackupLoadSelfTest; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheStoreUpdateTest; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearOffheapCacheStoreUpdateTest; +import org.apache.ignite.internal.processors.cache.distributed.near.GridPartitionedBackupLoadSelfTest; import org.apache.ignite.internal.processors.cache.distributed.near.NearCacheSyncUpdateTest; import org.apache.ignite.internal.processors.cache.distributed.near.NoneRebalanceModeSelfTest; -import org.apache.ignite.internal.processors.cache.distributed.near.GridNearOffheapCacheStoreUpdateTest; import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCacheReplicatedEvictionSelfTest; import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCacheReplicatedJobExecutionTest; import org.apache.ignite.internal.processors.cache.local.GridCacheLocalAtomicBasicStoreSelfTest; @@ -278,6 +279,8 @@ public static TestSuite suite() throws Exception { suite.addTest(new TestSuite(GridNearCacheStoreUpdateTest.class)); suite.addTest(new TestSuite(GridNearOffheapCacheStoreUpdateTest.class)); + suite.addTest(new TestSuite(IgniteCacheNoSyncForGetTest.class)); + return suite; } } diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/ttl/CacheTtlAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/ttl/CacheTtlAbstractSelfTest.java index ace7d94ee7854..402230d2e5414 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/ttl/CacheTtlAbstractSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/ttl/CacheTtlAbstractSelfTest.java @@ -69,7 +69,7 @@ public abstract class CacheTtlAbstractSelfTest extends GridCommonAbstractTest { private static final int SIZE = 11; /** */ - private static final long DEFAULT_TIME_TO_LIVE = 2000; + private static final long DEFAULT_TIME_TO_LIVE = 4000; /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { From 320def0b0cc357cfae4423352623ead7aa1d5984 Mon Sep 17 00:00:00 2001 From: Alexander Belyak Date: Tue, 18 Apr 2017 14:56:50 +0300 Subject: [PATCH 046/357] IGNITE-4927: Write behind - add an option to skip write coalescing --- .../configuration/CacheConfiguration.java | 33 + .../store/GridCacheWriteBehindStore.java | 614 +++++++++++++++--- ...CacheWriteBehindStoreAbstractSelfTest.java | 24 +- ...GridCacheWriteBehindStoreAbstractTest.java | 4 + ...WriteBehindStoreMultithreadedSelfTest.java | 88 ++- .../GridCacheWriteBehindStoreSelfTest.java | 159 ++++- ...ientWriteBehindStoreNonCoalescingTest.java | 175 +++++ .../IgniteCacheWriteBehindTestSuite.java | 2 + 8 files changed, 978 insertions(+), 121 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/IgnteCacheClientWriteBehindStoreNonCoalescingTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java index e160ba9963e22..31806e73e69e7 100644 --- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java @@ -192,6 +192,9 @@ public class CacheConfiguration extends MutableConfiguration { /** Default batch size for write-behind cache store. */ public static final int DFLT_WRITE_BEHIND_BATCH_SIZE = 512; + /** Default write coalescing for write-behind cache store. */ + public static final boolean DFLT_WRITE_BEHIND_COALESCING = true; + /** Default maximum number of query iterators that can be stored. */ public static final int DFLT_MAX_QUERY_ITERATOR_CNT = 1024; @@ -344,6 +347,9 @@ public class CacheConfiguration extends MutableConfiguration { /** Maximum batch size for write-behind cache store. */ private int writeBehindBatchSize = DFLT_WRITE_BEHIND_BATCH_SIZE; + /** Write coalescing flag for write-behind cache store */ + private boolean writeBehindCoalescing = DFLT_WRITE_BEHIND_COALESCING; + /** Maximum number of query iterators that can be stored. */ private int maxQryIterCnt = DFLT_MAX_QUERY_ITERATOR_CNT; @@ -505,6 +511,7 @@ public CacheConfiguration(CompleteConfiguration cfg) { topValidator = cc.getTopologyValidator(); typeMeta = cc.getTypeMetadata(); writeBehindBatchSize = cc.getWriteBehindBatchSize(); + writeBehindCoalescing = cc.getWriteBehindCoalescing(); writeBehindEnabled = cc.isWriteBehindEnabled(); writeBehindFlushFreq = cc.getWriteBehindFlushFrequency(); writeBehindFlushSize = cc.getWriteBehindFlushSize(); @@ -1460,6 +1467,32 @@ public CacheConfiguration setWriteBehindBatchSize(int writeBehindBatchSize return this; } + /** + * Write coalescing flag for write-behind cache store operations. Store operations (get or remove) + * with the same key are combined or coalesced to single, resulting operation + * to reduce pressure to underlying cache store. + *

    + * If not provided, default value is {@link #DFLT_WRITE_BEHIND_COALESCING}. + * + * @return Write coalescing flag. + */ + public boolean getWriteBehindCoalescing() { + return writeBehindCoalescing; + } + + /** + * Sets write coalescing flag for write-behind cache. + * + * @param writeBehindCoalescing Write coalescing flag. + * @see #getWriteBehindCoalescing() + * @return {@code this} for chaining. + */ + public CacheConfiguration setWriteBehindCoalescing(boolean writeBehindCoalescing) { + this.writeBehindCoalescing = writeBehindCoalescing; + + return this; + } + /** * Use {@link IgniteConfiguration#getRebalanceThreadPoolSize()} instead. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java index 0f70da5e7e160..4a1b11e807bf3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java @@ -17,17 +17,19 @@ package org.apache.ignite.internal.processors.cache.store; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.Iterator; -import java.util.LinkedList; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.LockSupport; import java.util.concurrent.locks.ReentrantLock; import java.util.concurrent.locks.ReentrantReadWriteLock; import javax.cache.integration.CacheWriterException; @@ -43,9 +45,11 @@ import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.internal.util.worker.GridWorker; import org.apache.ignite.lang.IgniteBiInClosure; +import org.apache.ignite.lang.IgniteBiTuple; import org.apache.ignite.lifecycle.LifecycleAware; import org.apache.ignite.thread.IgniteThread; import org.jetbrains.annotations.Nullable; +import org.jsr166.ConcurrentLinkedDeque8; import org.jsr166.ConcurrentLinkedHashMap; import static javax.cache.Cache.Entry; @@ -65,6 +69,8 @@ *

    * Since write operations to the cache store are deferred, transaction support is lost; no * transaction objects are passed to the underlying store. + *

    + * {@link GridCacheWriteBehindStore} doesn't support concurrent modifications of the same key. */ public class GridCacheWriteBehindStore implements CacheStore, LifecycleAware { /** Default write cache initial capacity. */ @@ -91,6 +97,9 @@ public class GridCacheWriteBehindStore implements CacheStore, Lifecy /** Count of worker threads performing underlying store updates. */ private int flushThreadCnt = CacheConfiguration.DFLT_WRITE_FROM_BEHIND_FLUSH_THREAD_CNT; + /** Is flush threads count power of two flag. */ + private boolean flushThreadCntIsPowerOfTwo; + /** Cache flush frequency. All pending operations will be performed in not less then this value ms. */ private long cacheFlushFreq = CacheConfiguration.DFLT_WRITE_BEHIND_FLUSH_FREQUENCY; @@ -98,29 +107,26 @@ public class GridCacheWriteBehindStore implements CacheStore, Lifecy private int batchSize = CacheConfiguration.DFLT_WRITE_BEHIND_BATCH_SIZE; /** Grid name. */ - private String gridName; + private final String gridName; /** Cache name. */ - private String cacheName; + private final String cacheName; /** Underlying store. */ - private CacheStore store; + private final CacheStore store; /** Write cache. */ private ConcurrentLinkedHashMap> writeCache; /** Flusher threads. */ - private GridWorker[] flushThreads; + private Flusher[] flushThreads; + + /** Write coalescing. */ + private boolean writeCoalescing = CacheConfiguration.DFLT_WRITE_BEHIND_COALESCING; /** Atomic flag indicating store shutdown. */ private AtomicBoolean stopping = new AtomicBoolean(true); - /** Flush lock. */ - private Lock flushLock = new ReentrantLock(); - - /** Condition to determine records available for flush. */ - private Condition canFlush = flushLock.newCondition(); - /** Variable for counting total cache overflows. */ private AtomicInteger cacheTotalOverflowCntr = new AtomicInteger(); @@ -131,10 +137,16 @@ public class GridCacheWriteBehindStore implements CacheStore, Lifecy private AtomicInteger retryEntriesCnt = new AtomicInteger(); /** Log. */ - private IgniteLogger log; + private final IgniteLogger log; /** Store manager. */ - private CacheStoreManager storeMgr; + private final CacheStoreManager storeMgr; + + /** Flush lock. */ + private final Lock flushLock = new ReentrantLock(); + + /** Condition to determine records available for flush. */ + private Condition canFlush = flushLock.newCondition(); /** * Creates a write-behind cache store for the given store. @@ -193,7 +205,7 @@ public void setFlushSize(int cacheMaxSize) { *

    * If this value is {@code 0}, then flush is performed only on time-elapsing basis. However, * when this value is {@code 0}, the cache critical size is set to - * {@link CacheConfiguration#DFLT_WRITE_BEHIND_CRITICAL_SIZE} + * {@link CacheConfiguration#DFLT_WRITE_BEHIND_CRITICAL_SIZE}. * * @return Buffer size that triggers flush procedure. */ @@ -208,6 +220,7 @@ public int getWriteBehindFlushSize() { */ public void setFlushThreadCount(int flushThreadCnt) { this.flushThreadCnt = flushThreadCnt; + this.flushThreadCntIsPowerOfTwo = (flushThreadCnt & (flushThreadCnt - 1)) == 0; } /** @@ -219,6 +232,24 @@ public int getWriteBehindFlushThreadCount() { return flushThreadCnt; } + /** + * Sets the write coalescing flag. + * + * @param writeCoalescing Write coalescing flag. + */ + public void setWriteCoalescing(boolean writeCoalescing) { + this.writeCoalescing = writeCoalescing; + } + + /** + * Gets the write coalescing flag. + * + * @return Write coalescing flag. + */ + public boolean getWriteCoalescing() { + return writeCoalescing; + } + /** * Sets the cache flush frequency. All pending operations on the underlying store will be performed * within time interval not less then this value. @@ -266,7 +297,16 @@ public int getWriteBehindStoreBatchSize() { * @return Total count of entries in cache store internal buffer. */ public int getWriteBehindBufferSize() { - return writeCache.sizex(); + if (writeCoalescing) + return writeCache.sizex(); + else { + int size = 0; + + for (Flusher f : flushThreads) + size += f.size(); + + return size; + } } /** @@ -292,14 +332,15 @@ public CacheStore store() { if (cacheCriticalSize == 0) cacheCriticalSize = CacheConfiguration.DFLT_WRITE_BEHIND_CRITICAL_SIZE; - flushThreads = new GridWorker[flushThreadCnt]; + flushThreads = new GridCacheWriteBehindStore.Flusher[flushThreadCnt]; - writeCache = new ConcurrentLinkedHashMap<>(initCap, 0.75f, concurLvl); + if (writeCoalescing) + writeCache = new ConcurrentLinkedHashMap<>(initCap, 0.75f, concurLvl); for (int i = 0; i < flushThreads.length; i++) { flushThreads[i] = new Flusher(gridName, "flusher-" + i, log); - new IgniteThread(flushThreads[i]).start(); + flushThreads[i].start(); } } } @@ -344,7 +385,10 @@ public int getWriteBehindErrorRetryCount() { if (log.isDebugEnabled()) log.debug("Stopping write-behind store for cache '" + cacheName + '\''); - wakeUp(); + for (Flusher f : flushThreads) { + if (!f.isEmpty()) + f.wakeUp(); + } boolean graceful = true; @@ -352,7 +396,7 @@ public int getWriteBehindErrorRetryCount() { graceful &= U.join(worker, log); if (!graceful) - log.warning("Shutdown was aborted"); + log.warning("Write behind store shutdown was aborted."); } } @@ -361,7 +405,10 @@ public int getWriteBehindErrorRetryCount() { * @throws IgniteCheckedException If failed. */ public void forceFlush() throws IgniteCheckedException { - wakeUp(); + for (Flusher f : flushThreads) { + if (!f.isEmpty()) + f.wakeUp(); + } } /** {@inheritDoc} */ @@ -376,10 +423,15 @@ public void forceFlush() throws IgniteCheckedException { Map loaded = new HashMap<>(); - Collection remaining = new LinkedList<>(); + Collection remaining = null; for (K key : keys) { - StatefulValue val = writeCache.get(key); + StatefulValue val; + + if (writeCoalescing) + val = writeCache.get(key); + else + val = flusher(key).flusherWriteMap.get(key); if (val != null) { val.readLock().lock(); @@ -394,12 +446,16 @@ public void forceFlush() throws IgniteCheckedException { val.readLock().unlock(); } } - else + else { + if (remaining == null) + remaining = new ArrayList<>(); + remaining.add(key); + } } // For items that were not found in queue. - if (!remaining.isEmpty()) { + if (remaining != null && !remaining.isEmpty()) { Map loaded0 = store.loadAll(remaining); if (loaded0 != null) @@ -414,7 +470,12 @@ public void forceFlush() throws IgniteCheckedException { if (log.isDebugEnabled()) log.debug("Store load [key=" + key + ']'); - StatefulValue val = writeCache.get(key); + StatefulValue val; + + if (writeCoalescing) + val = writeCache.get(key); + else + val = flusher(key).flusherWriteMap.get(key); if (val != null) { val.readLock().lock(); @@ -493,7 +554,7 @@ public void forceFlush() throws IgniteCheckedException { * * @param key Key for which update is performed. * @param val New value, may be null for remove operation. - * @param operation Updated value status + * @param operation Updated value status. * @throws IgniteInterruptedCheckedException If interrupted while waiting for value to be flushed. */ private void updateCache(K key, @@ -502,8 +563,27 @@ private void updateCache(K key, throws IgniteInterruptedCheckedException { StatefulValue newVal = new StatefulValue<>(val, operation); + if (writeCoalescing) + putToWriteCache(key, newVal); + else + flusher(key).putToFlusherWriteCache(key, newVal); + } + + /** + * Performs flush-consistent writeCache update for the given key. + * + * @param key Key for which update is performed. + * @param newVal stateful value to put + * @throws IgniteInterruptedCheckedException If interrupted while waiting for value to be flushed. + */ + private void putToWriteCache( + K key, + StatefulValue newVal) + throws IgniteInterruptedCheckedException { StatefulValue prev; + assert writeCoalescing : "Unexpected write coalescing."; + while ((prev = writeCache.putIfAbsent(key, newVal)) != null) { prev.writeLock().lock(); @@ -523,7 +603,7 @@ else if (prev.status() == ValueStatus.RETRY) assert prev.status() == ValueStatus.NEW || prev.status() == ValueStatus.RETRY; - prev.update(val, operation, ValueStatus.NEW); + prev.update(newVal.val, newVal.operation(), ValueStatus.NEW); break; } @@ -533,13 +613,32 @@ else if (prev.status() == ValueStatus.RETRY) } // Now check the map size - if (writeCache.sizex() > cacheCriticalSize) + int cacheSize = getWriteBehindBufferSize(); + + if (cacheSize > cacheCriticalSize) // Perform single store update in the same thread. flushSingleValue(); - else if (cacheMaxSize > 0 && writeCache.sizex() > cacheMaxSize) + else if (cacheMaxSize > 0 && cacheSize > cacheMaxSize) wakeUp(); } + /** + * Return flusher by by key. + * + * @param key Key for search. + * @return flusher. + */ + private Flusher flusher(K key) { + int h, idx; + + if (flushThreadCntIsPowerOfTwo) + idx = ((h = key.hashCode()) ^ (h >>> 16)) & (flushThreadCnt - 1); + else + idx = ((h = key.hashCode()) ^ (h >>> 16)) % flushThreadCnt; + + return flushThreads[idx]; + } + /** * Flushes one upcoming value to the underlying store. Called from * {@link #updateCache(Object, Entry, StoreOperation)} method in case when current map size exceeds @@ -549,7 +648,7 @@ private void flushSingleValue() { cacheOverflowCntr.incrementAndGet(); try { - Map> batch = null; + Map> batch; for (Map.Entry> e : writeCache.entrySet()) { StatefulValue val = e.getValue(); @@ -577,7 +676,7 @@ private void flushSingleValue() { } if (!batch.isEmpty()) { - applyBatch(batch, false); + applyBatch(batch, false, null); cacheTotalOverflowCntr.incrementAndGet(); @@ -595,9 +694,12 @@ private void flushSingleValue() { * * @param valMap Batch map. * @param initSes {@code True} if need to initialize session. + * @param flusher Flusher, assotiated with all keys in batch (have sense in write coalescing = false mode) + * @return {@code True} if batch was successfully applied, {@code False} otherwise. */ - private void applyBatch(Map> valMap, boolean initSes) { + private boolean applyBatch(Map> valMap, boolean initSes, Flusher flusher) { assert valMap.size() <= batchSize; + assert !valMap.isEmpty(); StoreOperation operation = null; @@ -615,7 +717,9 @@ private void applyBatch(Map> valMap, boolean initSes) { batch.put(e.getKey(), e.getValue().entry()); } - if (updateStore(operation, batch, initSes)) { + boolean result = updateStore(operation, batch, initSes, flusher); + + if (result) { for (Map.Entry> e : valMap.entrySet()) { StatefulValue val = e.getValue(); @@ -624,12 +728,22 @@ private void applyBatch(Map> valMap, boolean initSes) { try { val.status(ValueStatus.FLUSHED); - StatefulValue prev = writeCache.remove(e.getKey()); + if (writeCoalescing) { + StatefulValue prev = writeCache.remove(e.getKey()); - // Additional check to ensure consistency. - assert prev == val : "Map value for key " + e.getKey() + " was updated during flush"; + // Additional check to ensure consistency. + assert prev == val : "Map value for key " + e.getKey() + " was updated during flush"; - val.signalFlushed(); + val.signalFlushed(); + } + else { + Flusher f = flusher(e.getKey()); + + // Can remove using equal because if map contains another similar value it has different state. + f.flusherWriteMap.remove(e.getKey(), e.getValue()); + + val.signalFlushed(); + } } finally { val.writeLock().unlock(); @@ -653,6 +767,8 @@ private void applyBatch(Map> valMap, boolean initSes) { } } } + + return result; } /** @@ -666,13 +782,16 @@ private void applyBatch(Map> valMap, boolean initSes) { * @param operation Status indicating operation that should be performed. * @param vals Key-Value map. * @param initSes {@code True} if need to initialize session. + * @param flusher Flusher, assotiated with vals keys (in writeCoalescing=false mode) * @return {@code true} if value may be deleted from the write cache, * {@code false} otherwise */ - private boolean updateStore(StoreOperation operation, + private boolean updateStore( + StoreOperation operation, Map> vals, - boolean initSes) { - + boolean initSes, + Flusher flusher + ) { try { if (initSes && storeMgr != null) storeMgr.writeBehindSessionInit(); @@ -707,7 +826,14 @@ private boolean updateStore(StoreOperation operation, catch (Exception e) { LT.error(log, e, "Unable to update underlying store: " + store); - if (writeCache.sizex() > cacheCriticalSize || stopping.get()) { + boolean overflow; + + if (writeCoalescing) + overflow = writeCache.sizex() > cacheCriticalSize || stopping.get(); + else + overflow = flusher.isOverflowed() || stopping.get(); + + if (overflow) { for (Map.Entry> entry : vals.entrySet()) { Object val = entry.getValue() != null ? entry.getValue().getValue() : null; @@ -738,29 +864,163 @@ private void wakeUp() { } /** - * Thread that performs time-based flushing of written values to the underlying storage. + * Thread that performs time/size-based flushing of written values to the underlying storage. */ private class Flusher extends GridWorker { + /** Queue to flush. */ + private final ConcurrentLinkedDeque8>> queue; + + /** Flusher write map. */ + private final ConcurrentHashMap> flusherWriteMap; + + /** Critical size of flusher local queue. */ + private final int flusherCacheCriticalSize; + + /** Flusher parked flag. */ + private volatile boolean parked; + + /** Flusher thread. */ + protected Thread thread; + + /** Cache flushing frequence in nanos. */ + protected long cacheFlushFreqNanos = cacheFlushFreq * 1000; + + /** Writer lock. */ + private final Lock flusherWriterLock = new ReentrantLock(); + + /** Confition to determine available space for flush. */ + private Condition flusherWriterCanWrite = flusherWriterLock.newCondition(); + /** {@inheritDoc */ - protected Flusher(String gridName, String name, IgniteLogger log) { + protected Flusher(String gridName, + String name, + IgniteLogger log) { super(gridName, name, log); + + flusherCacheCriticalSize = cacheCriticalSize/flushThreadCnt; + + assert flusherCacheCriticalSize > batchSize; + + if (writeCoalescing) { + queue = null; + flusherWriteMap = null; + } + else { + queue = new ConcurrentLinkedDeque8<>(); + flusherWriteMap = new ConcurrentHashMap<>(initCap, 0.75f, concurLvl); + } + } + + /** Start flusher thread */ + protected void start() { + thread = new IgniteThread(this); + thread.start(); + } + + /** + * Performs flush-consistent flusher writeCache update for the given key. + * + * @param key Key for which update is performed. + * @param newVal stateful value to put + * @throws IgniteInterruptedCheckedException If interrupted while waiting for value to be flushed. + */ + private void putToFlusherWriteCache( + K key, + StatefulValue newVal) + throws IgniteInterruptedCheckedException { + assert !writeCoalescing : "Unexpected write coalescing."; + + if (queue.sizex() > flusherCacheCriticalSize) { + while (queue.sizex() > flusherCacheCriticalSize) { + wakeUp(); + + flusherWriterLock.lock(); + + try { + // Wait for free space in flusher queue + while (queue.sizex() >= flusherCacheCriticalSize && !stopping.get()) { + if (cacheFlushFreq > 0) + flusherWriterCanWrite.await(cacheFlushFreq, TimeUnit.MILLISECONDS); + else + flusherWriterCanWrite.await(); + } + + cacheTotalOverflowCntr.incrementAndGet(); + } + catch (InterruptedException e) { + if (log.isDebugEnabled()) + log.debug("Caught interrupted exception: " + e); + + Thread.currentThread().interrupt(); + } + finally { + flusherWriterLock.unlock(); + } + } + + cacheTotalOverflowCntr.incrementAndGet(); + } + + queue.add(F.t(key, newVal)); + + flusherWriteMap.put(key, newVal); + } + + /** + * Get overflowed flag. + * + * @return {@code True} if write behind flusher is overflowed, + * {@code False} otherwise. + */ + public boolean isOverflowed() { + if (writeCoalescing) + return writeCache.sizex() > cacheCriticalSize; + else + return queue.sizex() > flusherCacheCriticalSize; + } + + /** + * Get write behind flusher size. + * + * @return Flusher write behind size. + */ + public int size() { + return writeCoalescing ? writeCache.sizex() : queue.sizex(); + } + + /** + * Test if write behind flusher is empty + * + * @return {@code True} if write behind flusher is empty, {@code False} otherwise + */ + public boolean isEmpty() { + return writeCoalescing ? writeCache.isEmpty() : queue.isEmpty(); } /** {@inheritDoc} */ @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException { - while (!stopping.get() || writeCache.sizex() > 0) { - awaitOperationsAvailable(); + if (writeCoalescing) { + while (!stopping.get() || writeCache.sizex() > 0) { + awaitOperationsAvailableCoalescing(); - flushCache(writeCache.entrySet().iterator()); + flushCacheCoalescing(); + } + } + else { + while (!stopping.get() || queue.sizex() > 0) { + awaitOperationsAvailableNonCoalescing(); + + flushCacheNonCoalescing(); + } } } /** - * This method awaits until enough elements in map are available or given timeout is over. + * This method awaits until enough elements in flusher queue are available or given timeout is over. * * @throws InterruptedException If awaiting was interrupted. */ - private void awaitOperationsAvailable() throws InterruptedException { + private void awaitOperationsAvailableCoalescing() throws InterruptedException { flushLock.lock(); try { @@ -779,75 +1039,216 @@ private void awaitOperationsAvailable() throws InterruptedException { } } + /** + * This method awaits until enough elements in flusher queue are available or given timeout is over. + * + * @throws InterruptedException If awaiting was interrupted. + */ + private void awaitOperationsAvailableNonCoalescing() throws InterruptedException { + if (queue.sizex() >= batchSize) + return; + + parked = true; + + try { + for (;;) { + if (queue.sizex() >= batchSize) + return; + + if (cacheFlushFreq > 0) + LockSupport.parkNanos(cacheFlushFreqNanos); + else + LockSupport.park(); + + if (queue.sizex() > 0) + return; + + if (Thread.interrupted()) + throw new InterruptedException(); + + if (stopping.get()) + return; + } + } + finally { + parked = false; + } + } + + /** + * Wake up flusher thread. + */ + public void wakeUp() { + if (parked) + LockSupport.unpark(thread); + } + /** * Removes values from the write cache and performs corresponding operation * on the underlying store. - * - * @param it Iterator for write cache. */ - private void flushCache(Iterator>> it) { - StoreOperation operation = null; + private void flushCacheCoalescing() { + StoreOperation prevOperation = null; - Map> batch = null; - Map> pending = U.newLinkedHashMap(batchSize); + Map> pending = U.newLinkedHashMap(batchSize); + Iterator>> it = writeCache.entrySet().iterator(); while (it.hasNext()) { Map.Entry> e = it.next(); - StatefulValue val = e.getValue(); - val.writeLock().lock(); + if (!val.writeLock().tryLock()) // TODO: stripe write maps to avoid lock contention. + continue; try { - ValueStatus status = val.status(); + BatchingResult addRes = tryAddStatefulValue(pending, prevOperation, e.getKey(), val); - if (acquired(status)) - // Another thread is helping us, continue to the next entry. - continue; - - if (status == ValueStatus.RETRY) - retryEntriesCnt.decrementAndGet(); + switch (addRes) { + case NEW_BATCH: + applyBatch(pending, true, null); - assert retryEntriesCnt.get() >= 0; + pending = U.newLinkedHashMap(batchSize); - val.status(ValueStatus.PENDING); + // No need to test first value in batch + val.status(ValueStatus.PENDING); + pending.put(e.getKey(), val); + prevOperation = val.operation(); - // We scan for the next operation and apply batch on operation change. Null means new batch. - if (operation == null) - operation = val.operation(); + break; - if (operation != val.operation()) { - // Operation is changed, so we need to perform a batch. - batch = pending; - pending = U.newLinkedHashMap(batchSize); + case ADDED: + prevOperation = val.operation(); - operation = val.operation(); + break; - pending.put(e.getKey(), val); + default: + assert addRes == BatchingResult.SKIPPED : "Unexpected result: " + addRes; } - else - pending.put(e.getKey(), val); + } + finally { + val.writeLock().unlock(); + } + } + + // Process the remainder. + if (!pending.isEmpty()) + applyBatch(pending, true, null); + } + + /** + * Removes values from the flusher write queue and performs corresponding operation + * on the underlying store. + */ + private void flushCacheNonCoalescing() { + StoreOperation prevOperation; + Map> pending; + IgniteBiTuple> tuple; + boolean applied; + + while(!queue.isEmpty()) { + pending = U.newLinkedHashMap(batchSize); + prevOperation = null; + boolean needNewBatch = false; + + // Collect batch + while (!needNewBatch && (tuple = queue.peek()) != null) { + BatchingResult addRes = tryAddStatefulValue(pending, prevOperation, tuple.getKey(), + tuple.getValue()); + + switch (addRes) { + case ADDED: + prevOperation = tuple.getValue().operation(); + queue.poll(); + + break; + + case SKIPPED: + assert false : "Unexpected result: " + addRes; + + break; - if (pending.size() == batchSize) { - batch = pending; - pending = U.newLinkedHashMap(batchSize); + case NEW_BATCH: + needNewBatch = true; + prevOperation = null; - operation = null; + break; + + default: + assert false : "Unexpected result: " + addRes; } } - finally { - val.writeLock().unlock(); + + // Process collected batch + applied = applyBatch(pending, true, this); + + if (applied) { + // Wake up awaiting writers + flusherWriterLock.lock(); + + try { + flusherWriterCanWrite.signalAll(); + } + finally { + flusherWriterLock.unlock(); + } } + else { + // Return values to queue + ArrayList>> pendingList = new ArrayList(pending.entrySet()); - if (batch != null && !batch.isEmpty()) { - applyBatch(batch, true); - batch = null; + for (int i = pendingList.size() - 1; i >= 0; i--) + queue.addFirst(F.t(pendingList.get(i).getKey(), pendingList.get(i).getValue())); } } + } - // Process the remainder. - if (!pending.isEmpty()) - applyBatch(pending, true); + /** + * Trying to add key and statefull value pairs into pending map. + * + * @param pending Map to populate. + * @param key Key to add. + * @param val Stateful value to add. + * @return {@code BatchingResult.ADDED} if pair was sucessfully added, + * {@code BatchingResult.SKIPPED} if pair cannot be processed by this thread, + * {@code BatchingResult.NEW_BATCH} if pair require new batch (pending map) to be added. + */ + public BatchingResult tryAddStatefulValue( + Map> pending, + StoreOperation prevOperation, + K key, + StatefulValue val + ) { + ValueStatus status = val.status(); + + assert !(pending.isEmpty() && prevOperation != null) : "prev operation cannot be " + prevOperation + + " if prev map is empty!"; + + if (acquired(status)) + // Another thread is helping us, continue to the next entry. + return BatchingResult.SKIPPED; + + if (!writeCoalescing && pending.containsKey(key)) + return BatchingResult.NEW_BATCH; + + if (status == ValueStatus.RETRY) + retryEntriesCnt.decrementAndGet(); + + assert retryEntriesCnt.get() >= 0; + + if (pending.size() == batchSize) + return BatchingResult.NEW_BATCH; + + // We scan for the next operation and apply batch on operation change. Null means new batch. + if (prevOperation != val.operation() && prevOperation != null) + // Operation is changed, so we need to perform a batch. + return BatchingResult.NEW_BATCH; + else { + val.status(ValueStatus.PENDING); + + pending.put(key, val); + + return BatchingResult.ADDED; + } } } @@ -860,6 +1261,20 @@ Map> writeCache() { return writeCache; } + /** + * For test purposes only. + * + * @return Flusher maps for the underlying store operations. + */ + Map>[] flusherMaps() { + Map>[] result = new Map[flushThreadCnt]; + + for (int i=0; i < flushThreadCnt; i++) + result[i] = flushThreads[i].flusherWriteMap; + + return result; + } + /** * Enumeration that represents possible operations on the underlying store. */ @@ -888,6 +1303,20 @@ private enum ValueStatus { FLUSHED, } + /** + * Enumeration that represents possible result of "add to batch" operation. + */ + private enum BatchingResult { + /** Added to batch */ + ADDED, + + /** Skipped. */ + SKIPPED, + + /** Need new batch. */ + NEW_BATCH + } + /** * Checks if given status indicates pending or complete flush operation. * @@ -901,6 +1330,7 @@ private boolean acquired(ValueStatus status) { /** * A state-value-operation trio. * + * @param Key type. * @param Value type. */ private static class StatefulValue extends ReentrantReadWriteLock { @@ -949,7 +1379,7 @@ private StoreOperation operation() { } /** - * @return Value status + * @return Value status. */ private ValueStatus status() { return valStatus; @@ -980,7 +1410,7 @@ private void update(@Nullable Entry val, } /** - * Awaits a signal on flush condition + * Awaits a signal on flush condition. * * @throws IgniteInterruptedCheckedException If thread was interrupted. */ @@ -1023,4 +1453,4 @@ private void signalFlushed() { return S.toString(StatefulValue.class, this); } } -} \ No newline at end of file +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStoreAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStoreAbstractSelfTest.java index 323278f9c5d30..3bac90686ba33 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStoreAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStoreAbstractSelfTest.java @@ -20,12 +20,14 @@ import java.util.ArrayList; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Random; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.processors.cache.CacheEntryImpl; import org.apache.ignite.internal.processors.cache.GridCacheTestStore; @@ -59,16 +61,29 @@ public abstract class GridCacheWriteBehindStoreAbstractSelfTest extends GridComm /** * Initializes store. * - * @param flushThreadCnt Count of flush threads + * @param flushThreadCnt Count of flush threads. * @throws Exception If failed. */ protected void initStore(int flushThreadCnt) throws Exception { + initStore(flushThreadCnt, CacheConfiguration.DFLT_WRITE_BEHIND_COALESCING); + } + + /** + * Initializes store. + * + * @param flushThreadCnt Count of flush threads. + * @param writeCoalescing write coalescing flag. + * @throws Exception If failed. + */ + protected void initStore(int flushThreadCnt, boolean writeCoalescing) throws Exception { store = new GridCacheWriteBehindStore<>(null, "", "", log, delegate); store.setFlushFrequency(FLUSH_FREQUENCY); store.setFlushSize(CACHE_SIZE); + store.setWriteCoalescing(writeCoalescing); + store.setFlushThreadCount(flushThreadCnt); delegate.reset(); @@ -83,8 +98,11 @@ protected void initStore(int flushThreadCnt) throws Exception { */ protected void shutdownStore() throws Exception { store.stop(); - - assertTrue("Store cache must be empty after shutdown", store.writeCache().isEmpty()); + if (store.getWriteCoalescing()) + assertTrue("Store cache must be empty after shutdown", store.writeCache().isEmpty()); + else + for (Map fMap : store.flusherMaps()) + assertTrue("Store flusher cache must be empty after shutdown", fMap.isEmpty()); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStoreAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStoreAbstractTest.java index e9674f3c90b52..ca43ce3ba3d5c 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStoreAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStoreAbstractTest.java @@ -37,6 +37,10 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.apache.ignite.transactions.Transaction; import org.jetbrains.annotations.Nullable; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; +import org.junit.runners.Parameterized.Parameter; import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStoreMultithreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStoreMultithreadedSelfTest.java index bc6b7bd83f581..15c58d9f515f4 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStoreMultithreadedSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStoreMultithreadedSelfTest.java @@ -21,19 +21,38 @@ import java.util.HashSet; import java.util.Map; import java.util.Set; +import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.util.typedef.internal.U; /** * Multithreaded tests for {@link org.apache.ignite.internal.processors.cache.store.GridCacheWriteBehindStore}. */ public class GridCacheWriteBehindStoreMultithreadedSelfTest extends GridCacheWriteBehindStoreAbstractSelfTest { + /** + * This test performs complex set of operations on store with coalescing from multiple threads. + * + * @throws Exception If failed. + */ + public void testPutGetRemoveWithCoalescing() throws Exception { + testPutGetRemove(true); + } + + /** + * This test performs complex set of operations on store without coalescing from multiple threads. + * + * @throws Exception If failed. + */ + public void testPutGetRemoveWithoutCoalescing() throws Exception { + testPutGetRemove(false); + } + /** * This test performs complex set of operations on store from multiple threads. * * @throws Exception If failed. */ - public void testPutGetRemove() throws Exception { - initStore(2); + private void testPutGetRemove(boolean writeCoalescing) throws Exception { + initStore(2, writeCoalescing); Set exp; @@ -62,27 +81,55 @@ public void testPutGetRemove() throws Exception { assertEquals("Invalid value for key " + key, "val" + key, map.get(key)); } + /** + * Tests that cache with write coalescing would keep values if underlying store fails. + * + * @throws Exception if failed. + */ + public void testStoreFailureWithCoalescing() throws Exception { + testStoreFailure(true); + } + + /** + * Tests that cache without write coalescing would keep values if underlying store fails. + * + * @throws Exception if failed. + */ + public void testStoreFailureWithoutCoalescing() throws Exception { + testStoreFailure(false); + } + /** * Tests that cache would keep values if underlying store fails. * * @throws Exception If failed. */ - public void testStoreFailure() throws Exception { + private void testStoreFailure(boolean writeCoalescing) throws Exception { delegate.setShouldFail(true); - initStore(2); + initStore(2, writeCoalescing); Set exp; try { + Thread timer = new Thread(new Runnable() { + @Override + public void run() { + try { + U.sleep(FLUSH_FREQUENCY+50); + } catch (IgniteInterruptedCheckedException e) { + assertTrue("Timer was interrupted", false); + } + delegate.setShouldFail(false); + } + }); + timer.start(); exp = runPutGetRemoveMultithreaded(10, 10); - U.sleep(FLUSH_FREQUENCY); + timer.join(); info(">>> There are " + store.getWriteBehindErrorRetryCount() + " entries in RETRY state"); - delegate.setShouldFail(false); - // Despite that we set shouldFail flag to false, flush thread may just have caught an exception. // If we move store to the stopping state right away, this value will be lost. That's why this sleep // is inserted here to let all exception handlers in write-behind store exit. @@ -110,17 +157,38 @@ public void testStoreFailure() throws Exception { assertEquals("Invalid value for key " + key, "val" + key, map.get(key)); } + /** + * Tests store (with write coalescing) consistency in case of high put rate, + * when flush is performed from the same thread as put or remove operation. + * + * @throws Exception If failed. + */ + public void testFlushFromTheSameThreadWithCoalescing() throws Exception { + testFlushFromTheSameThread(true); + } + + /** + * Tests store (without write coalescing) consistency in case of high put rate, + * when flush is performed from the same thread as put or remove operation. + * + * @throws Exception If failed. + */ + public void testFlushFromTheSameThreadWithoutCoalescing() throws Exception { + testFlushFromTheSameThread(false); + } + /** * Tests store consistency in case of high put rate, when flush is performed from the same thread * as put or remove operation. * + * @param writeCoalescing write coalescing flag. * @throws Exception If failed. */ - public void testFlushFromTheSameThread() throws Exception { + private void testFlushFromTheSameThread(boolean writeCoalescing) throws Exception { // 50 milliseconds should be enough. delegate.setOperationDelay(50); - initStore(2); + initStore(2, writeCoalescing); Set exp; @@ -162,4 +230,4 @@ public void testFlushFromTheSameThread() throws Exception { for (Integer key : exp) assertEquals("Invalid value for key " + key, "val" + key, map.get(key)); } -} \ No newline at end of file +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStoreSelfTest.java index 67e26ab66bd72..9a487a4268726 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStoreSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStoreSelfTest.java @@ -35,11 +35,30 @@ */ public class GridCacheWriteBehindStoreSelfTest extends GridCacheWriteBehindStoreAbstractSelfTest { /** - * Tests correct store shutdown when underlying store fails, + * Tests correct store (with write coalescing) shutdown when underlying store fails. * * @throws Exception If failed. */ - public void testShutdownWithFailure() throws Exception { + public void testShutdownWithFailureWithCoalescing() throws Exception { + testShutdownWithFailure(true); + } + + /** + * Tests correct store (without write coalescing) shutdown when underlying store fails. + * + * @throws Exception If failed. + */ + public void testShutdownWithFailureWithoutCoalescing() throws Exception { + testShutdownWithFailure(false); + } + + /** + * Tests correct store shutdown when underlying store fails. + * + * @param writeCoalescing Write coalescing flag. + * @throws Exception If failed. + */ + private void testShutdownWithFailure(final boolean writeCoalescing) throws Exception { final AtomicReference err = new AtomicReference<>(); multithreadedAsync(new Runnable() { @@ -47,7 +66,7 @@ public void testShutdownWithFailure() throws Exception { try { delegate.setShouldFail(true); - initStore(2); + initStore(2, writeCoalescing); try { store.write(new CacheEntryImpl<>(1, "val1")); @@ -70,10 +89,31 @@ public void testShutdownWithFailure() throws Exception { } /** + * Simple store (with write coalescing) test. + * + * @throws Exception If failed. + */ + public void testSimpleStoreWithCoalescing() throws Exception { + testSimpleStore(true); + } + + /** + * Simple store (without write coalescing) test. + * * @throws Exception If failed. */ - public void testSimpleStore() throws Exception { - initStore(2); + public void testSimpleStoreWithoutCoalescing() throws Exception { + testSimpleStore(false); + } + + /** + * Simple store test. + * + * @param writeCoalescing Write coalescing flag. + * @throws Exception If failed. + */ + private void testSimpleStore(boolean writeCoalescing) throws Exception { + initStore(2, writeCoalescing); try { store.write(new CacheEntryImpl<>(1, "v1")); @@ -94,15 +134,36 @@ public void testSimpleStore() throws Exception { } } + /** + * Check that all values written to the store with coalescing will be in underlying store after timeout + * or due to size limits. + * + * @throws Exception If failed. + */ + public void testValuePropagationWithCoalescing() throws Exception { + testValuePropagation(true); + } + + /** + * Check that all values written to the store without coalescing will be in underlying store after timeout + * or due to size limits. + * + * @throws Exception If failed. + */ + public void testValuePropagationWithoutCoalescing() throws Exception { + testValuePropagation(false); + } + /** * Check that all values written to the store will be in underlying store after timeout or due to size limits. * + * @param writeCoalescing Write coalescing flag * @throws Exception If failed. */ @SuppressWarnings({"NullableProblems"}) - public void testValuePropagation() throws Exception { + private void testValuePropagation(boolean writeCoalescing) throws Exception { // Need to test size-based write. - initStore(1); + initStore(1, writeCoalescing); try { for (int i = 0; i < CACHE_SIZE * 2; i++) @@ -131,13 +192,32 @@ public void testValuePropagation() throws Exception { } } + /** + * Tests store with write coalescing behaviour under continuous put of the same key with different values. + * + * @throws Exception If failed. + */ + public void testContinuousPutWithCoalescing() throws Exception { + testContinuousPut(true); + } + + /** + * Tests store without write coalescing behaviour under continuous put of the same key with different values. + * + * @throws Exception If failed. + */ + public void testContinuousPutWithoutCoalescing() throws Exception { + testContinuousPut(false); + } + /** * Tests store behaviour under continuous put of the same key with different values. * - * @throws Exception If failed + * @param writeCoalescing Write coalescing flag for cache. + * @throws Exception If failed. */ - public void testContinuousPut() throws Exception { - initStore(2); + private void testContinuousPut(boolean writeCoalescing) throws Exception { + initStore(2, writeCoalescing); try { final AtomicBoolean running = new AtomicBoolean(true); @@ -168,18 +248,23 @@ public void testContinuousPut() throws Exception { } }, 1, "put"); + U.sleep(FLUSH_FREQUENCY * 2 + 500); + running.set(false); U.sleep(FLUSH_FREQUENCY * 2 + 500); int delegatePutCnt = delegate.getPutAllCount(); - running.set(false); fut.get(); log().info(">>> [putCnt = " + actualPutCnt.get() + ", delegatePutCnt=" + delegatePutCnt + "]"); assertTrue("No puts were made to the underlying store", delegatePutCnt > 0); - assertTrue("Too many puts were made to the underlying store", delegatePutCnt < actualPutCnt.get() / 10); + if (store.getWriteCoalescing()) { + assertTrue("Too many puts were made to the underlying store", delegatePutCnt < actualPutCnt.get() / 10); + } else { + assertTrue("Too few puts cnt=" + actualPutCnt.get() + " << storePutCnt=" + delegatePutCnt, delegatePutCnt > actualPutCnt.get() / 2); + } } finally { shutdownStore(); @@ -192,14 +277,35 @@ public void testContinuousPut() throws Exception { assertEquals("Invalid value stored", "val" + i, delegate.getMap().get(i)); } + /** + * Tests that all values were put into the store with write coalescing will be written to the underlying store + * after shutdown is called. + * + * @throws Exception If failed. + */ + public void testShutdownWithCoalescing() throws Exception { + testShutdown(true); + } + + /** + * Tests that all values were put into the store without write coalescing will be written to the underlying store + * after shutdown is called. + * + * @throws Exception If failed. + */ + public void testShutdownWithoutCoalescing() throws Exception { + testShutdown(false); + } + /** * Tests that all values were put into the store will be written to the underlying store * after shutdown is called. * + * @param writeCoalescing Write coalescing flag. * @throws Exception If failed. */ - public void testShutdown() throws Exception { - initStore(2); + private void testShutdown(boolean writeCoalescing) throws Exception { + initStore(2, writeCoalescing); try { final AtomicBoolean running = new AtomicBoolean(true); @@ -241,16 +347,37 @@ public void testShutdown() throws Exception { assertEquals("Invalid value stored", "val" + i, delegate.getMap().get(i)); } + /** + * Tests that all values will be written to the underlying store + * right in the same order as they were put into the store with coalescing. + * + * @throws Exception If failed. + */ + public void testBatchApplyWithCoalescing() throws Exception { + testBatchApply(true); + } + + /** + * Tests that all values will be written to the underlying store + * right in the same order as they were put into the store without coalescing. + * + * @throws Exception If failed. + */ + public void testBatchApplyWithoutCoalescing() throws Exception { + testBatchApply(false); + } + /** * Tests that all values will be written to the underlying store * right in the same order as they were put into the store. * + * @param writeCoalescing Write coalescing flag. * @throws Exception If failed. */ - public void testBatchApply() throws Exception { + private void testBatchApply(boolean writeCoalescing) throws Exception { delegate = new GridCacheTestStore(new ConcurrentLinkedHashMap()); - initStore(1); + initStore(1, writeCoalescing); List intList = new ArrayList<>(CACHE_SIZE); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/IgnteCacheClientWriteBehindStoreNonCoalescingTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/IgnteCacheClientWriteBehindStoreNonCoalescingTest.java new file mode 100644 index 0000000000000..8ea109dd2edfd --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/IgnteCacheClientWriteBehindStoreNonCoalescingTest.java @@ -0,0 +1,175 @@ +/* + * 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.store; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import javax.cache.Cache; +import javax.cache.configuration.Factory; +import javax.cache.processor.EntryProcessor; +import javax.cache.processor.EntryProcessorException; +import javax.cache.processor.MutableEntry; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheAtomicWriteOrderMode; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheMode; +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.NearCacheConfiguration; +import org.apache.ignite.internal.processors.cache.IgniteCacheAbstractTest; +import org.apache.ignite.lang.IgniteBiInClosure; +import org.apache.ignite.lang.IgniteFuture; + +import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.CLOCK; +import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC; + +/** + * This class provides non write coalescing tests for {@link org.apache.ignite.internal.processors.cache.store.GridCacheWriteBehindStore}. + */ +public class IgnteCacheClientWriteBehindStoreNonCoalescingTest extends IgniteCacheAbstractTest { + /** {@inheritDoc} */ + @Override protected int gridCount() { + return 1; + } + + /** {@inheritDoc} */ + @Override protected CacheMode cacheMode() { + return null; + } + + /** {@inheritDoc} */ + @Override protected CacheAtomicityMode atomicityMode() { + return ATOMIC; + } + + /** {@inheritDoc} */ + @Override protected CacheAtomicWriteOrderMode atomicWriteOrderMode() { + return CLOCK; + } + + /** {@inheritDoc} */ + @Override protected NearCacheConfiguration nearConfiguration() { + return null; + } + + /** {@inheritDoc} */ + @Override protected Factory cacheStoreFactory() { + return new TestIncrementStoreFactory(); + } + + /** + * @throws Exception If failed. + */ + public void testNonCoalescingIncrementing() throws Exception { + Ignite ignite = grid(0); + + IgniteCache cache = ignite.cache(null); + + assertEquals(cache.getConfiguration(CacheConfiguration.class).getCacheStoreFactory().getClass(), + TestIncrementStoreFactory.class); + + Set keys = new HashSet<>(); + + for (int i = 0; i < 1000; i++) { + keys.add(i); + + cache.put(i, i); + } + + Collection> futs = new ArrayList<>(); + + for (int i = 0; i < 100; i++) + futs.add(updateKeys(cache, keys)); + + for (IgniteFuture fut : futs) + fut.get(); + } + + /** + * Update specified keys in async mode. + * + * @param cache Cache to use. + * @param keys Keys to update. + * @return IgniteFuture. + */ + private IgniteFuture updateKeys(IgniteCache cache, Set keys) { + IgniteCache asyncCache = cache.withAsync(); + + // Using EntryProcessor.invokeAll to increment every value in place. + asyncCache.invokeAll(keys, new EntryProcessor() { + @Override public Object process(MutableEntry entry, Object... arguments) + throws EntryProcessorException { + entry.setValue(entry.getValue() + 1); + + return null; + } + }); + + return asyncCache.future(); + } + + /** + * Test increment store factory. + */ + public static class TestIncrementStoreFactory implements Factory { + /** {@inheritDoc} */ + @Override public CacheStore create() { + return new TestIncrementStore(); + } + } + + /** + * Test cache store to validate int value incrementing + */ + public static class TestIncrementStore extends CacheStoreAdapter { + /** {@inheritDoc} */ + @Override public void loadCache(IgniteBiInClosure clo, Object... args) { + for (Map.Entry e : storeMap.entrySet()) + clo.apply(e.getKey(), e.getValue()); + } + + /** {@inheritDoc} */ + @Override public Object load(Object key) { + return storeMap.get(key); + } + + /** {@inheritDoc} */ + @Override public void write(Cache.Entry entry) { + Object oldValue = storeMap.put(entry.getKey(), entry.getValue()); + + if (oldValue instanceof Integer && entry.getValue() instanceof Integer) { + Integer oldInt = (Integer)oldValue; + Integer newInt = (Integer)entry.getValue(); + + assertTrue( + "newValue(" + newInt + ") != oldValue(" + oldInt + ")+1 !", + newInt == oldInt + 1); + } + } + + /** {@inheritDoc} */ + @Override public void delete(Object key) { + storeMap.remove(key); + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheWriteBehindTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheWriteBehindTestSuite.java index b4cdfa8a76a20..dff93ffa5ecf0 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheWriteBehindTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheWriteBehindTestSuite.java @@ -26,6 +26,7 @@ import org.apache.ignite.internal.processors.cache.store.GridCacheWriteBehindStoreReplicatedTest; import org.apache.ignite.internal.processors.cache.store.GridCacheWriteBehindStoreSelfTest; import org.apache.ignite.internal.processors.cache.store.IgnteCacheClientWriteBehindStoreAtomicTest; +import org.apache.ignite.internal.processors.cache.store.IgnteCacheClientWriteBehindStoreNonCoalescingTest; import org.apache.ignite.internal.processors.cache.store.IgnteCacheClientWriteBehindStoreTxTest; /** @@ -49,6 +50,7 @@ public static TestSuite suite() throws Exception { suite.addTest(new TestSuite(GridCachePartitionedWritesTest.class)); suite.addTest(new TestSuite(IgnteCacheClientWriteBehindStoreAtomicTest.class)); suite.addTest(new TestSuite(IgnteCacheClientWriteBehindStoreTxTest.class)); + suite.addTest(new TestSuite(IgnteCacheClientWriteBehindStoreNonCoalescingTest.class)); return suite; } From c48322b79cafb561211501813c805aebd4b85540 Mon Sep 17 00:00:00 2001 From: Valentin Kulichenko Date: Thu, 13 Apr 2017 11:29:30 +0300 Subject: [PATCH 047/357] IGNITE-4954 - Configurable expiration timeout for Cassandra session. This closes #1785. --- .../cassandra/datasource/DataSource.java | 50 +++++++++++++------ .../session/CassandraSessionImpl.java | 23 +++++---- .../cassandra/session/pool/SessionPool.java | 6 +-- .../session/pool/SessionWrapper.java | 15 +++--- 4 files changed, 62 insertions(+), 32 deletions(-) diff --git a/modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/DataSource.java b/modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/DataSource.java index 1ba3c7d238328..754d902ac9392 100644 --- a/modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/DataSource.java +++ b/modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/DataSource.java @@ -17,6 +17,16 @@ package org.apache.ignite.cache.store.cassandra.datasource; +import java.io.Externalizable; +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import java.io.Serializable; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.util.LinkedList; +import java.util.List; +import java.util.UUID; import com.datastax.driver.core.AuthProvider; import com.datastax.driver.core.Cluster; import com.datastax.driver.core.ConsistencyLevel; @@ -31,25 +41,13 @@ import com.datastax.driver.core.policies.ReconnectionPolicy; import com.datastax.driver.core.policies.RetryPolicy; import com.datastax.driver.core.policies.SpeculativeExecutionPolicy; - -import java.io.Externalizable; -import java.io.IOException; -import java.io.ObjectInput; -import java.io.ObjectOutput; -import java.io.Serializable; -import java.net.InetAddress; -import java.net.InetSocketAddress; -import java.util.LinkedList; -import java.util.List; -import java.util.UUID; - import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.cache.store.cassandra.session.CassandraSession; import org.apache.ignite.cache.store.cassandra.session.CassandraSessionImpl; -import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; /** * Data source abstraction to specify configuration of the Cassandra session to be used. @@ -64,6 +62,9 @@ public class DataSource implements Externalizable { */ private static final UUID NULL_OBJECT = UUID.fromString("45ffae47-3193-5910-84a2-048fe65735d9"); + /** 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; @@ -141,6 +142,9 @@ public class DataSource implements Externalizable { /** 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; @@ -459,6 +463,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 * @@ -541,7 +562,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); } /** {@inheritDoc} */ diff --git a/modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/CassandraSessionImpl.java b/modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/CassandraSessionImpl.java index ac116865c1be5..19b88c988dc6c 100644 --- a/modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/CassandraSessionImpl.java +++ b/modules/cassandra/store/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; @@ -83,6 +83,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; @@ -102,11 +105,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; } @@ -504,7 +508,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/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/pool/SessionPool.java b/modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/pool/SessionPool.java index 95938bd7740a1..4de8516d95b32 100644 --- a/modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/pool/SessionPool.java +++ b/modules/cassandra/store/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/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/pool/SessionWrapper.java b/modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/pool/SessionWrapper.java index 7c5722bb8f21f..68b9dd480f8b8 100644 --- a/modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/pool/SessionWrapper.java +++ b/modules/cassandra/store/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 1c1a2f3e86c091bbbd2668db1cc7d72ff13d91ec Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Wed, 19 Apr 2017 13:27:49 +0300 Subject: [PATCH 048/357] Fix merge ignite-1.8.5-p1 --- .../query/h2/sql/GridSqlQueryParser.java | 2 +- .../query/h2/sql/GridQueryParsingTest.java | 40 ++----------------- 2 files changed, 5 insertions(+), 37 deletions(-) diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java index 2b086584017f0..0f940e9c3846c 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java @@ -1053,7 +1053,7 @@ private GridSqlElement parseExpression0(Expression expression, boolean calcTypes GridSqlOperation res = new GridSqlOperation(EXISTS); - res.addChild(new GridSqlSubquery(parse(qry, null))); + res.addChild(parseQueryExpression(qry)); return res; } diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java index 9dcc80bc649b6..477451a1cd255 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java @@ -289,37 +289,6 @@ public void testParseSelectAndUnion() throws Exception { checkQuery("select sch.\"#\".* from Person \"#\""); } - /** - * Query AST transformation heavily depends on this behavior. - * - * @throws Exception If failed. - */ - public void testParseTableFilter() throws Exception { - Prepared prepared = parse("select Person.old, p1.old from Person, Person p1"); - - GridSqlSelect select = (GridSqlSelect)new GridSqlQueryParser(false).parse(prepared); - - GridSqlJoin join = (GridSqlJoin)select.from(); - - GridSqlTable tbl1 = (GridSqlTable)join.leftTable(); - GridSqlAlias tbl2Alias = (GridSqlAlias)join.rightTable(); - GridSqlTable tbl2 = tbl2Alias.child(); - - // Must be distinct objects, even if it is the same table. - assertNotSame(tbl1, tbl2); - assertNotNull(tbl1.dataTable()); - assertNotNull(tbl2.dataTable()); - assertSame(tbl1.dataTable(), tbl2.dataTable()); - - GridSqlColumn col1 = (GridSqlColumn)select.column(0); - GridSqlColumn col2 = (GridSqlColumn)select.column(1); - - assertSame(tbl1, col1.expressionInFrom()); - - // Alias in FROM must be included in column. - assertSame(tbl2Alias, col2.expressionInFrom()); - } - /** * Query AST transformation heavily depends on this behavior. * @@ -329,7 +298,7 @@ public void testParseTableFilter() throws Exception { Prepared prepared = parse("select Person.old, p1.old, p1.addrId from Person, Person p1 " + "where exists(select 1 from Address a where a.id = p1.addrId)"); - GridSqlSelect select = (GridSqlSelect)new GridSqlQueryParser().parse(prepared); + GridSqlSelect select = (GridSqlSelect)new GridSqlQueryParser(false).parse(prepared); GridSqlJoin join = (GridSqlJoin)select.from(); @@ -338,8 +307,7 @@ public void testParseTableFilter() throws Exception { GridSqlTable tbl2 = tbl2Alias.child(); // Must be distinct objects, even if it is the same table. - //assertNotSame(tbl1, tbl2); - + assertNotSame(tbl1, tbl2); assertNotNull(tbl1.dataTable()); assertNotNull(tbl2.dataTable()); assertSame(tbl1.dataTable(), tbl2.dataTable()); @@ -353,9 +321,9 @@ public void testParseTableFilter() throws Exception { assertSame(tbl2Alias, col2.expressionInFrom()); // In EXISTS we must correctly reference the column from the outer query. - GridSqlElement exists = select.where(); + GridSqlAst exists = select.where(); GridSqlSubquery subqry = exists.child(); - GridSqlSelect subSelect = (GridSqlSelect)subqry.select(); + GridSqlSelect subSelect = subqry.child(); GridSqlColumn p1AddrIdCol = (GridSqlColumn)select.column(2); From ea3e625fe1269383d46d0be55adc85662a216792 Mon Sep 17 00:00:00 2001 From: Alexander Belyak Date: Wed, 19 Apr 2017 13:29:01 +0300 Subject: [PATCH 049/357] IGNITE-4927 Use write behind coalescing from CacheConfiguration in GridCacheStoreManagerAdapter to configure actual cache store. Signed-off-by: nikolay_tikhonov --- .../processors/cache/GridCacheAttributes.java | 7 +++ .../processors/cache/GridCacheProcessor.java | 4 ++ .../store/GridCacheStoreManagerAdapter.java | 1 + .../utils/PlatformConfigurationUtils.java | 2 + ...StoreSessionWriteBehindCoalescingTest.java | 47 +++++++++++++++++++ .../testsuites/IgniteCacheTestSuite4.java | 2 + 6 files changed, 63 insertions(+) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheTxStoreSessionWriteBehindCoalescingTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java index e17b91a0414f9..57f12f635489b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java @@ -302,6 +302,13 @@ public int writeBehindBatchSize() { return ccfg.getWriteBehindBatchSize(); } + /** + * @return Write-behind cache store write coalescing. + */ + public boolean writeBehindCoalescing() { + return ccfg.getWriteBehindCoalescing(); + } + /** * @return Interceptor class name. */ 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 50e1379961298..fe7da4d14e537 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 @@ -2927,6 +2927,10 @@ private void checkCache(CacheConfiguration locCfg, CacheConfiguration rmtCfg, Cl "Write behind batch size", locAttr.writeBehindBatchSize(), rmtAttr.writeBehindBatchSize(), false); + CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "writeBehindCacheCoalescing", + "Write behind coalescing", locAttr.writeBehindCoalescing(), rmtAttr.writeBehindCoalescing(), + false); + CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "writeBehindEnabled", "Write behind enabled", locAttr.writeBehindEnabled(), rmtAttr.writeBehindEnabled(), false); 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 a83018682109d..e9e90ea3c25d8 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 @@ -181,6 +181,7 @@ private CacheStore cacheStoreWrapper(GridKernalContext ctx, store.setFlushThreadCount(cfg.getWriteBehindFlushThreadCount()); store.setFlushFrequency(cfg.getWriteBehindFlushFrequency()); store.setBatchSize(cfg.getWriteBehindBatchSize()); + store.setWriteCoalescing(cfg.getWriteBehindCoalescing()); return store; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java index c0fde976d72ed..daec38b9bfaf9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java @@ -166,6 +166,7 @@ public static CacheConfiguration readCacheConfiguration(BinaryRawReaderEx in) { ccfg.setSqlOnheapRowCacheSize(in.readInt()); ccfg.setStartSize(in.readInt()); ccfg.setWriteBehindBatchSize(in.readInt()); + ccfg.setWriteBehindCoalescing(in.readBoolean()); ccfg.setWriteBehindEnabled(in.readBoolean()); ccfg.setWriteBehindFlushFrequency(in.readLong()); ccfg.setWriteBehindFlushSize(in.readInt()); @@ -791,6 +792,7 @@ public static void writeCacheConfiguration(BinaryRawWriter writer, CacheConfigur writer.writeInt(ccfg.getSqlOnheapRowCacheSize()); writer.writeInt(ccfg.getStartSize()); writer.writeInt(ccfg.getWriteBehindBatchSize()); + writer.writeBoolean(ccfg.getWriteBehindCoalescing()); writer.writeBoolean(ccfg.isWriteBehindEnabled()); writer.writeLong(ccfg.getWriteBehindFlushFrequency()); writer.writeInt(ccfg.getWriteBehindFlushSize()); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheTxStoreSessionWriteBehindCoalescingTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheTxStoreSessionWriteBehindCoalescingTest.java new file mode 100644 index 0000000000000..c15b2f4e2c334 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheTxStoreSessionWriteBehindCoalescingTest.java @@ -0,0 +1,47 @@ +/* + * 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.integration; + +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheAtomicityMode; +import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; + +/** + * Integration test write behind cache store with {@link CacheConfiguration#getWriteBehindCoalescing()}={@code False} + * parameter. + */ +public class IgniteCacheTxStoreSessionWriteBehindCoalescingTest extends IgniteCacheStoreSessionWriteBehindAbstractTest { + /** {@inheritDoc} */ + @Override protected CacheAtomicityMode atomicityMode() { + return TRANSACTIONAL; + } + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration result = super.getConfiguration(gridName); + + CacheConfiguration ccfg = result.getCacheConfiguration()[0]; + + ccfg.setWriteBehindCoalescing(false); + + return result; + } +} \ 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 9fcf31a322f2a..501d23e754517 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 @@ -140,6 +140,7 @@ import org.apache.ignite.internal.processors.cache.integration.IgniteCacheTxNoReadThroughTest; import org.apache.ignite.internal.processors.cache.integration.IgniteCacheTxNoWriteThroughTest; import org.apache.ignite.internal.processors.cache.integration.IgniteCacheTxStoreSessionTest; +import org.apache.ignite.internal.processors.cache.integration.IgniteCacheTxStoreSessionWriteBehindCoalescingTest; import org.apache.ignite.internal.processors.cache.integration.IgniteCacheTxStoreSessionWriteBehindTest; import org.apache.ignite.internal.processors.cache.version.CacheVersionedEntryLocalAtomicSwapDisabledSelfTest; import org.apache.ignite.internal.processors.cache.version.CacheVersionedEntryLocalTransactionalSelfTest; @@ -181,6 +182,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(IgniteCacheAtomicStoreSessionTest.class); suite.addTestSuite(IgniteCacheTxStoreSessionTest.class); suite.addTestSuite(IgniteCacheAtomicStoreSessionWriteBehindTest.class); + suite.addTestSuite(IgniteCacheTxStoreSessionWriteBehindCoalescingTest.class); suite.addTestSuite(IgniteCacheTxStoreSessionWriteBehindTest.class); suite.addTestSuite(IgniteCacheAtomicNoReadThroughTest.class); From c8155fd8c2b12ca69fd5877fd21327ec746e1225 Mon Sep 17 00:00:00 2001 From: Dmitriy Govorukhin Date: Fri, 21 Apr 2017 16:23:00 +0300 Subject: [PATCH 050/357] ignite-1.9.2 backport of ignite-4949 --- .../store/jdbc/CacheAbstractJdbcStore.java | 66 +++++---- .../cache/store/CacheOsStoreManager.java | 3 + .../store/GridCacheStoreManagerAdapter.java | 9 ++ .../CacheJdbcPojoStoreAbstractSelfTest.java | 19 ++- ...naryMarshallerStoreKeepBinarySelfTest.java | 28 ++++ ...rStoreKeepBinaryWithSqlEscapeSelfTest.java | 28 ++++ .../store/jdbc/CacheJdbcPojoStoreTest.java | 136 ++++++++++++++---- .../testframework/junits/IgniteMock.java | 4 + 8 files changed, 233 insertions(+), 60 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerStoreKeepBinarySelfTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerStoreKeepBinaryWithSqlEscapeSelfTest.java 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 7af238b1bc112..57bf311d2a9d8 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 @@ -48,7 +48,6 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; -import org.apache.ignite.binary.BinaryObject; import org.apache.ignite.cache.CacheTypeFieldMetadata; import org.apache.ignite.cache.CacheTypeMetadata; import org.apache.ignite.cache.store.CacheStore; @@ -61,6 +60,7 @@ import org.apache.ignite.cache.store.jdbc.dialect.OracleDialect; import org.apache.ignite.cache.store.jdbc.dialect.SQLServerDialect; import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.binary.BinaryEnumObjectImpl; import org.apache.ignite.internal.binary.BinaryMarshaller; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.typedef.C1; @@ -556,22 +556,35 @@ private JdbcTypeField[] translateFields(Collection oldFl /** * @param type Type name to check. + * @param binarySupported True if binary marshaller enable. * @return {@code True} if class not found. */ - protected TypeKind kindForName(String type) { + protected TypeKind kindForName(String type, boolean binarySupported) { if (BUILT_IN_TYPES.contains(type)) return TypeKind.BUILT_IN; + if (binarySupported) + return TypeKind.BINARY; + try { Class.forName(type); return TypeKind.POJO; } - catch(ClassNotFoundException ignored) { - return TypeKind.BINARY; + catch (ClassNotFoundException e) { + throw new CacheException("Can not find class " + type + + ", check your classPath or try to use BinaryMarshaller", e); } } + /** + * @param type Type name to check. + * @return {@code True} if class not found. + */ + protected TypeKind kindForName(String type) { + return kindForName(type, ignite.configuration().getMarshaller() instanceof BinaryMarshaller); + } + /** * @param cacheName Cache name to check mappings for. * @return Type mappings for specified cache name. @@ -636,11 +649,7 @@ private Map getOrCreateCacheMappings(@Nullable String cach String keyType = type.getKeyType(); String valType = type.getValueType(); - TypeKind keyKind = kindForName(keyType); - - if (!binarySupported && keyKind == TypeKind.BINARY) - throw new CacheException("Key type has no class [cache=" + U.maskName(cacheName) + - ", type=" + keyType + "]"); + TypeKind keyKind = kindForName(keyType, binarySupported); checkTypeConfiguration(cacheName, keyKind, keyType, type.getKeyFields()); @@ -650,21 +659,11 @@ private Map getOrCreateCacheMappings(@Nullable String cach throw new CacheException("Key type must be unique in type metadata [cache=" + U.maskName(cacheName) + ", type=" + keyType + "]"); - TypeKind valKind = kindForName(valType); + TypeKind valKind = kindForName(valType, binarySupported); checkTypeConfiguration(cacheName, valKind, valType, type.getValueFields()); entryMappings.put(keyTypeId, new EntryMapping(cacheName, dialect, type, keyKind, valKind, sqlEscapeAll)); - - // Add one more binding to binary typeId for POJOs, - // because object could be passed to store in binary format. - if (binarySupported && keyKind == TypeKind.POJO) { - keyTypeId = typeIdForTypeName(TypeKind.BINARY, keyType); - - valKind = valKind == TypeKind.POJO ? TypeKind.BINARY : valKind; - - entryMappings.put(keyTypeId, new EntryMapping(cacheName, dialect, type, TypeKind.BINARY, valKind, sqlEscapeAll)); - } } Map> mappings = new HashMap<>(cacheMappings); @@ -1398,10 +1397,17 @@ protected void fillParameter(PreparedStatement stmt, int idx, JdbcTypeField fiel // No-op. } } - else if (field.getJavaFieldType().isEnum() && fieldVal instanceof Enum) { - Enum val = (Enum)fieldVal; + else if (field.getJavaFieldType().isEnum()) { + if (fieldVal instanceof Enum) { + Enum val = (Enum)fieldVal; - fieldVal = NUMERIC_TYPES.contains(field.getDatabaseFieldType()) ? val.ordinal() : val.name(); + fieldVal = NUMERIC_TYPES.contains(field.getDatabaseFieldType()) ? val.ordinal() : val.name(); + } + else if (fieldVal instanceof BinaryEnumObjectImpl) { + BinaryEnumObjectImpl val = (BinaryEnumObjectImpl)fieldVal; + + fieldVal = val.enumOrdinal(); + } } stmt.setObject(idx, fieldVal); @@ -1453,14 +1459,14 @@ protected int fillKeyParameters(PreparedStatement stmt, EntryMapping m, Object k */ protected int fillValueParameters(PreparedStatement stmt, int idx, EntryMapping em, Object val) throws CacheWriterException { - TypeKind valKind = em.valueKind(); - - // Object could be passed by cache in binary format in case of cache configured with setStoreKeepBinary(true). - if (valKind == TypeKind.POJO && val instanceof BinaryObject) - valKind = TypeKind.BINARY; - for (JdbcTypeField field : em.uniqValFlds) { - Object fieldVal = extractParameter(em.cacheName, em.valueType(), valKind, field.getJavaFieldName(), val); + Object fieldVal = extractParameter( + em.cacheName, + em.valueType(), + em.valueKind(), + field.getJavaFieldName(), + val + ); fillParameter(stmt, idx++, field, fieldVal); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java index 27771ff3e629a..2e23d04f735d8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java @@ -77,6 +77,9 @@ public CacheOsStoreManager(GridKernalContext ctx, CacheConfiguration cfg) { /** {@inheritDoc} */ @Override public boolean convertBinary() { + if (alwaysKeepBinary) + return false; + return configuredConvertBinary() && !(cfgStore instanceof PlatformCacheStore); } 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 e9e90ea3c25d8..814166d7dafac 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 @@ -31,10 +31,13 @@ import javax.cache.integration.CacheLoaderException; import javax.cache.integration.CacheWriterException; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; +import org.apache.ignite.cache.store.CacheStore; import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.cache.store.CacheStore; import org.apache.ignite.cache.store.CacheStoreSession; import org.apache.ignite.cache.store.CacheStoreSessionListener; +import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStore; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.internal.GridKernalContext; @@ -113,6 +116,9 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt /** */ private boolean globalSesLsnrs; + /** Always keep binary. */ + protected boolean alwaysKeepBinary; + /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public void initialize(@Nullable CacheStore cfgStore, Map sesHolders) throws IgniteCheckedException { @@ -148,6 +154,9 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt sesHolder = sesHolder0; locStore = U.hasAnnotation(cfgStore, CacheLocalStore.class); + + if (cfgStore instanceof CacheJdbcPojoStore) + alwaysKeepBinary = true; } /** {@inheritDoc} */ 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 e890109574156..d6d235221448c 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 @@ -35,6 +35,7 @@ import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.ConnectorConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.binary.BinaryMarshaller; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; @@ -221,6 +222,7 @@ protected CacheConfiguration cacheConfiguration() throws Exception { cc.setAtomicityMode(transactional ? TRANSACTIONAL : ATOMIC); cc.setSwapEnabled(false); cc.setWriteBehindEnabled(false); + cc.setStoreKeepBinary(storeKeepBinary()); CacheJdbcPojoStoreFactory storeFactory = new CacheJdbcPojoStoreFactory<>(); storeFactory.setDialect(new H2Dialect()); @@ -237,6 +239,13 @@ protected CacheConfiguration cacheConfiguration() throws Exception { return cc; } + /** + * @return Flag indicate keep value in binary format or not. + */ + protected boolean storeKeepBinary(){ + return false; + } + /** * Fill in-memory database with sample data. * @@ -397,6 +406,8 @@ public void testLoadCachePrimitiveKeysTx() throws Exception { * @throws Exception If failed. */ private void checkPutRemove() throws Exception { + boolean binaryMarshaller = marshaller() instanceof BinaryMarshaller || marshaller() == null; + IgniteCache c1 = grid().cache(CACHE_NAME); Connection conn = getConnection(); @@ -429,7 +440,9 @@ private void checkPutRemove() throws Exception { assertEquals(-2, rs.getInt(2)); assertEquals(testDate, rs.getDate(3)); assertEquals("Person-to-test-put-insert", rs.getString(4)); - assertEquals(testGender.toString(), rs.getString(5)); + + assertEquals(testGender.toString(), + binaryMarshaller ? Gender.values()[rs.getInt(5)].toString(): rs.getString(5)); assertFalse("Unexpected more data in result set", rs.next()); @@ -448,7 +461,9 @@ private void checkPutRemove() throws Exception { assertEquals(-3, rs.getInt(2)); assertEquals(testDate, rs.getDate(3)); assertEquals("Person-to-test-put-update", rs.getString(4)); - assertEquals(testGender.toString(), rs.getString(5)); + + assertEquals(testGender.toString(), + binaryMarshaller ? Gender.values()[rs.getInt(5)].toString(): rs.getString(5)); assertFalse("Unexpected more data in result set", rs.next()); diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerStoreKeepBinarySelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerStoreKeepBinarySelfTest.java new file mode 100644 index 0000000000000..dfca8647e2840 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerStoreKeepBinarySelfTest.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.cache.store.jdbc; + +/** + * + */ +public class CacheJdbcPojoStoreBinaryMarshallerStoreKeepBinarySelfTest extends CacheJdbcPojoStoreBinaryMarshallerSelfTest { + /** {@inheritDoc} */ + @Override protected boolean storeKeepBinary() { + return true; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerStoreKeepBinaryWithSqlEscapeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerStoreKeepBinaryWithSqlEscapeSelfTest.java new file mode 100644 index 0000000000000..c7e1f794ab208 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerStoreKeepBinaryWithSqlEscapeSelfTest.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.cache.store.jdbc; + +/** + * + */ +public class CacheJdbcPojoStoreBinaryMarshallerStoreKeepBinaryWithSqlEscapeSelfTest extends CacheJdbcPojoStoreBinaryMarshallerWithSqlEscapeSelfTest { + /** {@inheritDoc} */ + @Override protected boolean storeKeepBinary() { + return true; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreTest.java index 4a0b1daf646d1..bb85cabbd25a5 100644 --- a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreTest.java +++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreTest.java @@ -17,6 +17,7 @@ package org.apache.ignite.cache.store.jdbc; +import java.lang.reflect.Field; import java.sql.Connection; import java.sql.PreparedStatement; import java.sql.SQLException; @@ -28,18 +29,22 @@ import java.util.UUID; import java.util.concurrent.ConcurrentLinkedQueue; import javax.cache.integration.CacheWriterException; - +import org.apache.ignite.Ignite; +import org.apache.ignite.binary.BinaryObject; +import org.apache.ignite.binary.BinaryObjectBuilder; import org.apache.ignite.cache.store.jdbc.dialect.H2Dialect; import org.apache.ignite.cache.store.jdbc.model.Organization; import org.apache.ignite.cache.store.jdbc.model.OrganizationKey; import org.apache.ignite.cache.store.jdbc.model.Person; import org.apache.ignite.cache.store.jdbc.model.PersonComplexKey; import org.apache.ignite.cache.store.jdbc.model.PersonKey; +import org.apache.ignite.internal.binary.BinaryMarshaller; import org.apache.ignite.internal.processors.cache.CacheEntryImpl; import org.apache.ignite.internal.util.typedef.CI2; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteBiInClosure; import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.config.GridTestProperties; import org.apache.ignite.testframework.junits.cache.GridAbstractCacheStoreSelfTest; import org.h2.jdbcx.JdbcConnectionPool; @@ -56,6 +61,12 @@ public class CacheJdbcPojoStoreTest extends GridAbstractCacheStoreSelfTest orgKeys = new ConcurrentLinkedQueue<>(); - final Collection prnKeys = new ConcurrentLinkedQueue<>(); - final Collection prnComplexKeys = new ConcurrentLinkedQueue<>(); + final Collection orgKeys = new ConcurrentLinkedQueue<>(); + final Collection prnKeys = new ConcurrentLinkedQueue<>(); + final Collection prnComplexKeys = new ConcurrentLinkedQueue<>(); IgniteBiInClosure c = new CI2() { @Override public void apply(Object k, Object v) { - if (k instanceof OrganizationKey && v instanceof Organization) - orgKeys.add((OrganizationKey)k); - else if (k instanceof PersonKey && v instanceof Person) - prnKeys.add((PersonKey)k); - else if (k instanceof PersonComplexKey && v instanceof Person) { - PersonComplexKey key = (PersonComplexKey)k; - - Person val = (Person)v; - - assertTrue("Key ID should be the same as value ID", key.getId() == val.getId()); - assertTrue("Key orgID should be the same as value orgID", key.getOrgId() == val.getOrgId()); - assertEquals("name" + key.getId(), val.getName()); - - prnComplexKeys.add((PersonComplexKey)k); + if (binaryEnable){ + if (k instanceof BinaryObject && v instanceof BinaryObject) { + BinaryObject key = (BinaryObject)k; + BinaryObject val = (BinaryObject)v; + + String keyType = key.type().typeName(); + String valType = val.type().typeName(); + + if (OrganizationKey.class.getName().equals(keyType) + && Organization.class.getName().equals(valType)) + orgKeys.add(key); + + if (PersonKey.class.getName().equals(keyType) + && Person.class.getName().equals(valType)) + prnKeys.add(key); + + if (PersonComplexKey.class.getName().equals(keyType) + && Person.class.getName().equals(valType)) + prnComplexKeys.add(key); + } + }else { + if (k instanceof OrganizationKey && v instanceof Organization) + orgKeys.add(k); + else if (k instanceof PersonKey && v instanceof Person) + prnKeys.add(k); + else if (k instanceof PersonComplexKey && v instanceof Person) { + PersonComplexKey key = (PersonComplexKey)k; + + Person val = (Person)v; + + assertTrue("Key ID should be the same as value ID", key.getId() == val.getId()); + assertTrue("Key orgID should be the same as value orgID", key.getOrgId() == val.getOrgId()); + assertEquals("name" + key.getId(), val.getName()); + + prnComplexKeys.add(k); + } } } }; @@ -319,15 +358,16 @@ else if (k instanceof PersonComplexKey && v instanceof Person) { assertEquals(PERSON_CNT, prnKeys.size()); assertEquals(PERSON_CNT, prnComplexKeys.size()); - Collection tmpOrgKeys = new ArrayList<>(orgKeys); - Collection tmpPrnKeys = new ArrayList<>(prnKeys); - Collection tmpPrnComplexKeys = new ArrayList<>(prnComplexKeys); + Collection tmpOrgKeys = new ArrayList<>(orgKeys); + Collection tmpPrnKeys = new ArrayList<>(prnKeys); + Collection tmpPrnComplexKeys = new ArrayList<>(prnComplexKeys); orgKeys.clear(); prnKeys.clear(); prnComplexKeys.clear(); - store.loadCache(c, OrganizationKey.class.getName(), "SELECT name, city, id FROM ORGANIZATION", + store.loadCache( + c, OrganizationKey.class.getName(), "SELECT name, city, id FROM ORGANIZATION", PersonKey.class.getName(), "SELECT org_id, id, name FROM Person WHERE id < 1000"); assertEquals(ORGANIZATION_CNT, orgKeys.size()); @@ -377,14 +417,29 @@ public void testParallelLoad() throws Exception { U.closeQuiet(conn); - final Collection prnComplexKeys = new ConcurrentLinkedQueue<>(); + final Collection prnComplexKeys = new ConcurrentLinkedQueue<>(); IgniteBiInClosure c = new CI2() { @Override public void apply(Object k, Object v) { - if (k instanceof PersonComplexKey && v instanceof Person) - prnComplexKeys.add((PersonComplexKey)k); - else - fail("Unexpected entry [key=" + k + ", value=" + v + "]"); + if (binaryEnable) { + if (k instanceof BinaryObject && v instanceof BinaryObject) { + BinaryObject key = (BinaryObject)k; + BinaryObject val = (BinaryObject)v; + + String keyType = key.type().typeName(); + String valType = val.type().typeName(); + + if (PersonComplexKey.class.getName().equals(keyType) + && Person.class.getName().equals(valType)) + prnComplexKeys.add(key); + } + } + else { + if (k instanceof PersonComplexKey && v instanceof Person) + prnComplexKeys.add(k); + else + fail("Unexpected entry [key=" + k + ", value=" + v + "]"); + } } }; @@ -437,7 +492,7 @@ public void testWriteRetry() throws Exception { ses.newSession(null); try { - store.write(new CacheEntryImpl<>(k1, v1)); + store.write(new CacheEntryImpl<>(wrap(k1), wrap(v1))); fail("CacheWriterException wasn't thrown."); } @@ -466,4 +521,29 @@ public void testTimestamp() throws Exception { assertNull(store.load(k)); } + + /** + * @param obj Object. + */ + private Object wrap(Object obj) throws IllegalAccessException { + if (binaryEnable) { + Class cls = obj.getClass(); + + BinaryObjectBuilder builder = ig.binary().builder(cls.getName()); + + for (Field f : cls.getDeclaredFields()) { + if (f.getName().contains("serialVersionUID")) + continue; + + f.setAccessible(true); + + builder.setField(f.getName(), f.get(obj)); + } + + return builder.build(); + } + + return obj; + } + } diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java index 5722fa3d6056a..14ba030e24590 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java @@ -55,6 +55,7 @@ import org.apache.ignite.configuration.NearCacheConfiguration; import org.apache.ignite.internal.binary.BinaryCachingMetadataHandler; import org.apache.ignite.internal.binary.BinaryContext; +import org.apache.ignite.internal.binary.BinaryMarshaller; import org.apache.ignite.internal.binary.builder.BinaryObjectBuilderImpl; import org.apache.ignite.internal.processors.cacheobject.NoOpBinary; import org.apache.ignite.internal.util.typedef.internal.U; @@ -327,6 +328,9 @@ public IgniteMock( return typeName.hashCode(); } }; + + if (marshaller instanceof BinaryMarshaller) + ctx.configure((BinaryMarshaller)marshaller, configuration()); } binaryMock = new NoOpBinary() { From 8cae6dfd7ed000d06b37ad53f5d04aa3fd534d66 Mon Sep 17 00:00:00 2001 From: Alexander Belyak Date: Fri, 21 Apr 2017 16:53:05 +0300 Subject: [PATCH 051/357] IGNITE-4927 Use write behind coalescing from CacheConfiguration in GridCacheStoreManagerAdapter to configure actual cache store. Signed-off-by: nikolay_tikhonov --- .../store/GridCacheWriteBehindStore.java | 20 ++++--- ...heStoreSessionWriteBehindAbstractTest.java | 56 ++++++++++++----- ...StoreSessionWriteBehindCoalescingTest.java | 60 ++++++++++++++++--- 3 files changed, 104 insertions(+), 32 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java index 4a1b11e807bf3..8537aaba69282 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java @@ -385,8 +385,10 @@ public int getWriteBehindErrorRetryCount() { if (log.isDebugEnabled()) log.debug("Stopping write-behind store for cache '" + cacheName + '\''); - for (Flusher f : flushThreads) { - if (!f.isEmpty()) + if (writeCoalescing) + wakeUp(); + else { + for (Flusher f : flushThreads) f.wakeUp(); } @@ -897,7 +899,7 @@ protected Flusher(String gridName, IgniteLogger log) { super(gridName, name, log); - flusherCacheCriticalSize = cacheCriticalSize/flushThreadCnt; + flusherCacheCriticalSize = cacheCriticalSize / flushThreadCnt; assert flusherCacheCriticalSize > batchSize; @@ -930,7 +932,9 @@ private void putToFlusherWriteCache( throws IgniteInterruptedCheckedException { assert !writeCoalescing : "Unexpected write coalescing."; - if (queue.sizex() > flusherCacheCriticalSize) { + int qSize = queue.sizex(); + + if (qSize > flusherCacheCriticalSize) { while (queue.sizex() > flusherCacheCriticalSize) { wakeUp(); @@ -960,6 +964,8 @@ private void putToFlusherWriteCache( cacheTotalOverflowCntr.incrementAndGet(); } + else if (qSize > batchSize) + wakeUp(); queue.add(F.t(key, newVal)); @@ -1055,6 +1061,9 @@ private void awaitOperationsAvailableNonCoalescing() throws InterruptedException if (queue.sizex() >= batchSize) return; + if (stopping.get()) + return; + if (cacheFlushFreq > 0) LockSupport.parkNanos(cacheFlushFreqNanos); else @@ -1065,9 +1074,6 @@ private void awaitOperationsAvailableNonCoalescing() throws InterruptedException if (Thread.interrupted()) throw new InterruptedException(); - - if (stopping.get()) - return; } } finally { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheStoreSessionWriteBehindAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheStoreSessionWriteBehindAbstractTest.java index bbb6f4c68e921..4b0f7c0bb22c0 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheStoreSessionWriteBehindAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheStoreSessionWriteBehindAbstractTest.java @@ -48,6 +48,9 @@ public abstract class IgniteCacheStoreSessionWriteBehindAbstractTest extends Ign /** */ private static volatile CountDownLatch latch; + /** */ + protected static volatile CountDownLatch entLatch; + /** */ private static volatile ExpectedData expData; @@ -66,32 +69,38 @@ public abstract class IgniteCacheStoreSessionWriteBehindAbstractTest extends Ign return null; } - /** {@inheritDoc} */ + /** + * @param igniteInstanceName Ignite instance name. + * @return Cache configuration. + * @throws Exception In case of error. + */ @SuppressWarnings("unchecked") - @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { - IgniteConfiguration cfg = super.getConfiguration(gridName); + protected CacheConfiguration cacheConfiguration(String igniteInstanceName) throws Exception { + CacheConfiguration ccfg0 = super.cacheConfiguration(igniteInstanceName); - assert cfg.getCacheConfiguration().length == 1; - - CacheConfiguration ccfg0 = cfg.getCacheConfiguration()[0]; ccfg0.setReadThrough(true); ccfg0.setWriteThrough(true); ccfg0.setWriteBehindBatchSize(10); ccfg0.setWriteBehindFlushSize(10); - ccfg0.setWriteBehindFlushFrequency(60_000); + ccfg0.setWriteBehindFlushFrequency(600); ccfg0.setWriteBehindEnabled(true); ccfg0.setCacheStoreFactory(singletonFactory(new TestStore())); - CacheConfiguration ccfg1 = cacheConfiguration(gridName); + return ccfg0; + } - ccfg1.setReadThrough(true); - ccfg1.setWriteThrough(true); - ccfg1.setWriteBehindBatchSize(10); - ccfg1.setWriteBehindFlushSize(10); - ccfg1.setWriteBehindFlushFrequency(60_000); - ccfg1.setWriteBehindEnabled(true); + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + assert cfg.getCacheConfiguration().length == 1; + + CacheConfiguration ccfg0 = cacheConfiguration(igniteInstanceName); + + CacheConfiguration ccfg1 = cacheConfiguration(igniteInstanceName); ccfg1.setName(CACHE_NAME1); @@ -120,6 +129,7 @@ private void testCache(String cacheName) throws Exception { try { latch = new CountDownLatch(2); + entLatch = new CountDownLatch(11); expData = new ExpectedData("writeAll", cacheName); @@ -127,13 +137,17 @@ private void testCache(String cacheName) throws Exception { cache.put(i, i); assertTrue(latch.await(10_000, TimeUnit.MILLISECONDS)); + + assertTrue(entLatch.await(10_000,TimeUnit.MILLISECONDS)); } finally { latch = null; + entLatch = null; } try { latch = new CountDownLatch(2); + entLatch = new CountDownLatch(11); expData = new ExpectedData("deleteAll", cacheName); @@ -141,16 +155,20 @@ private void testCache(String cacheName) throws Exception { cache.remove(i); assertTrue(latch.await(10_000, TimeUnit.MILLISECONDS)); + + assertTrue(entLatch.await(10_000,TimeUnit.MILLISECONDS)); } finally { latch = null; + entLatch = null; } } /** * */ - private class TestStore implements CacheStore { + protected class TestStore implements CacheStore { + /** Auto-injected store session. */ @CacheStoreSessionResource private CacheStoreSession ses; @@ -195,6 +213,9 @@ private class TestStore implements CacheStore { assertTrue("Unexpected entries: " + entries, entries.size() == 10 || entries.size() == 1); checkSession("writeAll"); + + for (int i = 0; i < entries.size(); i++) + entLatch.countDown(); } /** {@inheritDoc} */ @@ -209,6 +230,9 @@ private class TestStore implements CacheStore { assertTrue("Unexpected keys: " + keys, keys.size() == 10 || keys.size() == 1); checkSession("deleteAll"); + + for (int i = 0; i < keys.size(); i++) + entLatch.countDown(); } /** @@ -221,7 +245,7 @@ private CacheStoreSession session() { /** * @param mtd Called stored method. */ - private void checkSession(String mtd) { + protected void checkSession(String mtd) { assertNotNull(ignite); CacheStoreSession ses = session(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheTxStoreSessionWriteBehindCoalescingTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheTxStoreSessionWriteBehindCoalescingTest.java index c15b2f4e2c334..ad7b7c2bf50c2 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheTxStoreSessionWriteBehindCoalescingTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheTxStoreSessionWriteBehindCoalescingTest.java @@ -17,11 +17,13 @@ package org.apache.ignite.internal.processors.cache.integration; -import org.apache.ignite.IgniteCache; +import java.util.Collection; +import javax.cache.Cache; +import javax.cache.integration.CacheWriterException; import org.apache.ignite.cache.CacheAtomicityMode; -import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; import org.apache.ignite.configuration.CacheConfiguration; -import org.apache.ignite.configuration.IgniteConfiguration; + +import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; /** * Integration test write behind cache store with {@link CacheConfiguration#getWriteBehindCoalescing()}={@code False} @@ -33,15 +35,55 @@ public class IgniteCacheTxStoreSessionWriteBehindCoalescingTest extends IgniteCa return TRANSACTIONAL; } - /** {@inheritDoc} */ - @SuppressWarnings("unchecked") - @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { - IgniteConfiguration result = super.getConfiguration(gridName); - CacheConfiguration ccfg = result.getCacheConfiguration()[0]; + /** + * @param igniteInstanceName Ignite instance name. + * @return Cache configuration. + * @throws Exception In case of error. + */ + @SuppressWarnings("unchecked") + protected CacheConfiguration cacheConfiguration(String igniteInstanceName) throws Exception { + CacheConfiguration ccfg = super.cacheConfiguration(igniteInstanceName); ccfg.setWriteBehindCoalescing(false); - return result; + ccfg.setCacheStoreFactory(singletonFactory(new TestNonCoalescingStore())); + + return ccfg; + } + + /** + * + */ + private class TestNonCoalescingStore extends TestStore { + + /** {@inheritDoc} */ + @Override public void writeAll(Collection> entries) throws CacheWriterException { + log.info("writeAll: " + entries); + + assertTrue("Unexpected entries: " + entries, entries.size() <= 10); + + checkSession("writeAll"); + + for (int i = 0; i < entries.size(); i++) + entLatch.countDown(); + } + + /** {@inheritDoc} */ + @Override public void delete(Object key) throws CacheWriterException { + fail(); + } + + /** {@inheritDoc} */ + @Override public void deleteAll(Collection keys) throws CacheWriterException { + log.info("deleteAll: " + keys); + + assertTrue("Unexpected keys: " + keys, keys.size() <= 10); + + checkSession("deleteAll"); + + for (int i = 0; i < keys.size(); i++) + entLatch.countDown(); + } } } \ No newline at end of file From fde6d4427131f4b74fcdf88f1caae4f57d225a87 Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Thu, 20 Apr 2017 19:39:45 +0300 Subject: [PATCH 052/357] IGNITE-5043 .NET: CacheConfiguration.WriteBehindCoalescing (cherry picked from commit 217c6be) --- .../utils/PlatformConfigurationUtils.java | 4 +- .../Cache/CacheConfigurationTest.cs | 3 + .../IgniteConfigurationSerializerTest.cs | 4 +- .../Cache/Configuration/CacheConfiguration.cs | 62 +++++++++++++------ .../IgniteConfigurationSection.xsd | 7 +++ 5 files changed, 59 insertions(+), 21 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java index daec38b9bfaf9..5ca85e8c4143c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java @@ -166,11 +166,11 @@ public static CacheConfiguration readCacheConfiguration(BinaryRawReaderEx in) { ccfg.setSqlOnheapRowCacheSize(in.readInt()); ccfg.setStartSize(in.readInt()); ccfg.setWriteBehindBatchSize(in.readInt()); - ccfg.setWriteBehindCoalescing(in.readBoolean()); ccfg.setWriteBehindEnabled(in.readBoolean()); ccfg.setWriteBehindFlushFrequency(in.readLong()); ccfg.setWriteBehindFlushSize(in.readInt()); ccfg.setWriteBehindFlushThreadCount(in.readInt()); + ccfg.setWriteBehindCoalescing(in.readBoolean()); ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.fromOrdinal(in.readInt())); ccfg.setReadThrough(in.readBoolean()); ccfg.setWriteThrough(in.readBoolean()); @@ -792,11 +792,11 @@ public static void writeCacheConfiguration(BinaryRawWriter writer, CacheConfigur writer.writeInt(ccfg.getSqlOnheapRowCacheSize()); writer.writeInt(ccfg.getStartSize()); writer.writeInt(ccfg.getWriteBehindBatchSize()); - writer.writeBoolean(ccfg.getWriteBehindCoalescing()); writer.writeBoolean(ccfg.isWriteBehindEnabled()); writer.writeLong(ccfg.getWriteBehindFlushFrequency()); writer.writeInt(ccfg.getWriteBehindFlushSize()); writer.writeInt(ccfg.getWriteBehindFlushThreadCount()); + writer.writeBoolean(ccfg.getWriteBehindCoalescing()); writeEnumInt(writer, ccfg.getWriteSynchronizationMode()); writer.writeBoolean(ccfg.isReadThrough()); writer.writeBoolean(ccfg.isWriteThrough()); diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs index fb8725c59b221..aea5989c56b2e 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs @@ -212,6 +212,8 @@ private static void AssertConfigIsDefault(CacheConfiguration cfg) Assert.AreEqual(CacheConfiguration.DefaultWriteBehindEnabled, cfg.WriteBehindEnabled); Assert.AreEqual(CacheConfiguration.DefaultWriteBehindFlushFrequency, cfg.WriteBehindFlushFrequency); Assert.AreEqual(CacheConfiguration.DefaultWriteBehindFlushSize, cfg.WriteBehindFlushSize); + Assert.AreEqual(CacheConfiguration.DefaultWriteBehindFlushThreadCount, cfg.WriteBehindFlushThreadCount); + Assert.AreEqual(CacheConfiguration.DefaultWriteBehindCoalescing, cfg.WriteBehindCoalescing); } /// @@ -520,6 +522,7 @@ private static CacheConfiguration GetCustomCacheConfiguration(string name = null CacheStoreFactory = new CacheStoreFactoryTest(), ReadThrough = true, WriteThrough = true, + WriteBehindCoalescing = false, QueryEntities = new[] { new QueryEntity diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs index 26e04a9ba34e2..8bf518b415955 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs @@ -91,7 +91,7 @@ public void TestPredefinedXml() - + @@ -162,6 +162,7 @@ public void TestPredefinedXml() Assert.IsTrue(cacheCfg.WriteThrough); Assert.IsInstanceOf(cacheCfg.ExpiryPolicyFactory); Assert.IsTrue(cacheCfg.EnableStatistics); + Assert.IsFalse(cacheCfg.WriteBehindCoalescing); var queryEntity = cacheCfg.QueryEntities.Single(); Assert.AreEqual(typeof(int), queryEntity.KeyType); @@ -645,6 +646,7 @@ private static IgniteConfiguration GetTestConfig() WriteBehindFlushFrequency = TimeSpan.FromSeconds(55), WriteBehindFlushSize = 66, WriteBehindFlushThreadCount = 2, + WriteBehindCoalescing = false, WriteSynchronizationMode = CacheWriteSynchronizationMode.FullAsync, NearConfiguration = new NearCacheConfiguration { diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs index 2795111d66ca3..69befad2e4d23 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs @@ -150,6 +150,9 @@ public class CacheConfiguration /// Default value for write-through behavior. public const bool DefaultWriteThrough = false; + /// Default value for . + public const bool DefaultWriteBehindCoalescing = true; + /// /// Gets or sets the cache name. /// @@ -202,6 +205,7 @@ public CacheConfiguration(string name) WriteBehindFlushFrequency = DefaultWriteBehindFlushFrequency; WriteBehindFlushSize = DefaultWriteBehindFlushSize; WriteBehindFlushThreadCount= DefaultWriteBehindFlushThreadCount; + WriteBehindCoalescing = DefaultWriteBehindCoalescing; } /// @@ -269,6 +273,7 @@ internal CacheConfiguration(IBinaryRawReader reader) WriteBehindFlushFrequency = reader.ReadLongAsTimespan(); WriteBehindFlushSize = reader.ReadInt(); WriteBehindFlushThreadCount = reader.ReadInt(); + WriteBehindCoalescing = reader.ReadBoolean(); WriteSynchronizationMode = (CacheWriteSynchronizationMode) reader.ReadInt(); ReadThrough = reader.ReadBoolean(); WriteThrough = reader.ReadBoolean(); @@ -326,6 +331,7 @@ internal void Write(IBinaryRawWriter writer) writer.WriteLong((long) WriteBehindFlushFrequency.TotalMilliseconds); writer.WriteInt(WriteBehindFlushSize); writer.WriteInt(WriteBehindFlushThreadCount); + writer.WriteBoolean(WriteBehindCoalescing); writer.WriteInt((int) WriteSynchronizationMode); writer.WriteBoolean(ReadThrough); writer.WriteBoolean(WriteThrough); @@ -382,11 +388,11 @@ internal void Validate(ILogger log) public CacheWriteSynchronizationMode WriteSynchronizationMode { get; set; } /// - /// Gets or sets flag indicating whether eviction is synchronized between primary, backup and near nodes. + /// Gets or sets flag indicating whether eviction is synchronized between primary, backup and near nodes. /// If this parameter is true and swap is disabled then - /// will involve all nodes where an entry is kept. - /// If this property is set to false then eviction is done independently on different cache nodes. - /// Note that it's not recommended to set this value to true if cache store is configured since it will allow + /// will involve all nodes where an entry is kept. + /// If this property is set to false then eviction is done independently on different cache nodes. + /// Note that it's not recommended to set this value to true if cache store is configured since it will allow /// to significantly improve cache performance. /// [DefaultValue(DefaultEvictSynchronized)] @@ -399,12 +405,12 @@ internal void Validate(ILogger log) public int EvictSynchronizedKeyBufferSize { get; set; } /// - /// Gets or sets concurrency level for synchronized evictions. - /// This flag only makes sense with set to true. - /// When synchronized evictions are enabled, it is possible that local eviction policy will try - /// to evict entries faster than evictions can be synchronized with backup or near nodes. - /// This value specifies how many concurrent synchronous eviction sessions should be allowed - /// before the system is forced to wait and let synchronous evictions catch up with the eviction policy. + /// Gets or sets concurrency level for synchronized evictions. + /// This flag only makes sense with set to true. + /// When synchronized evictions are enabled, it is possible that local eviction policy will try + /// to evict entries faster than evictions can be synchronized with backup or near nodes. + /// This value specifies how many concurrent synchronous eviction sessions should be allowed + /// before the system is forced to wait and let synchronous evictions catch up with the eviction policy. /// [DefaultValue(DefaultEvictSynchronizedConcurrencyLevel)] public int EvictSynchronizedConcurrencyLevel { get; set; } @@ -416,11 +422,11 @@ internal void Validate(ILogger log) public TimeSpan EvictSynchronizedTimeout { get; set; } /// - /// This value denotes the maximum size of eviction queue in percents of cache size + /// This value denotes the maximum size of eviction queue in percents of cache size /// in case of distributed cache (replicated and partitioned) and using synchronized eviction - /// - /// That queue is used internally as a buffer to decrease network costs for synchronized eviction. - /// Once queue size reaches specified value all required requests for all entries in the queue + /// + /// That queue is used internally as a buffer to decrease network costs for synchronized eviction. + /// Once queue size reaches specified value all required requests for all entries in the queue /// are sent to remote nodes and the queue is cleared. /// [DefaultValue(DefaultMaxEvictionOverflowRatio)] @@ -562,7 +568,7 @@ internal void Validate(ILogger log) /// /// Maximum batch size for write-behind cache store operations. /// Store operations (get or remove) are combined in a batch of this size to be passed to - /// or methods. + /// or methods. /// [DefaultValue(DefaultWriteBehindBatchSize)] public int WriteBehindBatchSize { get; set; } @@ -596,8 +602,8 @@ internal void Validate(ILogger log) /// /// Gets or sets maximum amount of memory available to off-heap storage. Possible values are - /// -1 means that off-heap storage is disabled. 0 means that Ignite will not limit off-heap storage - /// (it's up to user to properly add and remove entries from cache to ensure that off-heap storage + /// -1 means that off-heap storage is disabled. 0 means that Ignite will not limit off-heap storage + /// (it's up to user to properly add and remove entries from cache to ensure that off-heap storage /// does not grow indefinitely. /// Any positive value specifies the limit of off-heap storage in bytes. /// @@ -700,7 +706,7 @@ internal void Validate(ILogger log) public IAffinityFunction AffinityFunction { get; set; } /// - /// Gets or sets the factory for to be used for all cache operations, + /// Gets or sets the factory for to be used for all cache operations, /// unless is called. /// /// Default is null, which means no expiration. @@ -712,5 +718,25 @@ internal void Validate(ILogger log) /// These statistics can be retrieved via . /// public bool EnableStatistics { get; set; } + + /// + /// Gets or sets the plugin configurations. + /// + [SuppressMessage("Microsoft.Usage", "CA2227:CollectionPropertiesShouldBeReadOnly")] + public ICollection PluginConfigurations { get; set; } + + /// + /// Gets or sets the name of the for this cache. + /// See . + /// + public string MemoryPolicyName { get; set; } + + /// + /// Gets or sets write coalescing flag for write-behind cache store operations. + /// Store operations (get or remove) with the same key are combined or coalesced to single, + /// resulting operation to reduce pressure to underlying cache store. + /// + [DefaultValue(DefaultWriteBehindCoalescing)] + public bool WriteBehindCoalescing { get; set; } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd index d54a2002bdc7b..5b512a7ce9527 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd +++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd @@ -653,6 +653,13 @@ + + + + Coalescing flag for write-behind cache store operations. Store operations (get or remove) with the same key are combined or coalesced to single, resulting operation to reduce pressure to underlying cache store. + + + From 474add7c79eb5c57b2f88ca71ceb58ed3e5cb20b Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Mon, 24 Apr 2017 14:52:07 +0300 Subject: [PATCH 053/357] IGNITE-5043 .NET: CacheConfiguration.WriteBehindCoalescing Fix merge. --- .../Cache/Configuration/CacheConfiguration.cs | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs index 69befad2e4d23..332ab2bd7d4c9 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs @@ -719,18 +719,6 @@ internal void Validate(ILogger log) /// public bool EnableStatistics { get; set; } - /// - /// Gets or sets the plugin configurations. - /// - [SuppressMessage("Microsoft.Usage", "CA2227:CollectionPropertiesShouldBeReadOnly")] - public ICollection PluginConfigurations { get; set; } - - /// - /// Gets or sets the name of the for this cache. - /// See . - /// - public string MemoryPolicyName { get; set; } - /// /// Gets or sets write coalescing flag for write-behind cache store operations. /// Store operations (get or remove) with the same key are combined or coalesced to single, From a4c70f12305c15cbf15a6e5eb7c500f883a8b317 Mon Sep 17 00:00:00 2001 From: agura Date: Thu, 20 Apr 2017 20:45:58 +0300 Subject: [PATCH 054/357] ignite-5041 NPE in deadlock detection fixed --- .../cache/DynamicCacheDescriptor.java | 24 ++ .../cache/GridCacheSharedContext.java | 16 ++ .../cache/transactions/IgniteTxManager.java | 21 +- .../cache/transactions/TxDeadlock.java | 19 +- .../cache/transactions/TxLocksResponse.java | 37 +-- ...adlockDetectionMessageMarshallingTest.java | 116 +++++++++ ...xDeadlockDetectionUnmasrhalErrorsTest.java | 225 ++++++++++++++++++ .../transactions/TxDeadlockNpeClientTest.java | 220 +++++++++++++++++ .../TxDeadlockDetectionTestSuite.java | 4 + 9 files changed, 655 insertions(+), 27 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionMessageMarshallingTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionUnmasrhalErrorsTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockNpeClientTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java index 8b62f03cb8649..1d0346a04e42a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java @@ -20,9 +20,11 @@ import java.util.HashMap; import java.util.Map; import java.util.UUID; +import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor; import org.apache.ignite.internal.processors.plugin.CachePluginManager; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.typedef.internal.CU; @@ -81,6 +83,12 @@ public class DynamicCacheDescriptor { /** */ private AffinityTopologyVersion rcvdFromVer; + /** Mutex. */ + private final Object mux = new Object(); + + /** Cached object context for marshalling issues when cache isn't started. */ + private volatile CacheObjectContext objCtx; + /** * @param ctx Context. * @param cacheCfg Cache configuration. @@ -209,6 +217,22 @@ public CacheConfiguration cacheConfiguration() { return cacheCfg; } + /** + * Creates and caches cache object context if needed. + * + * @param proc Object processor. + */ + public CacheObjectContext cacheObjectContext(IgniteCacheObjectProcessor proc) throws IgniteCheckedException { + if (objCtx == null) { + synchronized (mux) { + if (objCtx == null) + objCtx = proc.contextForCache(cacheCfg); + } + } + + return objCtx; + } + /** * @return Cache plugin manager. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java index 117a5c33189d7..da5ff2e2714fb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java @@ -392,6 +392,22 @@ public GridCacheContext cacheContext(int cacheId) { return ctxMap.get(cacheId); } + /** + * Returns cache object context if created or creates new and caches it until cache started. + * + * @param cacheId Cache id. + */ + public @Nullable CacheObjectContext cacheObjectContext(int cacheId) throws IgniteCheckedException { + GridCacheContext ctx = ctxMap.get(cacheId); + + if (ctx != null) + return ctx.cacheObjectContext(); + + DynamicCacheDescriptor desc = cache().cacheDescriptor(cacheId); + + return desc != null ? desc.cacheObjectContext(kernalContext().cacheObjects()) : null; + } + /** * @return Grid name. */ 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 24f8ea918b07a..1ba0102408eb0 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 @@ -31,7 +31,6 @@ import java.util.concurrent.ConcurrentMap; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteClientDisconnectedException; -import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.cluster.ClusterNode; @@ -2479,11 +2478,18 @@ private class DeadlockDetectionListener implements GridMessageListener { @Override public void onMessage(UUID nodeId, Object msg) { GridCacheMessage cacheMsg = (GridCacheMessage)msg; - unmarshall(nodeId, cacheMsg); + Throwable err = null; - if (cacheMsg.classError() != null) { + try { + unmarshall(nodeId, cacheMsg); + } + catch (Exception e) { + err = e; + } + + if (err != null || cacheMsg.classError() != null) { try { - processFailedMessage(nodeId, cacheMsg); + processFailedMessage(nodeId, cacheMsg, err); } catch(Throwable e){ U.error(log, "Failed to process message [senderId=" + nodeId + @@ -2536,7 +2542,7 @@ else if (msg instanceof TxLocksResponse) { * @param nodeId Node ID. * @param msg Message. */ - private void processFailedMessage(UUID nodeId, GridCacheMessage msg) throws IgniteCheckedException { + private void processFailedMessage(UUID nodeId, GridCacheMessage msg, Throwable err) throws IgniteCheckedException { switch (msg.directType()) { case -24: { TxLocksRequest req = (TxLocksRequest)msg; @@ -2568,7 +2574,10 @@ private void processFailedMessage(UUID nodeId, GridCacheMessage msg) throws Igni return; } - fut.onResult(nodeId, res); + if (err == null) + fut.onResult(nodeId, res); + else + fut.onDone(null, err); } break; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlock.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlock.java index f8130e11a2abd..97db6988052c9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlock.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlock.java @@ -21,11 +21,10 @@ import java.util.Map; import java.util.Set; import java.util.UUID; -import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.CacheObjectContext; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.util.typedef.T2; -import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.U; /** @@ -133,11 +132,21 @@ public String toString(GridCacheSharedContext ctx) { for (Map.Entry e : keyLabels.entrySet()) { IgniteTxKey txKey = e.getKey(); - GridCacheContext cctx = ctx.cacheContext(txKey.cacheId()); + try { + CacheObjectContext objCtx = ctx.cacheObjectContext(txKey.cacheId()); - Object val = CU.value(txKey.key(), cctx, true); + Object val = txKey.key().value(objCtx, true); - sb.append(e.getValue()).append(" [key=").append(val).append(", cache=").append(cctx.namexx()).append("]\n"); + sb.append(e.getValue()) + .append(" [key=") + .append(val) + .append(", cache=") + .append(objCtx.cacheName()) + .append("]\n"); + } + catch (Exception ex) { + sb.append("Unable to unmarshall deadlock information for key [key=").append(e.getValue()).append("]\n"); + } } return sb.toString(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxLocksResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxLocksResponse.java index fa6afdd0b8a8e..a2f3037afaa08 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxLocksResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxLocksResponse.java @@ -181,31 +181,36 @@ public void addKey(IgniteTxKey key) { /** {@inheritDoc} */ @Override public void finishUnmarshal(GridCacheSharedContext ctx, ClassLoader ldr) throws IgniteCheckedException { - super.finishUnmarshal(ctx, ldr); + try { + super.finishUnmarshal(ctx, ldr); - if (nearTxKeysArr != null) { - for (int i = 0; i < nearTxKeysArr.length; i++) { - IgniteTxKey key = nearTxKeysArr[i]; + if (nearTxKeysArr != null) { + for (int i = 0; i < nearTxKeysArr.length; i++) { + IgniteTxKey txKey = nearTxKeysArr[i]; - key.finishUnmarshal(ctx.cacheContext(key.cacheId()), ldr); + txKey.key().finishUnmarshal(ctx.cacheObjectContext(txKey.cacheId()), ldr); - txLocks().put(key, locksArr[i]); + txLocks().put(txKey, locksArr[i]); + } + + nearTxKeysArr = null; + locksArr = null; } - nearTxKeysArr = null; - locksArr = null; - } + if (txKeysArr != null) { + txKeys = U.newHashSet(txKeysArr.length); - if (txKeysArr != null) { - txKeys = U.newHashSet(txKeysArr.length); + for (IgniteTxKey txKey : txKeysArr) { + txKey.key().finishUnmarshal(ctx.cacheObjectContext(txKey.cacheId()), ldr); - for (IgniteTxKey key : txKeysArr) { - key.finishUnmarshal(ctx.cacheContext(key.cacheId()), ldr); + txKeys.add(txKey); + } - txKeys.add(key); + txKeysArr = null; } - - txKeysArr = null; + } + catch (Exception e) { + throw new IgniteCheckedException(e); } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionMessageMarshallingTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionMessageMarshallingTest.java new file mode 100644 index 0000000000000..eafd09f2fdbf9 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionMessageMarshallingTest.java @@ -0,0 +1,116 @@ +/* + * 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.transactions; + +import java.util.UUID; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteKernal; +import org.apache.ignite.internal.managers.communication.GridIoPolicy; +import org.apache.ignite.internal.managers.communication.GridMessageListener; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.IgniteCacheProxy; +import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * + */ +public class TxDeadlockDetectionMessageMarshallingTest extends GridCommonAbstractTest { + /** Topic. */ + private static final String TOPIC = "mytopic"; + + /** Client mode. */ + private static boolean clientMode; + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setClientMode(clientMode); + + return cfg; + } + + /** + * @throws Exception If failed. + */ + public void testMessageUnmarshallWithoutCacheContext() throws Exception { + try { + Ignite ignite = startGrid(0); + + CacheConfiguration ccfg = new CacheConfiguration<>(); + + IgniteCache cache = ignite.getOrCreateCache(ccfg); + + clientMode = true; + + Ignite client = startGrid(1); + + final GridCacheSharedContext clientCtx = ((IgniteKernal)client).context().cache().context(); + + final CountDownLatch latch = new CountDownLatch(1); + + final AtomicBoolean res = new AtomicBoolean(); + + clientCtx.gridIO().addMessageListener(TOPIC, new GridMessageListener() { + @Override public void onMessage(UUID nodeId, Object msg) { + if (msg instanceof TxLocksResponse) { + try { + ((TxLocksResponse)msg).finishUnmarshal(clientCtx, clientCtx.deploy().globalLoader()); + + res.set(true); + } + catch (Exception e) { + log.error("Message unmarshal failed", e); + } + finally { + latch.countDown(); + } + } + } + }); + + GridCacheContext cctx = ((IgniteCacheProxy)cache).context(); + + KeyCacheObject key = cctx.toCacheKeyObject(1); + + TxLocksResponse msg = new TxLocksResponse(); + msg.addKey(cctx.txKey(key)); + + msg.prepareMarshal(cctx.shared()); + + ((IgniteKernal)ignite).context().cache().context().gridIO().send( + ((IgniteKernal)client).getLocalNodeId(), TOPIC, msg, GridIoPolicy.PUBLIC_POOL); + + boolean await = latch.await(1, TimeUnit.SECONDS); + + assertTrue(await && res.get()); + } + finally { + stopAllGrids(); + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionUnmasrhalErrorsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionUnmasrhalErrorsTest.java new file mode 100644 index 0000000000000..598725e328f5f --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionUnmasrhalErrorsTest.java @@ -0,0 +1,225 @@ +/* + * 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.transactions; + +import java.util.Collection; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import javax.cache.CacheException; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.IgniteKernal; +import org.apache.ignite.internal.util.typedef.X; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.apache.ignite.transactions.Transaction; +import org.apache.ignite.transactions.TransactionDeadlockException; +import org.apache.ignite.transactions.TransactionTimeoutException; + +import static org.apache.ignite.internal.util.typedef.X.hasCause; +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 TxDeadlockDetectionUnmasrhalErrorsTest extends GridCommonAbstractTest { + /** Nodes count. */ + private static final int NODES_CNT = 2; + + /** Client. */ + private static boolean client; + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setClientMode(client); + + if (isDebug()) { + TcpDiscoverySpi discoSpi = new TcpDiscoverySpi(); + + discoSpi.failureDetectionTimeoutEnabled(false); + + cfg.setDiscoverySpi(discoSpi); + } + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + startGrid(0); + + client = true; + + startGrid(1); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + stopAllGrids(); + } + + /** + * @throws Exception If failed. + */ + public void testDeadlockCacheObjectContext() throws Exception { + IgniteCache cache0 = null; + IgniteCache cache1 = null; + try { + cache0 = getCache(ignite(0), "cache0"); + cache1 = getCache(ignite(0), "cache1"); + + IgniteCache clientCache0 = grid(1).cache("cache0"); + + awaitPartitionMapExchange(); + + final CyclicBarrier barrier = new CyclicBarrier(2); + + final CountDownLatch latch = new CountDownLatch(1); + + final AtomicInteger threadCnt = new AtomicInteger(); + + final AtomicBoolean deadlock = new AtomicBoolean(); + + IgniteInternalFuture fut = GridTestUtils.runMultiThreadedAsync(new Runnable() { + @Override public void run() { + int threadNum = threadCnt.getAndIncrement(); + + Ignite ignite = ignite(0); + + IgniteCache cache1 = ignite.cache("cache" + (threadNum == 0 ? 0 : 1)); + + IgniteCache cache2 = ignite.cache("cache" + (threadNum == 0 ? 1 : 0)); + + try (Transaction tx = ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ, 1000, 0)) { + int key1 = threadNum == 0 ? 0 : 1; + + log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode() + + ", tx=" + tx + ", key=" + key1 + ", cache=" + cache1.getName() + ']'); + + cache1.put(key1, 0); + + barrier.await(); + + int key2 = threadNum == 0 ? 1 : 0; + + log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode() + + ", tx=" + tx + ", key=" + key2 + ", cache=" + cache2.getName() + ']'); + + latch.countDown(); + + cache2.put(key2, 1); + + tx.commit(); + + log.info(">>> Commit done"); + } + catch (Throwable e) { + // At least one stack trace should contain TransactionDeadlockException. + if (hasCause(e, TransactionTimeoutException.class) && + hasCause(e, TransactionDeadlockException.class) + ) { + if (deadlock.compareAndSet(false, true)) + U.error(log, "At least one stack trace should contain " + + TransactionDeadlockException.class.getSimpleName(), e); + } + } + } + }, 2, "tx-thread"); + + latch.await(); + + Ignite client = grid(1); + + try (Transaction tx = client.transactions().txStart(PESSIMISTIC, READ_COMMITTED, 500, 0)) { + clientCache0.put(0, 3); + clientCache0.put(1, 3); + + tx.commit(); + + log.info(">>> Commit done"); + } + catch (CacheException e) { + assertTrue(X.hasCause(e, TransactionTimeoutException.class)); + } + catch (Throwable e) { + log.error("Unexpected exception occurred", e); + + fail(); + } + + fut.get(); + + assertTrue(deadlock.get()); + + for (int i = 0; i < NODES_CNT ; i++) { + Ignite ignite = ignite(i); + + IgniteTxManager txMgr = ((IgniteKernal)ignite).context().cache().context().tm(); + + Collection> futs = txMgr.deadlockDetectionFutures(); + + assertTrue(futs.isEmpty()); + } + + //assertNotNull(grid(1).context().cache().context().cacheContext(cacheId)); + } + finally { + if (cache0 != null) + cache0.destroy(); + + if (cache1 != null) + cache1.destroy(); + } + } + + + + /** + * @param ignite Ignite. + * @param name Name. + */ + private IgniteCache getCache(Ignite ignite, String name) { + CacheConfiguration ccfg = defaultCacheConfiguration(); + + ccfg.setName(name); + ccfg.setCacheMode(CacheMode.PARTITIONED); + ccfg.setBackups(0); + ccfg.setNearConfiguration(null); + + return ignite.getOrCreateCache(ccfg); + } + + +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockNpeClientTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockNpeClientTest.java new file mode 100644 index 0000000000000..f75750916728c --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockNpeClientTest.java @@ -0,0 +1,220 @@ +/* + * 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.transactions; + +import java.util.Collection; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import javax.cache.CacheException; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.IgniteKernal; +import org.apache.ignite.internal.util.typedef.X; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.apache.ignite.transactions.Transaction; +import org.apache.ignite.transactions.TransactionDeadlockException; +import org.apache.ignite.transactions.TransactionTimeoutException; + +import static org.apache.ignite.internal.util.typedef.X.hasCause; +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 TxDeadlockNpeClientTest extends GridCommonAbstractTest { + /** Nodes count. */ + private static final int NODES_CNT = 2; + + /** Client. */ + private static boolean client; + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setClientMode(client); + + if (isDebug()) { + TcpDiscoverySpi discoSpi = new TcpDiscoverySpi(); + + discoSpi.failureDetectionTimeoutEnabled(false); + + cfg.setDiscoverySpi(discoSpi); + } + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + startGrid(0); + + client = true; + + startGrid(1); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + stopAllGrids(); + } + + /** + * @throws Exception If failed. + */ + public void testDeadlockCacheObjectContext() throws Exception { + IgniteCache cache0 = null; + IgniteCache cache1 = null; + + try { + cache0 = getCache(ignite(0), "cache0"); + cache1 = getCache(ignite(0), "cache1"); + + IgniteCache clientCache0 = grid(1).cache("cache0"); + + awaitPartitionMapExchange(); + + final CyclicBarrier barrier = new CyclicBarrier(2); + + final CountDownLatch latch = new CountDownLatch(1); + + final AtomicInteger threadCnt = new AtomicInteger(); + + final AtomicBoolean deadlock = new AtomicBoolean(); + + IgniteInternalFuture fut = GridTestUtils.runMultiThreadedAsync(new Runnable() { + @Override public void run() { + int threadNum = threadCnt.getAndIncrement(); + + Ignite ignite = ignite(0); + + IgniteCache cache1 = ignite.cache("cache" + (threadNum == 0 ? 0 : 1)); + + IgniteCache cache2 = ignite.cache("cache" + (threadNum == 0 ? 1 : 0)); + + try (Transaction tx = ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ, 1000, 0)) { + int key1 = threadNum == 0 ? 0 : 1; + + log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode() + + ", tx=" + tx + ", key=" + key1 + ", cache=" + cache1.getName() + ']'); + + cache1.put(key1, 0); + + barrier.await(); + + int key2 = threadNum == 0 ? 1 : 0; + + log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode() + + ", tx=" + tx + ", key=" + key2 + ", cache=" + cache2.getName() + ']'); + + latch.countDown(); + + cache2.put(key2, 1); + + tx.commit(); + + log.info(">>> Commit done"); + } + catch (Throwable e) { + // At least one stack trace should contain TransactionDeadlockException. + if (hasCause(e, TransactionTimeoutException.class) && + hasCause(e, TransactionDeadlockException.class) + ) { + if (deadlock.compareAndSet(false, true)) + U.error(log, "At least one stack trace should contain " + + TransactionDeadlockException.class.getSimpleName(), e); + } + } + } + }, 2, "tx-thread"); + + latch.await(); + + Ignite client = grid(1); + + try (Transaction tx = client.transactions().txStart(PESSIMISTIC, READ_COMMITTED, 500, 0)) { + clientCache0.put(0, 3); + clientCache0.put(1, 3); + + tx.commit(); + + log.info(">>> Commit done"); + } + catch (CacheException e) { + assertTrue(X.hasCause(e, TransactionTimeoutException.class)); + } + catch (Throwable e) { + fail(); + } + + fut.get(); + + assertTrue(deadlock.get()); + + for (int i = 0; i < NODES_CNT ; i++) { + Ignite ignite = ignite(i); + + IgniteTxManager txMgr = ((IgniteKernal)ignite).context().cache().context().tm(); + + Collection> futs = txMgr.deadlockDetectionFutures(); + + assertTrue(futs.isEmpty()); + } + } + finally { + if (cache0 != null) + cache0.destroy(); + + if (cache1 != null) + cache1.destroy(); + } + } + + /** + * @param ignite Ignite. + * @param name Name. + */ + private IgniteCache getCache(Ignite ignite, String name) { + CacheConfiguration ccfg = defaultCacheConfiguration(); + + ccfg.setName(name); + ccfg.setCacheMode(CacheMode.PARTITIONED); + ccfg.setBackups(0); + ccfg.setNearConfiguration(null); + + return ignite.getOrCreateCache(ccfg); + } + + +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/TxDeadlockDetectionTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/TxDeadlockDetectionTestSuite.java index 5a1b1ad37168a..337d4d46a06d0 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/TxDeadlockDetectionTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/TxDeadlockDetectionTestSuite.java @@ -19,8 +19,10 @@ import junit.framework.TestSuite; import org.apache.ignite.internal.processors.cache.transactions.DepthFirstSearchTest; +import org.apache.ignite.internal.processors.cache.transactions.TxDeadlockDetectionMessageMarshallingTest; import org.apache.ignite.internal.processors.cache.transactions.TxDeadlockDetectionNoHangsTest; import org.apache.ignite.internal.processors.cache.transactions.TxDeadlockDetectionTest; +import org.apache.ignite.internal.processors.cache.transactions.TxDeadlockDetectionUnmasrhalErrorsTest; import org.apache.ignite.internal.processors.cache.transactions.TxOptimisticDeadlockDetectionCrossCacheTest; import org.apache.ignite.internal.processors.cache.transactions.TxOptimisticDeadlockDetectionTest; import org.apache.ignite.internal.processors.cache.transactions.TxPessimisticDeadlockDetectionCrossCacheTest; @@ -44,6 +46,8 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(TxPessimisticDeadlockDetectionCrossCacheTest.class); suite.addTestSuite(TxDeadlockDetectionTest.class); suite.addTestSuite(TxDeadlockDetectionNoHangsTest.class); + suite.addTestSuite(TxDeadlockDetectionUnmasrhalErrorsTest.class); + suite.addTestSuite(TxDeadlockDetectionMessageMarshallingTest.class); return suite; } From a1fec4b0b3ddcc208154114e0bf8f202eb597fc9 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Tue, 25 Apr 2017 09:14:07 +0300 Subject: [PATCH 055/357] Fix index initialization. --- .../ignite/internal/processors/query/h2/IgniteH2Indexing.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 407819746d4b5..85fb4989658d3 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 @@ -2899,7 +2899,7 @@ private SpatialIndex createH2SpatialIndex( private Index createTreeIndex(String idxName, GridH2Table tbl, boolean pk, List columns) { final int segments = tbl.rowDescriptor().configuration().getQueryParallelism(); - return new GridH2TreeIndex(idxName, tbl, pk, columns, segments); + return new GridH2TreeIndex(idxName, tbl, pk, columns, segments > 0 ? segments : 1); } } From d94172e7bb7ff4ebab422997e5c70f8cd03c0ae4 Mon Sep 17 00:00:00 2001 From: agura Date: Wed, 26 Apr 2017 15:16:46 +0300 Subject: [PATCH 056/357] Redundant test removed --- .../transactions/TxDeadlockNpeClientTest.java | 220 ------------------ 1 file changed, 220 deletions(-) delete mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockNpeClientTest.java diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockNpeClientTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockNpeClientTest.java deleted file mode 100644 index f75750916728c..0000000000000 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockNpeClientTest.java +++ /dev/null @@ -1,220 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.internal.processors.cache.transactions; - -import java.util.Collection; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.CyclicBarrier; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; -import javax.cache.CacheException; -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.cache.CacheMode; -import org.apache.ignite.configuration.CacheConfiguration; -import org.apache.ignite.configuration.IgniteConfiguration; -import org.apache.ignite.internal.IgniteInternalFuture; -import org.apache.ignite.internal.IgniteKernal; -import org.apache.ignite.internal.util.typedef.X; -import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; -import org.apache.ignite.testframework.GridTestUtils; -import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; -import org.apache.ignite.transactions.Transaction; -import org.apache.ignite.transactions.TransactionDeadlockException; -import org.apache.ignite.transactions.TransactionTimeoutException; - -import static org.apache.ignite.internal.util.typedef.X.hasCause; -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 TxDeadlockNpeClientTest extends GridCommonAbstractTest { - /** Nodes count. */ - private static final int NODES_CNT = 2; - - /** Client. */ - private static boolean client; - - /** {@inheritDoc} */ - @SuppressWarnings("unchecked") - @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { - IgniteConfiguration cfg = super.getConfiguration(gridName); - - cfg.setClientMode(client); - - if (isDebug()) { - TcpDiscoverySpi discoSpi = new TcpDiscoverySpi(); - - discoSpi.failureDetectionTimeoutEnabled(false); - - cfg.setDiscoverySpi(discoSpi); - } - - return cfg; - } - - /** {@inheritDoc} */ - @Override protected void beforeTestsStarted() throws Exception { - super.beforeTestsStarted(); - - startGrid(0); - - client = true; - - startGrid(1); - } - - /** {@inheritDoc} */ - @Override protected void afterTestsStopped() throws Exception { - super.afterTestsStopped(); - - stopAllGrids(); - } - - /** - * @throws Exception If failed. - */ - public void testDeadlockCacheObjectContext() throws Exception { - IgniteCache cache0 = null; - IgniteCache cache1 = null; - - try { - cache0 = getCache(ignite(0), "cache0"); - cache1 = getCache(ignite(0), "cache1"); - - IgniteCache clientCache0 = grid(1).cache("cache0"); - - awaitPartitionMapExchange(); - - final CyclicBarrier barrier = new CyclicBarrier(2); - - final CountDownLatch latch = new CountDownLatch(1); - - final AtomicInteger threadCnt = new AtomicInteger(); - - final AtomicBoolean deadlock = new AtomicBoolean(); - - IgniteInternalFuture fut = GridTestUtils.runMultiThreadedAsync(new Runnable() { - @Override public void run() { - int threadNum = threadCnt.getAndIncrement(); - - Ignite ignite = ignite(0); - - IgniteCache cache1 = ignite.cache("cache" + (threadNum == 0 ? 0 : 1)); - - IgniteCache cache2 = ignite.cache("cache" + (threadNum == 0 ? 1 : 0)); - - try (Transaction tx = ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ, 1000, 0)) { - int key1 = threadNum == 0 ? 0 : 1; - - log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode() + - ", tx=" + tx + ", key=" + key1 + ", cache=" + cache1.getName() + ']'); - - cache1.put(key1, 0); - - barrier.await(); - - int key2 = threadNum == 0 ? 1 : 0; - - log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode() + - ", tx=" + tx + ", key=" + key2 + ", cache=" + cache2.getName() + ']'); - - latch.countDown(); - - cache2.put(key2, 1); - - tx.commit(); - - log.info(">>> Commit done"); - } - catch (Throwable e) { - // At least one stack trace should contain TransactionDeadlockException. - if (hasCause(e, TransactionTimeoutException.class) && - hasCause(e, TransactionDeadlockException.class) - ) { - if (deadlock.compareAndSet(false, true)) - U.error(log, "At least one stack trace should contain " + - TransactionDeadlockException.class.getSimpleName(), e); - } - } - } - }, 2, "tx-thread"); - - latch.await(); - - Ignite client = grid(1); - - try (Transaction tx = client.transactions().txStart(PESSIMISTIC, READ_COMMITTED, 500, 0)) { - clientCache0.put(0, 3); - clientCache0.put(1, 3); - - tx.commit(); - - log.info(">>> Commit done"); - } - catch (CacheException e) { - assertTrue(X.hasCause(e, TransactionTimeoutException.class)); - } - catch (Throwable e) { - fail(); - } - - fut.get(); - - assertTrue(deadlock.get()); - - for (int i = 0; i < NODES_CNT ; i++) { - Ignite ignite = ignite(i); - - IgniteTxManager txMgr = ((IgniteKernal)ignite).context().cache().context().tm(); - - Collection> futs = txMgr.deadlockDetectionFutures(); - - assertTrue(futs.isEmpty()); - } - } - finally { - if (cache0 != null) - cache0.destroy(); - - if (cache1 != null) - cache1.destroy(); - } - } - - /** - * @param ignite Ignite. - * @param name Name. - */ - private IgniteCache getCache(Ignite ignite, String name) { - CacheConfiguration ccfg = defaultCacheConfiguration(); - - ccfg.setName(name); - ccfg.setCacheMode(CacheMode.PARTITIONED); - ccfg.setBackups(0); - ccfg.setNearConfiguration(null); - - return ignite.getOrCreateCache(ccfg); - } - - -} From c64dc2426a70896116dbdffbf90faea5e52b991c Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Thu, 27 Apr 2017 16:35:25 +0300 Subject: [PATCH 057/357] Revert IGNITE-4863 partially. Warn user if default log level was changed. As default log level value (DEBUG) can cause performance issues. --- .../ignite/logger/log4j/Log4JLogger.java | 51 +++++++++---------- .../log4j/GridLog4jInitializationTest.java | 4 +- 2 files changed, 27 insertions(+), 28 deletions(-) diff --git a/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4JLogger.java b/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4JLogger.java index f6ed83025af55..a49e24716c399 100644 --- a/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4JLogger.java +++ b/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4JLogger.java @@ -56,20 +56,20 @@ *

    * Here is a typical example of configuring log4j logger in Ignite configuration file: *

    - *      <property name="gridLogger">
    - *          <bean class="org.apache.ignite.logger.log4j.Log4JLogger">
    - *              <constructor-arg type="java.lang.String" value="config/ignite-log4j.xml"/>
    - *          </bean>
    - *      </property>
    + * <property name="gridLogger">
    + * <bean class="org.apache.ignite.logger.log4j.Log4JLogger">
    + * <constructor-arg type="java.lang.String" value="config/ignite-log4j.xml"/>
    + * </bean>
    + * </property>
      * 
    * and from your code: *
    - *      IgniteConfiguration cfg = new IgniteConfiguration();
    - *      ...
    - *      URL xml = U.resolveIgniteUrl("config/custom-log4j.xml");
    - *      IgniteLogger log = new Log4JLogger(xml);
    - *      ...
    - *      cfg.setGridLogger(log);
    + * IgniteConfiguration cfg = new IgniteConfiguration();
    + * ...
    + * URL xml = U.resolveIgniteUrl("config/custom-log4j.xml");
    + * IgniteLogger log = new Log4JLogger(xml);
    + * ...
    + * cfg.setGridLogger(log);
      * 
    * * Please take a look at Apache Log4j 1.2 @@ -352,21 +352,20 @@ private void addConsoleAppenderIfNeeded(@Nullable Level logLevel, if (errAppender.getThreshold() == Level.ERROR) errAppender.setThreshold(Level.WARN); } - else { - // No error console appender => create console appender with. - final AppenderSkeleton consoleAppender = createConsoleAppender(Level.OFF); - - consoleAppender.setThreshold(Level.INFO); - - rootCategory.addAppender(consoleAppender); - } + else + // No error console appender => create console appender with no level limit. + rootCategory.addAppender(createConsoleAppender(Level.OFF)); - // Won't raise LogLevel if there is other loggers configured. As LogLevel can be inherited. - if (logLevel != null && !logLevel.isGreaterOrEqual(impl.getEffectiveLevel())) { + if (logLevel != null) { impl.setLevel(logLevel); - impl.warn("RootLogger logging level has been dropped by Apache Ignite.\n"+ - "Set lower log level or configure ConsoleAppender manually or disable ConsoleAppender automatic creation."); + // Warn if LogLevel is changed and there is other logger configured that can inherits LogLevel. + if (!impl.getLevel().equals(logLevel) && hasOtherLoggers()) + impl.warn("RootLogger logging level has been changed by Apache Ignite to " + + logLevel.toString() + " level.\n" + + "Please, configure ConsoleAppender manually or disable ConsoleAppender automatic creation."); + else + impl.info("Apache Ignite set default logging level to " + logLevel.toString()); } } else if (!isConfigured() && !hasOtherLoggers()) { @@ -403,11 +402,11 @@ private boolean hasOtherLoggers() { * @param maxLevel Max logging level. * @return New console appender. */ - private AppenderSkeleton createConsoleAppender(Level maxLevel) { + private Appender createConsoleAppender(Level maxLevel) { String fmt = "[%d{ABSOLUTE}][%-5p][%t][%c{1}] %m%n"; // Configure output that should go to System.out - AppenderSkeleton app = new ConsoleAppender(new PatternLayout(fmt), ConsoleAppender.SYSTEM_OUT); + Appender app = new ConsoleAppender(new PatternLayout(fmt), ConsoleAppender.SYSTEM_OUT); LevelRangeFilter lvlFilter = new LevelRangeFilter(); @@ -569,7 +568,7 @@ public static Collection logFiles() { /** * For test purposes only. */ - static void reset(){ + static void reset() { inited = false; quiet0 = false; diff --git a/modules/log4j/src/test/java/org/apache/ignite/logger/log4j/GridLog4jInitializationTest.java b/modules/log4j/src/test/java/org/apache/ignite/logger/log4j/GridLog4jInitializationTest.java index 2a98490744db5..1f35c8ec55090 100644 --- a/modules/log4j/src/test/java/org/apache/ignite/logger/log4j/GridLog4jInitializationTest.java +++ b/modules/log4j/src/test/java/org/apache/ignite/logger/log4j/GridLog4jInitializationTest.java @@ -133,7 +133,7 @@ public void testAutoAddConsoleAppender2() { if (VERBOSE) printLoggerResults(log); - assertEquals(Level.DEBUG, logger.getEffectiveLevel()); // LogLevel should not change. + assertEquals(Level.INFO, logger.getEffectiveLevel()); } /** */ @@ -171,7 +171,7 @@ public void testAutoAddConsoleAppenderWithOtherLoggerConfigured() { if (VERBOSE) printLoggerResults(log); - assertEquals(Level.DEBUG, logger.getEffectiveLevel()); // LogLevel should not be raised. + assertEquals(Level.INFO, logger.getEffectiveLevel()); } /** */ From f9ecacc625b458539775e6550bd9b7613ed38f21 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Fri, 28 Apr 2017 11:46:23 +0300 Subject: [PATCH 058/357] 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 059/357] 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 060/357] 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 061/357] 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 062/357] 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 063/357] 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 064/357] 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 7d8d672295e6554a82f63a16b73c330ca6050618 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Fri, 5 May 2017 14:29:59 +0300 Subject: [PATCH 065/357] IGNITE-4992: Fix Segmented SQL Index self tests. - Fixes #1801. Signed-off-by: Sergi Vladykin (cherry picked from commit 0c0cf2c00355ae2aec340473ed19dc5f0e0ddef0) --- ...iteSqlSegmentedIndexMultiNodeSelfTest.java | 28 ++++ .../IgniteSqlSegmentedIndexSelfTest.java | 148 ++++++++++-------- .../IgniteCacheQuerySelfTestSuite.java | 2 + 3 files changed, 113 insertions(+), 65 deletions(-) create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSegmentedIndexMultiNodeSelfTest.java diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSegmentedIndexMultiNodeSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSegmentedIndexMultiNodeSelfTest.java new file mode 100644 index 0000000000000..549ea5e043048 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSegmentedIndexMultiNodeSelfTest.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.query; + +/** + * Tests for correct distributed queries with index consisted of many segments running on multi-node grid. + */ +public class IgniteSqlSegmentedIndexMultiNodeSelfTest extends IgniteSqlSegmentedIndexSelfTest { + /** {@inheritDoc} */ + @Override protected int nodesCount() { + return 4; + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSegmentedIndexSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSegmentedIndexSelfTest.java index 800138c1f32ac..f236ebf4cf5c6 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSegmentedIndexSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSegmentedIndexSelfTest.java @@ -18,11 +18,13 @@ package org.apache.ignite.internal.processors.query; import java.io.Serializable; +import java.util.Arrays; import java.util.HashSet; import java.util.List; import java.util.Random; import java.util.Set; import javax.cache.Cache; +import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheKeyConfiguration; @@ -46,8 +48,22 @@ public class IgniteSqlSegmentedIndexSelfTest extends GridCommonAbstractTest { private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); /** */ - private static int QRY_PARALLELISM_LVL = 97; + private static final String ORG_CACHE_NAME = "org"; + + /** */ + private static final String PERSON_CAHE_NAME = "pers"; + + /** */ + private static final int ORG_CACHE_SIZE = 500; + + /** */ + private static final int PERSON_CACHE_SIZE = 1000; + + /** */ + private static final int ORPHAN_ROWS = 10; + /** */ + private static int QRY_PARALLELISM_LVL = 97; /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { @@ -70,9 +86,26 @@ public class IgniteSqlSegmentedIndexSelfTest extends GridCommonAbstractTest { return cfg; } + /** @return number of nodes to be prestarted. */ + protected int nodesCount() { + return 1; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + startGridsMultiThreaded(nodesCount(), false); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + } + /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { - stopAllGrids(true); + super.afterTest(); + + grid(0).destroyCaches(Arrays.asList(PERSON_CAHE_NAME, ORG_CACHE_NAME)); } /** @@ -91,14 +124,11 @@ protected CacheConfiguration cacheConfig(String name, boolean parti } /** - * Run tests on single-node grid * @throws Exception If failed. */ - public void testSingleNodeIndexSegmentation() throws Exception { - startGridsMultiThreaded(1, true); - - ignite(0).createCache(cacheConfig("pers", true, Integer.class, Person.class)); - ignite(0).createCache(cacheConfig("org", true, Integer.class, Organization.class)); + public void testSegmentedIndex() throws Exception { + ignite(0).createCache(cacheConfig(PERSON_CAHE_NAME, true, Integer.class, Person.class)); + ignite(0).createCache(cacheConfig(ORG_CACHE_NAME, true, Integer.class, Organization.class)); fillCache(); @@ -109,11 +139,10 @@ public void testSingleNodeIndexSegmentation() throws Exception { /** * Run tests on single-node grid + * * @throws Exception If failed. */ - public void testSingleNodeIndexSegmentationWithSwapEnabled() throws Exception { - startGridsMultiThreaded(1, true); - + public void testSegmentedIndexWithEvictionPolicy() throws Exception { final IgniteCache cache = ignite(0).createCache(cacheConfig("org", true, Integer.class, Organization.class) .setOffHeapMaxMemory(-1) .setSwapEnabled(true) @@ -131,30 +160,12 @@ public void testSingleNodeIndexSegmentationWithSwapEnabled() throws Exception { /** * Run tests on multi-node grid + * * @throws Exception If failed. */ - public void testMultiNodeIndexSegmentation() throws Exception { - startGridsMultiThreaded(4, true); - - ignite(0).createCache(cacheConfig("pers", true, Integer.class, Person.class)); - ignite(0).createCache(cacheConfig("org", true, Integer.class, Organization.class)); - - fillCache(); - - checkDistributedQueryWithSegmentedIndex(); - - checkLocalQueryWithSegmentedIndex(); - } - - /** - * Run tests on multi-node grid - * @throws Exception If failed. - */ - public void testMultiNodeSegmentedPartitionedWithReplicated() throws Exception { - startGridsMultiThreaded(4, true); - - ignite(0).createCache(cacheConfig("pers", true, Integer.class, Person.class)); - ignite(0).createCache(cacheConfig("org", false, Integer.class, Organization.class)); + public void testSegmentedPartitionedWithReplicated() throws Exception { + ignite(0).createCache(cacheConfig(PERSON_CAHE_NAME, true, Integer.class, Person.class)); + ignite(0).createCache(cacheConfig(ORG_CACHE_NAME, false, Integer.class, Organization.class)); fillCache(); @@ -165,71 +176,78 @@ public void testMultiNodeSegmentedPartitionedWithReplicated() throws Exception { /** * Check distributed joins. + * * @throws Exception If failed. */ public void checkDistributedQueryWithSegmentedIndex() throws Exception { - IgniteCache c1 = ignite(0).cache("pers"); + for (int i = 0; i < nodesCount(); i++) { + IgniteCache c1 = ignite(i).cache(PERSON_CAHE_NAME); - int expectedPersons = 0; + int expectedPersons = 0; - for (Cache.Entry e : c1) { - final Integer orgId = e.getValue().orgId; + for (Cache.Entry e : c1) { + final Integer orgId = e.getValue().orgId; - if (10 <= orgId && orgId < 500) - expectedPersons++; - } + // We have as orphan ORG rows as orphan PERSON rows. + if (ORPHAN_ROWS <= orgId && orgId < 500) + expectedPersons++; + } - String select0 = "select o.name n1, p.name n2 from \"pers\".Person p, \"org\".Organization o where p.orgId = o._key"; + String select0 = "select o.name n1, p.name n2 from \"pers\".Person p, \"org\".Organization o where p.orgId = o._key"; - List> result = c1.query(new SqlFieldsQuery(select0).setDistributedJoins(true)).getAll(); + List> result = c1.query(new SqlFieldsQuery(select0).setDistributedJoins(true)).getAll(); - assertEquals(expectedPersons, result.size()); + assertEquals(expectedPersons, result.size()); + } } /** * Test local query. + * * @throws Exception If failed. */ public void checkLocalQueryWithSegmentedIndex() throws Exception { - IgniteCache c1 = ignite(0).cache("pers"); - IgniteCache c2 = ignite(0).cache("org"); + for (int i = 0; i < nodesCount(); i++) { + final Ignite node = ignite(i); - Set localOrgIds = new HashSet<>(); + IgniteCache c1 = node.cache(PERSON_CAHE_NAME); + IgniteCache c2 = node.cache(ORG_CACHE_NAME); - for (Cache.Entry e : c2.localEntries()) - localOrgIds.add(e.getKey()); + Set localOrgIds = new HashSet<>(); - int expectedPersons = 0; + for (Cache.Entry e : c2.localEntries()) + localOrgIds.add(e.getKey()); - for (Cache.Entry e : c1.localEntries()) { - final Integer orgId = e.getValue().orgId; + int expectedPersons = 0; - if (localOrgIds.contains(orgId)) - expectedPersons++; - } + for (Cache.Entry e : c1.localEntries()) { + final Integer orgId = e.getValue().orgId; + + if (localOrgIds.contains(orgId)) + expectedPersons++; + } - String select0 = "select o.name n1, p.name n2 from \"pers\".Person p, \"org\".Organization o where p.orgId = o._key"; + String select0 = "select o.name n1, p.name n2 from \"pers\".Person p, \"org\".Organization o where p.orgId = o._key"; - List> result = c1.query(new SqlFieldsQuery(select0).setLocal(true)).getAll(); + List> result = c1.query(new SqlFieldsQuery(select0).setLocal(true)).getAll(); - assertEquals(expectedPersons, result.size()); + assertEquals(expectedPersons, result.size()); + } } /** */ private void fillCache() { - IgniteCache c1 = ignite(0).cache("pers"); - - IgniteCache c2 = ignite(0).cache("org"); - - final int orgCount = 500; + IgniteCache c1 = ignite(0).cache(PERSON_CAHE_NAME); + IgniteCache c2 = ignite(0).cache(ORG_CACHE_NAME); - for (int i = 0; i < orgCount; i++) + for (int i = 0; i < ORG_CACHE_SIZE; i++) c2.put(i, new Organization("org-" + i)); final Random random = new Random(); - for (int i = 0; i < 1000; i++) { - int orgID = 10 + random.nextInt(orgCount + 10); + for (int i = 0; i < PERSON_CACHE_SIZE; i++) { + // We have as orphan ORG rows as orphan PERSON rows. + int orgID = ORPHAN_ROWS + random.nextInt(ORG_CACHE_SIZE + ORPHAN_ROWS); c1.put(i, new Person(orgID, "pers-" + i)); } diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java index 0c74f127fbff0..4fb729d56c1ed 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java +++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java @@ -98,6 +98,7 @@ import org.apache.ignite.internal.processors.cache.query.IndexingSpiQueryTxSelfTest; import org.apache.ignite.internal.processors.query.IgniteSqlEntryCacheModeAgnosticTest; import org.apache.ignite.internal.processors.query.IgniteSqlSchemaIndexingTest; +import org.apache.ignite.internal.processors.query.IgniteSqlSegmentedIndexMultiNodeSelfTest; import org.apache.ignite.internal.processors.query.IgniteSqlSegmentedIndexSelfTest; import org.apache.ignite.internal.processors.query.IgniteSqlSplitterSelfTest; import org.apache.ignite.internal.processors.query.h2.GridH2IndexRebuildTest; @@ -137,6 +138,7 @@ public static TestSuite suite() throws Exception { // Queries tests. suite.addTestSuite(IgniteSqlSplitterSelfTest.class); suite.addTestSuite(IgniteSqlSegmentedIndexSelfTest.class); + suite.addTestSuite(IgniteSqlSegmentedIndexMultiNodeSelfTest.class); suite.addTestSuite(IgniteSqlSchemaIndexingTest.class); suite.addTestSuite(GridCacheQueryIndexDisabledSelfTest.class); suite.addTestSuite(IgniteCacheQueryLoadSelfTest.class); From 61c118f7b8097c554d7e2e4a09133fe3978d73d7 Mon Sep 17 00:00:00 2001 From: Alexey Goncharuk Date: Wed, 19 Apr 2017 13:55:02 +0300 Subject: [PATCH 066/357] IGNITE-4993 - Fixing distributed joins on segmented index. (cherry picked from commit 800b8bd) --- .../query/h2/opt/GridH2IndexBase.java | 31 ++++++++++--------- .../IgniteSqlSegmentedIndexSelfTest.java | 2 +- 2 files changed, 18 insertions(+), 15 deletions(-) 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 fc5eb4b35c911..efa95a8979b34 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 @@ -406,9 +406,7 @@ protected static IndexingQueryFilter threadLocalFilter() { GridCacheContext cctx = getTable().rowDescriptor().context(); - boolean isLocal = qctx.distributedJoinMode() == LOCAL_ONLY; - - return new DistributedLookupBatch(cctx, ucast, affColId, isLocal); + return new DistributedLookupBatch(cctx, ucast, affColId); } /** @@ -1078,9 +1076,6 @@ private class DistributedLookupBatch implements IndexLookupBatch { /** */ final int affColId; - /** */ - private final boolean localQuery; - /** */ GridH2QueryContext qctx; @@ -1106,13 +1101,11 @@ private class DistributedLookupBatch implements IndexLookupBatch { * @param cctx Cache Cache context. * @param ucast Unicast or broadcast query. * @param affColId Affinity column ID. - * @param localQuery Local query flag. */ - DistributedLookupBatch(GridCacheContext cctx, boolean ucast, int affColId, boolean localQuery) { + DistributedLookupBatch(GridCacheContext cctx, boolean ucast, int affColId) { this.cctx = cctx; this.ucast = ucast; this.affColId = affColId; - this.localQuery = localQuery; } /** @@ -1184,25 +1177,26 @@ private Object getAffinityKey(SearchRow firstRow, SearchRow lastRow) { Object affKey = affColId == -1 ? null : getAffinityKey(firstRow, lastRow); + boolean locQry = localQuery(); + List segmentKeys; - Future fut; if (affKey != null) { // Affinity key is provided. if (affKey == EXPLICIT_NULL) // Affinity key is explicit null, we will not find anything. return false; - segmentKeys = F.asList(rangeSegment(cctx, qctx, affKey, localQuery)); + segmentKeys = F.asList(rangeSegment(cctx, qctx, affKey, locQry)); } else { // Affinity key is not provided or is not the same in upper and lower bounds, we have to broadcast. if (broadcastSegments == null) - broadcastSegments = broadcastSegments(qctx, cctx, localQuery); + broadcastSegments = broadcastSegments(qctx, cctx, locQry); segmentKeys = broadcastSegments; } - if (localQuery && segmentKeys.isEmpty()) + if (locQry && segmentKeys.isEmpty()) return false; // Nothing to do assert !F.isEmpty(segmentKeys) : segmentKeys; @@ -1243,7 +1237,7 @@ private Object getAffinityKey(SearchRow firstRow, SearchRow lastRow) { batchFull = true; } - fut = new DoneFuture<>(segmentKeys.size() == 1 ? + Future fut = new DoneFuture<>(segmentKeys.size() == 1 ? new UnicastCursor(rangeId, segmentKeys, rangeStreams) : new BroadcastCursor(rangeId, segmentKeys, rangeStreams)); @@ -1257,6 +1251,15 @@ private Object getAffinityKey(SearchRow firstRow, SearchRow lastRow) { return batchFull; } + /** + * @return {@code True} if local query execution is enforced. + */ + private boolean localQuery() { + assert qctx != null : "Missing query context: " + this; + + return qctx.distributedJoinMode() == LOCAL_ONLY; + } + /** * */ diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSegmentedIndexSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSegmentedIndexSelfTest.java index f236ebf4cf5c6..2c21d1a9b8e41 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSegmentedIndexSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSegmentedIndexSelfTest.java @@ -93,7 +93,7 @@ protected int nodesCount() { /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { - startGridsMultiThreaded(nodesCount(), false); + startGrids(nodesCount()); } /** {@inheritDoc} */ From 1a4c1b66f7122b4fadc57f6047caf80dfe75e2f4 Mon Sep 17 00:00:00 2001 From: AMRepo Date: Sat, 6 May 2017 01:21:46 +0300 Subject: [PATCH 067/357] Fix segmented index snapshot usage. --- .../query/h2/opt/GridH2IndexBase.java | 6 +++- .../query/h2/opt/GridH2TreeIndex.java | 20 +++++++------ .../IgniteSqlSegmentedIndexSelfTest.java | 28 +++++++++++++++++++ 3 files changed, 45 insertions(+), 9 deletions(-) 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 efa95a8979b34..8eb082a112b9a 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 @@ -480,7 +480,11 @@ private void onIndexRangeRequest(final ClusterNode node, final GridH2IndexRangeR if (msg.bounds() != null) { // This is the first request containing all the search rows. - ConcurrentNavigableMap snapshot0 = qctx.getSnapshot(idxId); + Object[] snapshotObj = qctx.getSnapshot(idxId); + + // We should get correct segment here. + ConcurrentNavigableMap snapshot0 = snapshotObj == null ? null : + (ConcurrentNavigableMap)snapshotObj[msg.segment()]; assert !msg.bounds().isEmpty() : "empty bounds"; diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java index 663d86386b8de..2baeca9be4d7f 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java @@ -166,13 +166,17 @@ public GridH2TreeIndex(String name, GridH2Table tbl, boolean pk, List tree = segments[i]; - ConcurrentNavigableMap tree = segments[seg]; + snapshot[i] = tree instanceof SnapTreeMap ? + ((SnapTreeMap)tree).clone() : + ((GridOffHeapSnapTreeMap)tree).clone(); + } - return tree instanceof SnapTreeMap ? - ((SnapTreeMap)tree).clone() : - ((GridOffHeapSnapTreeMap)tree).clone(); + return snapshot; } /** {@inheritDoc} */ @@ -180,12 +184,12 @@ protected ConcurrentNavigableMap treeForRead(in if (!snapshotEnabled) return segments[seg]; - ConcurrentNavigableMap res = threadLocalSnapshot(); + final ConcurrentNavigableMap[] snapshot = threadLocalSnapshot(); - if (res == null) + if (snapshot == null) return segments[seg]; - return res; + return snapshot[seg]; } /** {@inheritDoc} */ diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSegmentedIndexSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSegmentedIndexSelfTest.java index 2c21d1a9b8e41..7c4fe9c790e57 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSegmentedIndexSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSegmentedIndexSelfTest.java @@ -28,6 +28,7 @@ import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheKeyConfiguration; +import org.apache.ignite.cache.CacheMemoryMode; import org.apache.ignite.cache.CacheMode; import org.apache.ignite.cache.eviction.fifo.FifoEvictionPolicy; import org.apache.ignite.cache.query.SqlFieldsQuery; @@ -137,6 +138,33 @@ public void testSegmentedIndex() throws Exception { checkLocalQueryWithSegmentedIndex(); } + /** + * Check correct index snapshots with segmented indices. + * @throws Exception If failed. + */ + public void testSegmentedIndexReproducableResults() throws Exception { + ignite(0).createCache(cacheConfig(ORG_CACHE_NAME, true, Integer.class, Organization.class) + .setOffHeapMaxMemory(-1) // Make index snapshot to be used. + .setMemoryMode(CacheMemoryMode.OFFHEAP_TIERED)); + + IgniteCache cache = ignite(0).cache(ORG_CACHE_NAME); + + // Unequal entries distribution among partitions. + int expectedSize = nodesCount() * QRY_PARALLELISM_LVL * 3 / 2; + + for (int i = 0; i < expectedSize; i++) + cache.put(i, new Organization("org-" + i)); + + String select0 = "select * from \"org\".Organization o"; + + // Check for stable results. + for(int i = 0; i < 10; i++) { + List> result = cache.query(new SqlFieldsQuery(select0)).getAll(); + + assertEquals(expectedSize, result.size()); + } + } + /** * Run tests on single-node grid * From 2f832a3621432ba3947c8d519ff124b29eb8d9a0 Mon Sep 17 00:00:00 2001 From: Sergi Vladykin Date: Tue, 9 May 2017 06:11:34 +0300 Subject: [PATCH 068/357] GG-12182 Incorrect splitting of aggregate subqueries in SELECT cheerry picked from commit d10091d6f2 --- .../query/h2/sql/GridSqlQuerySplitter.java | 7 ++++- .../query/IgniteSqlSplitterSelfTest.java | 30 +++++++++++++++++++ 2 files changed, 36 insertions(+), 1 deletion(-) diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java index aec0b3664f47b..5bd7f2b9772b6 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java @@ -1751,13 +1751,18 @@ private static void set(List list, int idx, Z item) { } /** - * @param el Expression. + * @param el Expression part in SELECT clause. * @return {@code true} If expression contains aggregates. */ private static boolean hasAggregates(GridSqlAst el) { if (el instanceof GridSqlAggregateFunction) return true; + // If in SELECT clause we have a subquery expression with aggregate, + // we should not split it. Run the whole subquery on MAP stage. + if (el instanceof GridSqlQuery) + return false; + for (int i = 0; i < el.size(); i++) { if (hasAggregates(el.child(i))) return true; diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java index 1aa7e7215b972..8daf97d50b69a 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java @@ -36,6 +36,7 @@ import org.apache.ignite.cache.CacheMode; import org.apache.ignite.cache.CachePeekMode; import org.apache.ignite.cache.affinity.Affinity; +import org.apache.ignite.cache.affinity.AffinityKey; import org.apache.ignite.cache.affinity.AffinityKeyMapped; import org.apache.ignite.cache.query.QueryCursor; import org.apache.ignite.cache.query.SqlFieldsQuery; @@ -182,6 +183,35 @@ public void testReplicatedOnlyTables() { } } + /** + */ + public void testSubQueryWithAggregate() { + CacheConfiguration ccfg1 = cacheConfig("pers", true, + AffinityKey.class, Person2.class); + + IgniteCache, Person2> c1 = ignite(0).getOrCreateCache(ccfg1); + + try { + int orgId = 100500; + + c1.put(new AffinityKey<>(1, orgId), new Person2(orgId, "Vasya")); + c1.put(new AffinityKey<>(2, orgId), new Person2(orgId, "Another Vasya")); + + List> rs = c1.query(new SqlFieldsQuery("select name, " + + "(select count(1) from Person2 q where q.orgId = p.orgId) " + + "from Person2 p order by name desc")).getAll(); + + assertEquals(2, rs.size()); + assertEquals("Vasya", rs.get(0).get(0)); + assertEquals(2L, rs.get(0).get(1)); + assertEquals("Another Vasya", rs.get(1).get(0)); + assertEquals(2L, rs.get(1).get(1)); + } + finally { + c1.destroy(); + } + } + /** * @throws Exception If failed. */ From d3b2e99a1e2c580f156d89185414abcd0ebcd729 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Fri, 12 May 2017 19:28:36 +0300 Subject: [PATCH 069/357] Fixed segmented indices snapshots. (cherry picked from commit b807e4b) --- .../query/h2/opt/GridH2IndexBase.java | 6 +- .../processors/query/h2/opt/GridH2Table.java | 80 ++++++++++++------- .../query/h2/opt/GridH2TreeIndex.java | 16 ++-- .../h2/twostep/GridMapQueryExecutor.java | 2 +- .../IgniteSqlSegmentedIndexSelfTest.java | 2 +- 5 files changed, 59 insertions(+), 47 deletions(-) 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 8eb082a112b9a..efa95a8979b34 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 @@ -480,11 +480,7 @@ private void onIndexRangeRequest(final ClusterNode node, final GridH2IndexRangeR if (msg.bounds() != null) { // This is the first request containing all the search rows. - Object[] snapshotObj = qctx.getSnapshot(idxId); - - // We should get correct segment here. - ConcurrentNavigableMap snapshot0 = snapshotObj == null ? null : - (ConcurrentNavigableMap)snapshotObj[msg.segment()]; + ConcurrentNavigableMap snapshot0 = qctx.getSnapshot(idxId); assert !msg.bounds().isEmpty() : "empty bounds"; diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java index 4d5ea4bfec9e2..6b087c2cdf91e 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java @@ -25,6 +25,7 @@ import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.atomic.AtomicReferenceArray; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -74,10 +75,10 @@ public class GridH2Table extends TableBase { private boolean destroyed; /** */ - private final Set sessions = Collections.newSetFromMap(new ConcurrentHashMap8()); + private final Set sessions = Collections.newSetFromMap(new ConcurrentHashMap8()); /** */ - private final AtomicReference actualSnapshot = new AtomicReference<>(); + private final AtomicReferenceArray actualSnapshot; /** */ private IndexColumn affKeyCol; @@ -141,6 +142,11 @@ public GridH2Table(CreateTableData createTblData, @Nullable GridH2RowDescriptor snapshotEnabled = desc == null || desc.snapshotableIndex(); + final int segments = desc != null ? desc.configuration().getQueryParallelism() : + index(1).segmentsCount(); // Get index segments count from PK index. Null desc can be passed from tests. + + actualSnapshot = snapshotEnabled ? new AtomicReferenceArray(Math.max(segments, 1)) : null; + lock = snapshotEnabled ? new ReentrantReadWriteLock() : null; } @@ -259,11 +265,22 @@ private boolean onSwapUnswap(CacheObject key, @Nullable CacheObject val) throws } if (snapshotInLock()) - snapshotIndexes(null); + snapshotIndexes(null, threadLocalSegmentId()); return false; } + /** + * @return segmentId for current thread. + */ + private int threadLocalSegmentId() { + final GridH2QueryContext qctx = GridH2QueryContext.get(); + + assert qctx != null; + + return qctx.segment(); + } + /** * @return {@code True} If we must snapshot and release index snapshots in {@link #lock(Session, boolean, boolean)} * and {@link #unlock(Session)} methods. @@ -281,22 +298,22 @@ private boolean snapshotInLock() { /** * @param qctx Query context. */ - public void snapshotIndexes(GridH2QueryContext qctx) { + public void snapshotIndexes(GridH2QueryContext qctx, int segment) { if (!snapshotEnabled) return; - Object[] snapshots; + Object[] segmentSnapshot; Lock l; // Try to reuse existing snapshots outside of the lock. - for (long waitTime = 200;; waitTime *= 2) { // Increase wait time to avoid starvation. - snapshots = actualSnapshot.get(); + for (long waitTime = 200; ; waitTime *= 2) { // Increase wait time to avoid starvation. + segmentSnapshot = actualSnapshot.get(segment); - if (snapshots != null) { // Reuse existing snapshot without locking. - snapshots = doSnapshotIndexes(snapshots, qctx); + if (segmentSnapshot != null) { // Reuse existing snapshot without locking. + segmentSnapshot = doSnapshotIndexes(segmentSnapshot, qctx); - if (snapshots != null) + if (segmentSnapshot != null) return; // Reused successfully. } @@ -308,17 +325,17 @@ public void snapshotIndexes(GridH2QueryContext qctx) { try { // Try again inside of the lock. - snapshots = actualSnapshot.get(); + segmentSnapshot = actualSnapshot.get(segment); - if (snapshots != null) // Try reusing. - snapshots = doSnapshotIndexes(snapshots, qctx); + if (segmentSnapshot != null) // Try reusing. + segmentSnapshot = doSnapshotIndexes(segmentSnapshot, qctx); - if (snapshots == null) { // Reuse failed, produce new snapshots. - snapshots = doSnapshotIndexes(null, qctx); + if (segmentSnapshot == null) { // Reuse failed, produce new snapshots. + segmentSnapshot = doSnapshotIndexes(null, qctx); - assert snapshots != null; + assert segmentSnapshot != null; - actualSnapshot.set(snapshots); + actualSnapshot.set(segment, segmentSnapshot); } } finally { @@ -373,19 +390,20 @@ public Lock lock(boolean exclusive, long waitMillis) { * Must be called inside of write lock because when using multiple indexes we have to ensure that all of them have * the same contents at snapshot taking time. * + * @param segmentSnapshot snapshot to be reused. * @param qctx Query context. * @return New indexes data snapshot. */ @SuppressWarnings("unchecked") - private Object[] doSnapshotIndexes(Object[] snapshots, GridH2QueryContext qctx) { + private Object[] doSnapshotIndexes(Object[] segmentSnapshot, GridH2QueryContext qctx) { assert snapshotEnabled; - if (snapshots == null) // Nothing to reuse, create new snapshots. - snapshots = new Object[idxs.size() - 1]; + if (segmentSnapshot == null) // Nothing to reuse, create new snapshots. + segmentSnapshot = new Object[idxs.size() - 1]; // Take snapshots on all except first which is scan. for (int i = 1, len = idxs.size(); i < len; i++) { - Object s = snapshots[i - 1]; + Object s = segmentSnapshot[i - 1]; boolean reuseExisting = s != null; @@ -400,15 +418,15 @@ private Object[] doSnapshotIndexes(Object[] snapshots, GridH2QueryContext qctx) index(j).releaseSnapshot(); // Drop invalidated snapshot. - actualSnapshot.compareAndSet(snapshots, null); + actualSnapshot.compareAndSet(threadLocalSegmentId(), segmentSnapshot, null); return null; } - snapshots[i - 1] = s; + segmentSnapshot[i - 1] = s; } - return snapshots; + return segmentSnapshot; } /** {@inheritDoc} */ @@ -585,7 +603,7 @@ else if (old != null) // Row was not replaced, need to remove manually. for (int i = 2, len = idxs.size(); i < len; i++) { Row res = index(i).remove(old); - assert eq(pk, res, old): "\n" + old + "\n" + res + "\n" + i + " -> " + index(i).getName(); + assert eq(pk, res, old) : "\n" + old + "\n" + res + "\n" + i + " -> " + index(i).getName(); } } else @@ -593,7 +611,7 @@ else if (old != null) // Row was not replaced, need to remove manually. } // The snapshot is not actual after update. - actualSnapshot.set(null); + actualSnapshot.set(pk.segmentForRow(row), null); return true; } @@ -625,7 +643,7 @@ private static boolean eq(Index pk, SearchRow r1, SearchRow r2) { ArrayList indexes() { ArrayList res = new ArrayList<>(idxs.size() - 1); - for (int i = 1, len = idxs.size(); i < len ; i++) + for (int i = 1, len = idxs.size(); i < len; i++) res.add(index(i)); return res; @@ -643,7 +661,9 @@ public void rebuildIndexes() { ArrayList idxs0 = new ArrayList<>(idxs); try { - snapshotIndexes(null); // Allow read access while we are rebuilding indexes. + // Allow read access while we are rebuilding indexes. + for (int seg = 0; seg < actualSnapshot.length(); seg++) + snapshotIndexes(null, seg); for (int i = 1, len = idxs.size(); i < len; i++) { GridH2IndexBase newIdx = index(i).rebuild(); @@ -804,8 +824,8 @@ public static class Engine implements TableEngine { * @param desc Row descriptor. * @param factory Indexes factory. * @param space Space name. - * @throws SQLException If failed. * @return Created table. + * @throws SQLException If failed. */ public static synchronized GridH2Table createTable(Connection conn, String sql, @Nullable GridH2RowDescriptor desc, IndexesFactory factory, String space) @@ -867,7 +887,7 @@ public ScanIndex(GridH2IndexBase delegate) { double baseCost = getCostRangeIndex(masks, rows, filters, filter, sortOrder, true); int mul = delegate.getDistributedMultiplier(ses, filters, filter); - return mul * baseCost; + return mul * baseCost; } /** {@inheritDoc} */ diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java index 2baeca9be4d7f..d4970bd2e1e9e 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java @@ -166,30 +166,26 @@ public GridH2TreeIndex(String name, GridH2Table tbl, boolean pk, List tree = segments[i]; + ConcurrentNavigableMap tree = segments[seg]; - snapshot[i] = tree instanceof SnapTreeMap ? + return tree instanceof SnapTreeMap ? ((SnapTreeMap)tree).clone() : ((GridOffHeapSnapTreeMap)tree).clone(); } - return snapshot; - } - /** {@inheritDoc} */ protected ConcurrentNavigableMap treeForRead(int seg) { if (!snapshotEnabled) return segments[seg]; - final ConcurrentNavigableMap[] snapshot = threadLocalSnapshot(); + ConcurrentNavigableMap res = threadLocalSnapshot(); - if (snapshot == null) + if (res == null) return segments[seg]; - return snapshot[seg]; + return res; } /** {@inheritDoc} */ 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 6416b21d8e18e..fdf27cd2fa3aa 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 @@ -582,7 +582,7 @@ private void onQueryRequest0( Objects.requireNonNull(tbl, identifier); - tbl.snapshotIndexes(qctx); + tbl.snapshotIndexes(qctx, segmentId); snapshotedTbls.add(tbl); } diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSegmentedIndexSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSegmentedIndexSelfTest.java index 7c4fe9c790e57..951b42d858d11 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSegmentedIndexSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSegmentedIndexSelfTest.java @@ -144,7 +144,7 @@ public void testSegmentedIndex() throws Exception { */ public void testSegmentedIndexReproducableResults() throws Exception { ignite(0).createCache(cacheConfig(ORG_CACHE_NAME, true, Integer.class, Organization.class) - .setOffHeapMaxMemory(-1) // Make index snapshot to be used. + .setOffHeapMaxMemory(-1) // Force index snapshots enabling. .setMemoryMode(CacheMemoryMode.OFFHEAP_TIERED)); IgniteCache cache = ignite(0).cache(ORG_CACHE_NAME); From 987c182686962673e70398395cb27e94f894713b Mon Sep 17 00:00:00 2001 From: nikolay_tikhonov Date: Mon, 15 May 2017 11:54:16 +0300 Subject: [PATCH 070/357] 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 350734f8f9ea8d54626992248b9a1b4a15d758c1 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Mon, 15 May 2017 17:39:52 +0300 Subject: [PATCH 071/357] IGNITE-5225: 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 aaea4f08eacb0..67ae9608b0f65 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.execute(jobs.get(i)); - } + for (int i = 1; i < jobs.size(); i++) + execSvc.execute(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 74e4450c218cb..8ee794188c827 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 @@ -1873,33 +1873,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 00cb5dddb3c56a41efdf2daf80a3de0bfed68678 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Mon, 15 May 2017 20:24:10 +0300 Subject: [PATCH 072/357] 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 8ee794188c827..7c250b6c10c93 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 @@ -1853,11 +1853,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 765b6151e6ce7..9b45982ff893a 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 @@ -2769,22 +2769,27 @@ protected GridCommunicationClient createTcpClient(ClusterNode node, int connIdx) 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 a5d2b9ba3ed1e72ef3970ecfed499a97f925ff2c Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Mon, 15 May 2017 17:39:52 +0300 Subject: [PATCH 073/357] Fixed thread pools incorrect shutdown. (cherry picked from commit dacf973) --- .../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 a50661e137600..7a35a2abaa303 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.execute(jobs.get(i)); - } + for (int i = 1; i < jobs.size(); i++) + execSvc.execute(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 a204e9ac729a5..cc21b86db7c94 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 @@ -1867,33 +1867,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 5e08c2951522a640bf84b6c60f03f29edd970ffc Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Mon, 15 May 2017 20:24:10 +0300 Subject: [PATCH 074/357] 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 cc21b86db7c94..f8f1fa5bddfa0 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 @@ -1847,11 +1847,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 89ecc36d2ace1..60d97348d42cf 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 @@ -2769,22 +2769,27 @@ protected GridCommunicationClient createTcpClient(ClusterNode node, int connIdx) 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 62765b5b7598ab2a82b9c595e4a1276d9f2f5860 Mon Sep 17 00:00:00 2001 From: sboikov Date: Tue, 16 May 2017 11:30:29 +0300 Subject: [PATCH 075/357] 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 d7963cb7c5ec5965ec071f2dc603540290e5445c Mon Sep 17 00:00:00 2001 From: sboikov Date: Tue, 16 May 2017 11:30:29 +0300 Subject: [PATCH 076/357] 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 ebc4a1648a80fbbd485e4c351fce9bee163318f9 Mon Sep 17 00:00:00 2001 From: sboikov Date: Tue, 16 May 2017 11:30:29 +0300 Subject: [PATCH 077/357] 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 078/357] 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 079/357] 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 72882126c047b937bd5ed93c85e28262530e4977 Mon Sep 17 00:00:00 2001 From: Vasiliy Sisko Date: Thu, 30 Mar 2017 11:08:10 +0700 Subject: [PATCH 080/357] 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 0d3d93cd4eeb589f1b6a11b48e429defad01c82f Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Thu, 18 May 2017 19:11:08 +0300 Subject: [PATCH 081/357] IGNITE-4842 Now containsKey() respects isReadFromBackup() flag. (cherry picked from commit d84fd29) --- .../processors/cache/GridCacheAdapter.java | 4 +- .../IgniteCacheContainsKeyAtomicTest.java | 103 ++++++++++++++++++ .../testsuites/IgniteCacheTestSuite4.java | 3 + 3 files changed, 108 insertions(+), 2 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheContainsKeyAtomicTest.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 a50661e137600..da4893229799f 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, diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheContainsKeyAtomicTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheContainsKeyAtomicTest.java new file mode 100644 index 0000000000000..981d245245a20 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheContainsKeyAtomicTest.java @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import java.util.Collections; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.testframework.GridTestUtils; + +import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC; +import static org.apache.ignite.cache.CacheMode.REPLICATED; +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; + +/** + * Verifies that containsKey() works as expected on atomic cache. + */ +public class IgniteCacheContainsKeyAtomicTest extends GridCacheAbstractSelfTest { + /** Cache name. */ + public static final String CACHE_NAME = "replicated"; + + /** {@inheritDoc} */ + @Override protected int gridCount() { + return 4; + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + super.afterTest(); + + IgniteCache cache = ignite(0).cache(CACHE_NAME); + + if (cache != null) + cache.clear(); + } + + /** + * @throws Exception If failed. + */ + public void testContainsPutIfAbsent() throws Exception { + checkPutIfAbsent(false); + } + + /** + * @throws Exception If failed. + */ + public void testContainsPutIfAbsentAll() throws Exception { + checkPutIfAbsent(true); + } + + /** + * @param all Check for set of keys. + * @throws Exception If failed. + */ + private void checkPutIfAbsent(final boolean all) throws Exception { + Ignite srv = ignite(0); + + final IgniteCache cache1 = srv.getOrCreateCache(replicatedCache()); + final IgniteCache cache2 = ignite(1).getOrCreateCache(replicatedCache()); + + final AtomicInteger fails = new AtomicInteger(0); + + GridTestUtils.runMultiThreaded(new Runnable() { + @Override public void run() { + for (int i = 0; i < 100; i++) { + if (!cache1.putIfAbsent(i, i)) { + if (all ? !cache2.containsKeys(Collections.singleton(i)) : !cache2.containsKey(i)) + fails.incrementAndGet(); + } + } + } + }, 100, "put-if-abs"); + + assertEquals(0, fails.get()); + } + + /** + * @return replicated cache configuration. + */ + private CacheConfiguration replicatedCache() { + return new CacheConfiguration(CACHE_NAME) + .setAtomicityMode(ATOMIC) + .setWriteSynchronizationMode(FULL_SYNC) + .setReadFromBackup(false) // containsKey() must respect this flag + .setCacheMode(REPLICATED); + } +} 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 914aedb40764a..a13a9a4d7245a 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 @@ -66,6 +66,7 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheAtomicStoreValueTest; import org.apache.ignite.internal.processors.cache.IgniteCacheConfigurationDefaultTemplateTest; import org.apache.ignite.internal.processors.cache.IgniteCacheConfigurationTemplateTest; +import org.apache.ignite.internal.processors.cache.IgniteCacheContainsKeyAtomicTest; import org.apache.ignite.internal.processors.cache.IgniteCacheDynamicStopSelfTest; import org.apache.ignite.internal.processors.cache.IgniteCacheGetCustomCollectionsSelfTest; import org.apache.ignite.internal.processors.cache.IgniteCacheInvokeReadThroughSingleNodeTest; @@ -339,6 +340,8 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(CacheAtomicPrimarySyncBackPressureTest.class); + suite.addTestSuite(IgniteCacheContainsKeyAtomicTest.class); + return suite; } } \ No newline at end of file From 2a818d36395dd1af23acf444adf396b2e2edbede Mon Sep 17 00:00:00 2001 From: Konstantin Dudkov Date: Mon, 22 May 2017 16:28:07 +0300 Subject: [PATCH 082/357] 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 083/357] 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 084/357] 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 085/357] 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 086/357] 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 087/357] 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 088/357] 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 15d94b432fdfe458a826df6ad3c30a0408a93f49 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Thu, 25 May 2017 14:27:08 +0300 Subject: [PATCH 089/357] Backport of IGNITE-4336: Manual rebalance can't be requested twice. (cherry picked from commit 9a691c4) --- .../processors/cache/GridCacheAdapter.java | 5 +- .../GridCachePartitionExchangeManager.java | 11 ++- .../processors/cache/GridCachePreloader.java | 8 +- .../cache/GridCachePreloaderAdapter.java | 7 +- .../processors/cache/IgniteCacheProxy.java | 4 +- .../preloader/GridDhtPartitionDemander.java | 46 ++++++++++-- .../GridDhtPartitionsExchangeFuture.java | 14 +++- .../dht/preloader/GridDhtPreloader.java | 8 +- .../cache/CacheRebalancingSelfTest.java | 75 +++++++++++++++++++ .../testsuites/IgniteCacheTestSuite5.java | 3 + 10 files changed, 156 insertions(+), 25 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRebalancingSelfTest.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 b49becaacbc74..f10406fce2a80 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 @@ -86,6 +86,7 @@ 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; @@ -4535,9 +4536,7 @@ protected Object readResolve() throws ObjectStreamException { /** {@inheritDoc} */ @Override public IgniteInternalFuture rebalance() { - ctx.preloader().forcePreload(); - - return ctx.preloader().syncFuture(); + return ctx.preloader().forceRebalance(); } /** {@inheritDoc} */ 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 679395727123a..c175c43ae2f2a 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 @@ -723,9 +723,13 @@ public void forceDummyExchange(boolean reassign, * * @param exchFut Exchange future. */ - public void forcePreloadExchange(GridDhtPartitionsExchangeFuture exchFut) { + public IgniteInternalFuture forceRebalance(GridDhtPartitionsExchangeFuture exchFut) { + GridFutureAdapter fut = new GridFutureAdapter<>(); + exchWorker.addFuture( - new GridDhtPartitionsExchangeFuture(cctx, exchFut.discoveryEvent(), exchFut.exchangeId())); + new GridDhtPartitionsExchangeFuture(cctx, exchFut.discoveryEvent(), exchFut.exchangeId(), fut)); + + return fut; } /** @@ -1815,7 +1819,8 @@ void addFuture(GridDhtPartitionsExchangeFuture exchFut) { Runnable cur = cacheCtx.preloader().addAssignments(assigns, forcePreload, cnt, - r); + r, + exchFut.forcedRebalanceFuture()); if (cur != null) { rebList.add(U.maskName(cacheCtx.name())); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java index 3c4456d5984d9..0c2869101aa79 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java @@ -28,6 +28,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessageV2; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloaderAssignments; +import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.lang.IgnitePredicate; import org.jetbrains.annotations.Nullable; @@ -90,7 +91,8 @@ public interface GridCachePreloader { public Runnable addAssignments(GridDhtPreloaderAssignments assignments, boolean forcePreload, int cnt, - Runnable next); + Runnable next, + @Nullable GridFutureAdapter forcedRebFut); /** * @param p Preload predicate. @@ -150,9 +152,9 @@ public IgniteInternalFuture request(GridNearAtomicAbstractUpdateRequest AffinityTopologyVersion topVer); /** - * Force preload process. + * Force Rebalance process. */ - public void forcePreload(); + public IgniteInternalFuture forceRebalance(); /** * Unwinds undeploys. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java index 656a960b56b9f..8ae67215dabc9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java @@ -31,6 +31,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloaderAssignments; import org.apache.ignite.internal.util.future.GridFinishedFuture; +import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.lang.IgnitePredicate; import org.jetbrains.annotations.Nullable; @@ -88,8 +89,8 @@ public GridCachePreloaderAdapter(GridCacheContext cctx) { } /** {@inheritDoc} */ - @Override public void forcePreload() { - // No-op. + @Override public IgniteInternalFuture forceRebalance() { + return new GridFinishedFuture<>(true); } /** {@inheritDoc} */ @@ -166,7 +167,7 @@ public GridCachePreloaderAdapter(GridCacheContext cctx) { /** {@inheritDoc} */ @Override public Runnable addAssignments(GridDhtPreloaderAssignments assignments, boolean forcePreload, - int cnt, Runnable next) { + int cnt, Runnable next, @Nullable GridFutureAdapter forcedRebFut) { return null; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java index 873c8221ae921..13816709557a0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java @@ -2329,9 +2329,7 @@ private void onLeave(GridCacheGateway gate) { /** {@inheritDoc} */ @Override public IgniteFuture rebalance() { - ctx.preloader().forcePreload(); - - return new IgniteFutureImpl<>(ctx.preloader().syncFuture()); + return new IgniteFutureImpl<>(ctx.preloader().forceRebalance()); } /** 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..9ece00cd01049 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 @@ -55,6 +55,7 @@ import org.apache.ignite.internal.processors.timeout.GridTimeoutObject; import org.apache.ignite.internal.processors.timeout.GridTimeoutObjectAdapter; import org.apache.ignite.internal.util.GridLeanSet; +import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.tostring.GridToStringInclude; @@ -64,6 +65,7 @@ import org.apache.ignite.internal.util.typedef.internal.LT; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.lang.IgnitePredicate; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.spi.IgniteSpiException; @@ -216,9 +218,9 @@ void preloadPredicate(IgnitePredicate preloadPred) { } /** - * Force preload. + * Force Rebalance. */ - void forcePreload() { + IgniteInternalFuture forceRebalance() { GridTimeoutObject obj = lastTimeoutObj.getAndSet(null); if (obj != null) @@ -230,14 +232,31 @@ void forcePreload() { if (log.isDebugEnabled()) log.debug("Forcing rebalance event for future: " + exchFut); + final GridFutureAdapter fut = new GridFutureAdapter<>(); + exchFut.listen(new CI1>() { @Override public void apply(IgniteInternalFuture t) { - cctx.shared().exchange().forcePreloadExchange(exchFut); + IgniteInternalFuture fut0 = cctx.shared().exchange().forceRebalance(exchFut); + + fut0.listen(new IgniteInClosure>() { + @Override public void apply(IgniteInternalFuture future) { + try { + fut.onDone(future.get()); + } + catch (Exception e) { + fut.onDone(e); + } + } + }); } }); + + return fut; } else if (log.isDebugEnabled()) log.debug("Ignoring force rebalance request (no topology event happened yet)."); + + return new GridFinishedFuture<>(true); } /** @@ -274,16 +293,20 @@ void onTopologyChanged(GridDhtPartitionsExchangeFuture lastFut) { * @param assigns Assignments. * @param force {@code True} if dummy reassign. * @param cnt Counter. + * * @param forcedRebFut External future for forced rebalance. * @param next Runnable responsible for cache rebalancing start. * @return Rebalancing runnable. */ Runnable addAssignments(final GridDhtPreloaderAssignments assigns, boolean force, int cnt, - final Runnable next) { + final Runnable next, + @Nullable final GridFutureAdapter forcedRebFut) { if (log.isDebugEnabled()) log.debug("Adding partition assignments: " + assigns); + assert force == (forcedRebFut != null); + long delay = cctx.config().getRebalanceDelay(); if (delay == 0 || force) { @@ -301,6 +324,19 @@ Runnable addAssignments(final GridDhtPreloaderAssignments assigns, }); } + if (forcedRebFut != null) { + fut.listen(new CI1>() { + @Override public void apply(IgniteInternalFuture future) { + try { + forcedRebFut.onDone(future.get()); + } + catch (Exception e) { + forcedRebFut.onDone(e); + } + } + }); + } + rebalanceFut = fut; fut.sendRebalanceStartedEvent(); @@ -383,7 +419,7 @@ else if (delay > 0) { @Override public void onTimeout() { exchFut.listen(new CI1>() { @Override public void apply(IgniteInternalFuture f) { - cctx.shared().exchange().forcePreloadExchange(exchFut); + cctx.shared().exchange().forceRebalance(exchFut); } }); } 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 9aa0755269b9f..2245d17e83523 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 @@ -197,6 +197,9 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter forcedRebFut; + /** * Dummy future created to trigger reassignments if partition * topology changed while preloading. @@ -230,15 +233,17 @@ public GridDhtPartitionsExchangeFuture( * @param cctx Cache context. * @param discoEvt Discovery event. * @param exchId Exchange id. + * @param forcedRebFut Forced Rebalance future. */ public GridDhtPartitionsExchangeFuture(GridCacheSharedContext cctx, DiscoveryEvent discoEvt, - GridDhtPartitionExchangeId exchId) { + GridDhtPartitionExchangeId exchId, GridFutureAdapter forcedRebFut) { dummy = false; forcePreload = true; this.exchId = exchId; this.discoEvt = discoEvt; this.cctx = cctx; + this.forcedRebFut = forcedRebFut; reassign = true; @@ -409,6 +414,13 @@ public GridDhtPartitionExchangeId exchangeId() { return exchId; } + /** + * @return Forced Rebalance future. + */ + @Nullable public GridFutureAdapter forcedRebalanceFuture() { + return forcedRebFut; + } + /** * @return {@code true} if entered to busy state. */ 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..4aff4d5a71f42 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 @@ -413,8 +413,8 @@ public void handleDemandMessage(int idx, UUID id, GridDhtPartitionDemandMessage /** {@inheritDoc} */ @Override public Runnable addAssignments(GridDhtPreloaderAssignments assignments, - boolean forcePreload, int cnt, Runnable next) { - return demander.addAssignments(assignments, forcePreload, cnt, next); + boolean forcePreload, int cnt, Runnable next, @Nullable GridFutureAdapter forcedRebFut) { + return demander.addAssignments(assignments, forcePreload, cnt, next, forcedRebFut); } /** @@ -728,8 +728,8 @@ private GridDhtFuture request0(Collection keys, Affinity } /** {@inheritDoc} */ - @Override public void forcePreload() { - demander.forcePreload(); + @Override public IgniteInternalFuture forceRebalance() { + return demander.forceRebalance(); } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRebalancingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRebalancingSelfTest.java new file mode 100644 index 0000000000000..8d1f67af2c33f --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRebalancingSelfTest.java @@ -0,0 +1,75 @@ +/* + * 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 org.apache.ignite.IgniteCache; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.util.future.IgniteFutureImpl; +import org.apache.ignite.lang.IgniteFuture; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Test for rebalancing. + */ +public class CacheRebalancingSelfTest extends GridCommonAbstractTest { + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setCacheConfiguration(new CacheConfiguration()); + + return cfg; + } + + /** + * @throws Exception If failed. + */ + public void testRebalanceFuture() throws Exception { + IgniteEx ignite0 = startGrid(0); + startGrid(1); + + IgniteCache cache = ignite0.cache(null); + + IgniteFuture fut1 = cache.rebalance(); + + fut1.get(); + + startGrid(2); + + IgniteFuture fut2 = cache.rebalance(); + + assert internalFuture(fut2) != internalFuture(fut1); + + fut2.get(); + } + + /** + * @param future Future. + * @return Internal future. + */ + private static IgniteInternalFuture internalFuture(IgniteFuture future) { + assert future instanceof IgniteFutureImpl; + + return ((IgniteFutureImpl)future).internalFuture(); + } + +} 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 421676654e7d5..58a26e76ddd2f 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 @@ -20,6 +20,7 @@ import junit.framework.TestSuite; import org.apache.ignite.internal.processors.cache.CacheKeepBinaryTransactionTest; import org.apache.ignite.internal.processors.cache.CacheNearReaderUpdateTest; +import org.apache.ignite.internal.processors.cache.CacheRebalancingSelfTest; import org.apache.ignite.internal.processors.cache.CacheSerializableTransactionsTest; import org.apache.ignite.internal.processors.cache.GridCacheSwapSpaceSpiConsistencySelfTest; import org.apache.ignite.internal.processors.cache.EntryVersionConsistencyReadThroughTest; @@ -64,6 +65,8 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(GridCacheOffHeapCleanupTest.class); + suite.addTestSuite(CacheRebalancingSelfTest.class); + return suite; } } From 3a12fee29625de8d75a291e39b7d52c5f5111fb4 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Thu, 25 May 2017 19:38:59 +0300 Subject: [PATCH 090/357] Minors fix segmented indices snapshots. --- .../processors/query/h2/opt/GridH2Table.java | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java index 6b087c2cdf91e..27bcf93bbcbb2 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java @@ -311,7 +311,7 @@ public void snapshotIndexes(GridH2QueryContext qctx, int segment) { segmentSnapshot = actualSnapshot.get(segment); if (segmentSnapshot != null) { // Reuse existing snapshot without locking. - segmentSnapshot = doSnapshotIndexes(segmentSnapshot, qctx); + segmentSnapshot = doSnapshotIndexes(segment, segmentSnapshot, qctx); if (segmentSnapshot != null) return; // Reused successfully. @@ -328,10 +328,10 @@ public void snapshotIndexes(GridH2QueryContext qctx, int segment) { segmentSnapshot = actualSnapshot.get(segment); if (segmentSnapshot != null) // Try reusing. - segmentSnapshot = doSnapshotIndexes(segmentSnapshot, qctx); + segmentSnapshot = doSnapshotIndexes(segment, segmentSnapshot, qctx); if (segmentSnapshot == null) { // Reuse failed, produce new snapshots. - segmentSnapshot = doSnapshotIndexes(null, qctx); + segmentSnapshot = doSnapshotIndexes(segment, null, qctx); assert segmentSnapshot != null; @@ -390,12 +390,13 @@ public Lock lock(boolean exclusive, long waitMillis) { * Must be called inside of write lock because when using multiple indexes we have to ensure that all of them have * the same contents at snapshot taking time. * + * @param segment id of index segment snapshot. * @param segmentSnapshot snapshot to be reused. * @param qctx Query context. * @return New indexes data snapshot. */ @SuppressWarnings("unchecked") - private Object[] doSnapshotIndexes(Object[] segmentSnapshot, GridH2QueryContext qctx) { + private Object[] doSnapshotIndexes(int segment, Object[] segmentSnapshot, GridH2QueryContext qctx) { assert snapshotEnabled; if (segmentSnapshot == null) // Nothing to reuse, create new snapshots. @@ -418,7 +419,7 @@ private Object[] doSnapshotIndexes(Object[] segmentSnapshot, GridH2QueryContext index(j).releaseSnapshot(); // Drop invalidated snapshot. - actualSnapshot.compareAndSet(threadLocalSegmentId(), segmentSnapshot, null); + actualSnapshot.compareAndSet(segment, segmentSnapshot, null); return null; } @@ -611,7 +612,8 @@ else if (old != null) // Row was not replaced, need to remove manually. } // The snapshot is not actual after update. - actualSnapshot.set(pk.segmentForRow(row), null); + if (snapshotEnabled) + actualSnapshot.set(pk.segmentForRow(row), null); return true; } From 7362d3c692c28245b193658d727b20caa62ffd38 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Thu, 25 May 2017 20:13:01 +0300 Subject: [PATCH 091/357] Merge compilation fix --- ...niteClientCacheInitializationFailTest.java | 29 +++++++++++++++---- 1 file changed, 23 insertions(+), 6 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java index 63dd57527183b..948461e19c482 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java @@ -30,6 +30,7 @@ import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteDataStreamer; import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.query.QueryCursor; import org.apache.ignite.cache.query.SqlFieldsQuery; @@ -242,16 +243,21 @@ private static class FailedIndexing implements GridQueryIndexing { } /** {@inheritDoc} */ - @Override public GridQueryFieldsResult queryLocalSqlFields(@Nullable String spaceName, String qry, - Collection params, IndexingQueryFilter filter, boolean enforceJoinOrder, int timeout, - GridQueryCancel cancel) throws IgniteCheckedException { + @Override public QueryCursor> queryLocalSqlFields(GridCacheContext cctx, + SqlFieldsQuery qry, + IndexingQueryFilter filter, GridQueryCancel cancel) throws IgniteCheckedException { return null; } /** {@inheritDoc} */ - @Override public GridCloseableIterator> queryLocalSql(@Nullable String spaceName, - String qry, String alias, Collection params, GridQueryTypeDescriptor type, - IndexingQueryFilter filter) throws IgniteCheckedException { + @Override public long streamUpdateQuery(@Nullable String spaceName, String qry, + @Nullable Object[] params, IgniteDataStreamer streamer) throws IgniteCheckedException { + return 0; + } + + /** {@inheritDoc} */ + @Override public QueryCursor> queryLocalSql(GridCacheContext cctx, SqlQuery qry, + IndexingQueryFilter filter, boolean keepBinary) throws IgniteCheckedException { return null; } @@ -328,6 +334,11 @@ private static class FailedIndexing implements GridQueryIndexing { return null; } + /** {@inheritDoc} */ + @Override public String space(String schemaName) { + return null; + } + /** {@inheritDoc} */ @Override public Collection runningQueries(long duration) { return null; @@ -342,5 +353,11 @@ private static class FailedIndexing implements GridQueryIndexing { @Override public void cancelAllQueries() { } + + /** {@inheritDoc} */ + @Override public IgniteDataStreamer createStreamer(String spaceName, PreparedStatement nativeStmt, + long autoFlushFreq, int nodeBufSize, int nodeParOps, boolean allowOverwrite) { + return null; + } } } From 26072dffb8f5b28693731f8367872a8e1e6dfe7e Mon Sep 17 00:00:00 2001 From: agura Date: Thu, 18 May 2017 19:40:09 +0300 Subject: [PATCH 092/357] ignite-5203 Simple BLOB support added --- .../JdbcAbstractDmlStatementSelfTest.java | 54 +- .../JdbcAbstractUpdateStatementSelfTest.java | 11 +- .../ignite/internal/jdbc2/JdbcBlobTest.java | 485 ++++++++++++++++++ .../jdbc2/JdbcInsertStatementSelfTest.java | 12 +- .../jdbc2/JdbcMergeStatementSelfTest.java | 12 +- .../jdbc2/JdbcPreparedStatementSelfTest.java | 46 ++ .../jdbc/suite/IgniteJdbcDriverTestSuite.java | 6 +- .../ignite/internal/jdbc2/JdbcBlob.java | 188 +++++++ .../ignite/internal/jdbc2/JdbcConnection.java | 2 +- .../internal/jdbc2/JdbcPreparedStatement.java | 4 +- .../ignite/internal/jdbc2/JdbcResultSet.java | 8 +- 11 files changed, 799 insertions(+), 29 deletions(-) create mode 100644 modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcBlobTest.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcBlob.java diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java index 4a97aefcfcbe0..76597f1668bc8 100644 --- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java @@ -18,9 +18,12 @@ package org.apache.ignite.internal.jdbc2; import java.io.Serializable; +import java.io.UnsupportedEncodingException; +import java.sql.Blob; import java.sql.Connection; import java.sql.DriverManager; import java.sql.ResultSet; +import java.sql.SQLException; import java.sql.Statement; import java.util.Collections; import org.apache.ignite.cache.CachePeekMode; @@ -44,6 +47,9 @@ * Statement test. */ public abstract class JdbcAbstractDmlStatementSelfTest extends GridCommonAbstractTest { + /** UTF 16 character set name. */ + private static final String UTF_16 = "UTF-16"; // RAWTOHEX function use UTF-16 for conversion strings to byte arrays. + /** IP finder. */ private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); @@ -54,7 +60,7 @@ public abstract class JdbcAbstractDmlStatementSelfTest extends GridCommonAbstrac static final String BASE_URL_BIN = CFG_URL_PREFIX + "modules/clients/src/test/config/jdbc-bin-config.xml"; /** SQL SELECT query for verification. */ - private static final String SQL_SELECT = "select _key, id, firstName, lastName, age from Person"; + private static final String SQL_SELECT = "select _key, id, firstName, lastName, age, data from Person"; /** Connection. */ protected Connection conn; @@ -117,6 +123,7 @@ IgniteConfiguration getBinaryConfiguration(String gridName) throws Exception { e.addQueryField("age", Integer.class.getName(), null); e.addQueryField("firstName", String.class.getName(), null); e.addQueryField("lastName", String.class.getName(), null); + e.addQueryField("data", byte[].class.getName(), null); ccfg.setQueryEntities(Collections.singletonList(e)); @@ -165,6 +172,7 @@ protected String getCfgUrl() { assertEquals("John", rs.getString("firstName")); assertEquals("White", rs.getString("lastName")); assertEquals(25, rs.getInt("age")); + assertEquals("White", str(getBytes(rs.getBlob("data")))); break; case 2: @@ -172,6 +180,7 @@ protected String getCfgUrl() { assertEquals("Joe", rs.getString("firstName")); assertEquals("Black", rs.getString("lastName")); assertEquals(35, rs.getInt("age")); + assertEquals("Black", str(getBytes(rs.getBlob("data")))); break; case 3: @@ -179,6 +188,7 @@ protected String getCfgUrl() { assertEquals("Mike", rs.getString("firstName")); assertEquals("Green", rs.getString("lastName")); assertEquals(40, rs.getInt("age")); + assertEquals("Green", str(getBytes(rs.getBlob("data")))); break; case 4: @@ -186,6 +196,7 @@ protected String getCfgUrl() { assertEquals("Leah", rs.getString("firstName")); assertEquals("Grey", rs.getString("lastName")); assertEquals(22, rs.getInt("age")); + assertEquals("Grey", str(getBytes(rs.getBlob("data")))); break; default: @@ -199,6 +210,42 @@ protected String getCfgUrl() { assertEquals(0, grid(0).cache(null).size(CachePeekMode.ALL)); } + /** + * @param str String. + */ + static byte[] getBytes(String str) { + try { + return str.getBytes(UTF_16); + } + catch (UnsupportedEncodingException e) { + throw new RuntimeException(e); + } + } + + /** + * @param blob Blob. + */ + static byte[] getBytes(Blob blob) { + try { + return blob.getBytes(1, (int)blob.length()); + } + catch (SQLException e) { + throw new RuntimeException(e); + } + } + + /** + * @param arr Array. + */ + static String str(byte[] arr) { + try { + return new String(arr, UTF_16); + } + catch (UnsupportedEncodingException e) { + throw new RuntimeException(e); + } + } + /** * Person. */ @@ -220,6 +267,10 @@ static class Person implements Serializable { @QuerySqlField private final int age; + /** Binary data. */ + @QuerySqlField + private final byte[] data; + /** * @param id ID. * @param firstName First name. @@ -235,6 +286,7 @@ static class Person implements Serializable { this.firstName = firstName; this.lastName = lastName; this.age = age; + this.data = getBytes(lastName); } /** {@inheritDoc} */ diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractUpdateStatementSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractUpdateStatementSelfTest.java index a20b8157ef1e8..ace1be665614c 100644 --- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractUpdateStatementSelfTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractUpdateStatementSelfTest.java @@ -19,12 +19,15 @@ import java.sql.Statement; +/** + * + */ public abstract class JdbcAbstractUpdateStatementSelfTest extends JdbcAbstractDmlStatementSelfTest { /** SQL query to populate cache. */ - private static final String ITEMS_SQL = "insert into Person(_key, id, firstName, lastName, age) values " + - "('p1', 1, 'John', 'White', 25), " + - "('p2', 2, 'Joe', 'Black', 35), " + - "('p3', 3, 'Mike', 'Green', 40)"; + private static final String ITEMS_SQL = "insert into Person(_key, id, firstName, lastName, age, data) values " + + "('p1', 1, 'John', 'White', 25, RAWTOHEX('White')), " + + "('p2', 2, 'Joe', 'Black', 35, RAWTOHEX('Black')), " + + "('p3', 3, 'Mike', 'Green', 40, RAWTOHEX('Green'))"; /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcBlobTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcBlobTest.java new file mode 100644 index 0000000000000..9e0e0d2f6aab1 --- /dev/null +++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcBlobTest.java @@ -0,0 +1,485 @@ +/* + * 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.jdbc2; + +import java.io.IOException; +import java.io.InputStream; +import java.sql.SQLException; +import java.util.Arrays; +import junit.framework.TestCase; + +/** + * + */ +public class JdbcBlobTest extends TestCase { + /** + * @throws Exception If failed. + */ + public void testLength() throws Exception { + JdbcBlob blob = new JdbcBlob(new byte[16]); + + assertEquals(16, (int)blob.length()); + + blob.free(); + + try { + blob.length(); + + fail(); + } + catch (SQLException e) { + // No-op. + } + } + + /** + * @throws Exception If failed. + */ + public void testGetBytes() throws Exception { + byte[] arr = new byte[] {0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15}; + + JdbcBlob blob = new JdbcBlob(arr); + + try { + blob.getBytes(0, 16); + + fail(); + } + catch (SQLException e) { + // No-op. + } + + try { + blob.getBytes(17, 16); + + fail(); + } + catch (SQLException e) { + // No-op. + } + + try { + blob.getBytes(1, -1); + + fail(); + } + catch (SQLException e) { + // No-op. + } + + byte[] res = blob.getBytes(1, 0); + assertEquals(0, res.length); + + assertTrue(Arrays.equals(arr, blob.getBytes(1, 16))); + + res = blob.getBytes(1, 20); + assertEquals(16, res.length); + assertTrue(Arrays.equals(arr, res)); + + res = blob.getBytes(1, 10); + assertEquals(10, res.length); + assertEquals(0, res[0]); + assertEquals(9, res[9]); + + res = blob.getBytes(7, 10); + assertEquals(10, res.length); + assertEquals(6, res[0]); + assertEquals(15, res[9]); + + res = blob.getBytes(7, 20); + assertEquals(10, res.length); + assertEquals(6, res[0]); + assertEquals(15, res[9]); + + res = blob.getBytes(1, 0); + assertEquals(0, res.length); + + blob.free(); + + try { + blob.getBytes(1, 16); + + fail(); + } + catch (SQLException e) { + // No-op. + } + } + + /** + * @throws Exception If failed. + */ + public void testGetBinaryStream() throws Exception { + byte[] arr = new byte[] {0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15}; + + JdbcBlob blob = new JdbcBlob(arr); + + InputStream is = blob.getBinaryStream(); + + byte[] res = readBytes(is); + + assertTrue(Arrays.equals(arr, res)); + + blob.free(); + + try { + blob.getBinaryStream(); + + fail(); + } + catch (SQLException e) { + // No-op. + } + } + + /** + * @throws Exception If failed. + */ + public void testGetBinaryStreamWithParams() throws Exception { + byte[] arr = new byte[] {0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15}; + + JdbcBlob blob = new JdbcBlob(arr); + + try { + blob.getBinaryStream(0, arr.length); + + fail(); + } + catch (SQLException e) { + // No-op. + } + + try { + blob.getBinaryStream(1, 0); + + fail(); + } + catch (SQLException e) { + // No-op. + } + + try { + blob.getBinaryStream(17, arr.length); + + fail(); + } + catch (SQLException e) { + // No-op. + } + + try { + blob.getBinaryStream(1, arr.length + 1); + + fail(); + } + catch (SQLException e) { + // No-op. + } + + InputStream is = blob.getBinaryStream(1, arr.length); + byte[] res = readBytes(is); + assertTrue(Arrays.equals(arr, res)); + + is = blob.getBinaryStream(1, 10); + res = readBytes(is); + assertEquals(10, res.length); + assertEquals(0, res[0]); + assertEquals(9, res[9]); + + is = blob.getBinaryStream(6, 10); + res = readBytes(is); + assertEquals(10, res.length); + assertEquals(5, res[0]); + assertEquals(14, res[9]); + + blob.free(); + + try { + blob.getBinaryStream(1, arr.length); + + fail(); + } + catch (SQLException e) { + // No-op. + } + } + + /** + * @throws Exception If failed. + */ + public void testPositionBytePattern() throws Exception { + byte[] arr = new byte[] {0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15}; + + JdbcBlob blob = new JdbcBlob(arr); + + assertEquals(-1, blob.position(new byte[] {1, 2, 3}, 0)); + assertEquals(-1, blob.position(new byte[] {1, 2, 3}, arr.length + 1)); + assertEquals(-1, blob.position(new byte[0], 1)); + assertEquals(-1, blob.position(new byte[17], 1)); + assertEquals(-1, blob.position(new byte[] {3, 2, 1}, 1)); + assertEquals(1, blob.position(new byte[] {0, 1, 2}, 1)); + assertEquals(2, blob.position(new byte[] {1, 2, 3}, 1)); + assertEquals(2, blob.position(new byte[] {1, 2, 3}, 2)); + assertEquals(-1, blob.position(new byte[] {1, 2, 3}, 3)); + assertEquals(14, blob.position(new byte[] {13, 14, 15}, 3)); + assertEquals(-1, blob.position(new byte[] {0, 1, 3}, 1)); + assertEquals(-1, blob.position(new byte[] {0, 2, 3}, 1)); + assertEquals(-1, blob.position(new byte[] {1, 2, 4}, 1)); + + blob.free(); + + try { + blob.position(new byte[] {0, 1, 2}, 1); + + fail(); + } + catch (SQLException e) { + // No-op. + } + } + + /** + * @throws Exception If failed. + */ + public void testPositionBlobPattern() throws Exception { + byte[] arr = new byte[] {0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15}; + + JdbcBlob blob = new JdbcBlob(arr); + + assertEquals(-1, blob.position(new JdbcBlob(new byte[] {1, 2, 3}), 0)); + assertEquals(-1, blob.position(new JdbcBlob(new byte[] {1, 2, 3}), arr.length + 1)); + assertEquals(-1, blob.position(new JdbcBlob(new byte[0]), 1)); + assertEquals(-1, blob.position(new JdbcBlob(new byte[17]), 1)); + assertEquals(-1, blob.position(new JdbcBlob(new byte[] {3, 2, 1}), 1)); + assertEquals(1, blob.position(new JdbcBlob(new byte[] {0, 1, 2}), 1)); + assertEquals(2, blob.position(new JdbcBlob(new byte[] {1, 2, 3}), 1)); + assertEquals(2, blob.position(new JdbcBlob(new byte[] {1, 2, 3}), 2)); + assertEquals(-1, blob.position(new JdbcBlob(new byte[] {1, 2, 3}), 3)); + assertEquals(14, blob.position(new JdbcBlob(new byte[] {13, 14, 15}), 3)); + assertEquals(-1, blob.position(new JdbcBlob(new byte[] {0, 1, 3}), 1)); + assertEquals(-1, blob.position(new JdbcBlob(new byte[] {0, 2, 3}), 1)); + assertEquals(-1, blob.position(new JdbcBlob(new byte[] {1, 2, 4}), 1)); + + blob.free(); + + try { + blob.position(new JdbcBlob(new byte[] {0, 1, 2}), 1); + + fail(); + } + catch (SQLException e) { + // No-op. + } + } + + /** + * @throws Exception If failed. + */ + public void testSetBytes() throws Exception { + byte[] arr = new byte[] {0, 1, 2, 3, 4, 5, 6, 7}; + + JdbcBlob blob = new JdbcBlob(arr); + + try { + blob.setBytes(0, new byte[4]); + + fail(); + } + catch (SQLException e) { + // No-op. + } + + try { + blob.setBytes(17, new byte[4]); + + fail(); + } + catch (ArrayIndexOutOfBoundsException e) { + // No-op. + } + + assertEquals(4, blob.setBytes(1, new byte[] {3, 2, 1, 0})); + assertTrue(Arrays.equals(new byte[] {3, 2, 1, 0, 4, 5, 6, 7}, blob.getBytes(1, arr.length))); + + assertEquals(4, blob.setBytes(5, new byte[] {7, 6, 5, 4})); + assertTrue(Arrays.equals(new byte[] {3, 2, 1, 0, 7, 6, 5, 4}, blob.getBytes(1, arr.length))); + + assertEquals(4, blob.setBytes(7, new byte[] {8, 9, 10, 11})); + assertTrue(Arrays.equals(new byte[] {3, 2, 1, 0, 7, 6, 8, 9, 10, 11}, blob.getBytes(1, (int)blob.length()))); + + blob = new JdbcBlob(new byte[] {15, 16}); + assertEquals(8, blob.setBytes(1, new byte[] {0, 1, 2, 3, 4, 5, 6, 7})); + assertTrue(Arrays.equals(new byte[] {0, 1, 2, 3, 4, 5, 6, 7}, blob.getBytes(1, (int)blob.length()))); + + blob.free(); + + try { + blob.setBytes(1, new byte[] {0, 1, 2}); + + fail(); + } + catch (SQLException e) { + // No-op. + } + } + + /** + * @throws Exception If failed. + */ + public void testSetBytesWithOffsetAndLength() throws Exception { + byte[] arr = new byte[] {0, 1, 2, 3, 4, 5, 6, 7}; + + JdbcBlob blob = new JdbcBlob(arr); + + try { + blob.setBytes(0, new byte[4], 0, 2); + + fail(); + } + catch (SQLException e) { + // No-op. + } + + try { + blob.setBytes(17, new byte[4], 0, 2); + + fail(); + } + catch (ArrayIndexOutOfBoundsException e) { + // No-op. + } + + try { + blob.setBytes(1, new byte[4], -1, 2); + + fail(); + } + catch (ArrayIndexOutOfBoundsException e) { + // No-op. + } + + try { + blob.setBytes(1, new byte[4], 0, 5); + + fail(); + } + catch (ArrayIndexOutOfBoundsException e) { + // No-op. + } + + assertEquals(4, blob.setBytes(1, new byte[] {3, 2, 1, 0}, 0, 4)); + assertTrue(Arrays.equals(new byte[] {3, 2, 1, 0, 4, 5, 6, 7}, blob.getBytes(1, arr.length))); + + assertEquals(4, blob.setBytes(5, new byte[] {7, 6, 5, 4}, 0, 4)); + assertTrue(Arrays.equals(new byte[] {3, 2, 1, 0, 7, 6, 5, 4}, blob.getBytes(1, arr.length))); + + assertEquals(4, blob.setBytes(7, new byte[] {8, 9, 10, 11}, 0, 4)); + assertTrue(Arrays.equals(new byte[] {3, 2, 1, 0, 7, 6, 8, 9, 10, 11}, blob.getBytes(1, (int)blob.length()))); + + assertEquals(2, blob.setBytes(1, new byte[] {3, 2, 1, 0}, 2, 2)); + assertTrue(Arrays.equals(new byte[] {1, 0, 1, 0, 7, 6, 8, 9, 10, 11}, blob.getBytes(1, (int)blob.length()))); + + assertEquals(2, blob.setBytes(9, new byte[] {3, 2, 1, 0}, 1, 2)); + assertTrue(Arrays.equals(new byte[] {1, 0, 1, 0, 7, 6, 8, 9, 2, 1}, blob.getBytes(1, (int)blob.length()))); + + assertEquals(3, blob.setBytes(9, new byte[] {3, 2, 1, 0}, 0, 3)); + assertTrue(Arrays.equals(new byte[] {1, 0, 1, 0, 7, 6, 8, 9, 3, 2, 1}, blob.getBytes(1, (int)blob.length()))); + + blob = new JdbcBlob(new byte[] {15, 16}); + assertEquals(8, blob.setBytes(1, new byte[] {0, 1, 2, 3, 4, 5, 6, 7}, 0, 8)); + assertTrue(Arrays.equals(new byte[] {0, 1, 2, 3, 4, 5, 6, 7}, blob.getBytes(1, (int)blob.length()))); + + blob.free(); + + try { + blob.setBytes(1, new byte[] {0, 1, 2}, 0, 2); + + fail(); + } + catch (SQLException e) { + // No-op. + } + } + + /** + * @throws Exception If failed. + */ + public void testTruncate() throws Exception { + byte[] arr = new byte[] {0, 1, 2, 3, 4, 5, 6, 7}; + + JdbcBlob blob = new JdbcBlob(arr); + + try { + blob.truncate(-1); + + fail(); + } + catch(SQLException e) { + // No-op. + } + + try { + blob.truncate(arr.length + 1); + + fail(); + } + catch(SQLException e) { + // No-op. + } + + blob.truncate(4); + assertTrue(Arrays.equals(new byte[] {0, 1, 2, 3}, blob.getBytes(1, (int)blob.length()))); + + blob.truncate(0); + assertEquals(0, (int)blob.length()); + + blob.free(); + + try { + blob.truncate(0); + + fail(); + } + catch (SQLException e) { + // No-op. + System.out.println(); + } + } + + /** + * @param is Input stream. + */ + private static byte[] readBytes(InputStream is) throws IOException { + byte[] tmp = new byte[16]; + + int i = 0; + int read; + int cnt = 0; + + while ((read = is.read()) != -1) { + tmp[i++] = (byte)read; + cnt++; + } + + byte[] res = new byte[cnt]; + + System.arraycopy(tmp, 0, res, 0, cnt); + + return res; + } +} \ No newline at end of file diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcInsertStatementSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcInsertStatementSelfTest.java index 7fc92de20595c..9a44006dd6055 100644 --- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcInsertStatementSelfTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcInsertStatementSelfTest.java @@ -31,14 +31,14 @@ */ public class JdbcInsertStatementSelfTest extends JdbcAbstractDmlStatementSelfTest { /** SQL query. */ - private static final String SQL = "insert into Person(_key, id, firstName, lastName, age) values " + - "('p1', 1, 'John', 'White', 25), " + - "('p2', 2, 'Joe', 'Black', 35), " + - "('p3', 3, 'Mike', 'Green', 40)"; + private static final String SQL = "insert into Person(_key, id, firstName, lastName, age, data) values " + + "('p1', 1, 'John', 'White', 25, RAWTOHEX('White')), " + + "('p2', 2, 'Joe', 'Black', 35, RAWTOHEX('Black')), " + + "('p3', 3, 'Mike', 'Green', 40, RAWTOHEX('Green'))"; /** SQL query. */ - private static final String SQL_PREPARED = "insert into Person(_key, id, firstName, lastName, age) values " + - "(?, ?, ?, ?, ?), (?, ?, ?, ?, ?)"; + private static final String SQL_PREPARED = "insert into Person(_key, id, firstName, lastName, age, data) values " + + "(?, ?, ?, ?, ?, ?), (?, ?, ?, ?, ?, ?)"; /** Statement. */ private Statement stmt; diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcMergeStatementSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcMergeStatementSelfTest.java index ecf6032db745f..1c93239ddf518 100644 --- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcMergeStatementSelfTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcMergeStatementSelfTest.java @@ -26,14 +26,14 @@ */ public class JdbcMergeStatementSelfTest extends JdbcAbstractDmlStatementSelfTest { /** SQL query. */ - private static final String SQL = "merge into Person(_key, id, firstName, lastName, age) values " + - "('p1', 1, 'John', 'White', 25), " + - "('p2', 2, 'Joe', 'Black', 35), " + - "('p3', 3, 'Mike', 'Green', 40)"; + private static final String SQL = "merge into Person(_key, id, firstName, lastName, age, data) values " + + "('p1', 1, 'John', 'White', 25, RAWTOHEX('White')), " + + "('p2', 2, 'Joe', 'Black', 35, RAWTOHEX('Black')), " + + "('p3', 3, 'Mike', 'Green', 40, RAWTOHEX('Green'))"; /** SQL query. */ - protected static final String SQL_PREPARED = "merge into Person(_key, id, firstName, lastName, age) values " + - "(?, ?, ?, ?, ?), (?, ?, ?, ?, ?)"; + protected static final String SQL_PREPARED = "merge into Person(_key, id, firstName, lastName, age, data) values " + + "(?, ?, ?, ?, ?, ?), (?, ?, ?, ?, ?, ?)"; /** Statement. */ protected Statement stmt; 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..edc6031e7d3ec 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 @@ -98,6 +98,7 @@ public class JdbcPreparedStatementSelfTest extends GridCommonAbstractTest { o.bigVal = new BigDecimal(1); o.strVal = "str"; o.arrVal = new byte[] {1}; + o.blobVal = new byte[] {1}; o.dateVal = new Date(1); o.timeVal = new Time(1); o.tsVal = new Timestamp(1); @@ -507,6 +508,47 @@ public void testArray() throws Exception { assert cnt == 1; } + /** + * @throws Exception If failed. + */ + public void testBlob() throws Exception { + stmt = conn.prepareStatement("select * from TestObject where blobVal is not distinct from ?"); + + Blob blob = conn.createBlob(); + + blob.setBytes(1, new byte[] {1}); + + stmt.setBlob(1, blob); + + ResultSet rs = stmt.executeQuery(); + + int cnt = 0; + + while (rs.next()) { + if (cnt == 0) + assert rs.getInt("id") == 1; + + cnt++; + } + + assertEquals(1, cnt); + + stmt.setNull(1, BINARY); + + rs = stmt.executeQuery(); + + cnt = 0; + + while (rs.next()) { + if (cnt == 0) + assert rs.getInt("id") == 2; + + cnt++; + } + + assert cnt == 1; + } + /** * @throws Exception If failed. */ @@ -704,6 +746,10 @@ private static class TestObject implements Serializable { @QuerySqlField private byte[] arrVal; + /** */ + @QuerySqlField + private byte[] blobVal; + /** */ @QuerySqlField private Date dateVal; diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java index a28d29e00c52b..2718013f9d014 100644 --- a/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java +++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java @@ -18,6 +18,7 @@ package org.apache.ignite.jdbc.suite; import junit.framework.TestSuite; +import org.apache.ignite.internal.jdbc2.JdbcBlobTest; import org.apache.ignite.internal.jdbc2.JdbcDistributedJoinsQueryTest; import org.apache.ignite.jdbc.JdbcComplexQuerySelfTest; import org.apache.ignite.jdbc.JdbcConnectionSelfTest; @@ -42,7 +43,7 @@ public class IgniteJdbcDriverTestSuite extends TestSuite { public static TestSuite suite() throws Exception { TestSuite suite = new TestSuite("Ignite JDBC Driver Test Suite"); - // Thin client based driver tests + // Thin client based driver tests. suite.addTest(new TestSuite(JdbcConnectionSelfTest.class)); suite.addTest(new TestSuite(JdbcStatementSelfTest.class)); suite.addTest(new TestSuite(JdbcPreparedStatementSelfTest.class)); @@ -61,7 +62,7 @@ public static TestSuite suite() throws Exception { suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcPreparedStatementSelfTest.class)); suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcResultSetSelfTest.class)); suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcComplexQuerySelfTest.class)); - suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcDistributedJoinsQueryTest.class)); + suite.addTest(new TestSuite(JdbcDistributedJoinsQueryTest.class)); suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcMetadataSelfTest.class)); suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcEmptyCacheSelfTest.class)); suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcLocalCachesSelfTest.class)); @@ -71,6 +72,7 @@ public static TestSuite suite() throws Exception { suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcInsertStatementSelfTest.class)); suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcBinaryMarshallerInsertStatementSelfTest.class)); suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcDeleteStatementSelfTest.class)); + suite.addTest(new TestSuite(JdbcBlobTest.class)); return suite; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcBlob.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcBlob.java new file mode 100644 index 0000000000000..daa0d1fbd357d --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcBlob.java @@ -0,0 +1,188 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.jdbc2; + +import java.io.ByteArrayInputStream; +import java.io.InputStream; +import java.io.OutputStream; +import java.sql.Blob; +import java.sql.SQLException; +import java.sql.SQLFeatureNotSupportedException; +import java.util.Arrays; +import org.apache.ignite.internal.util.typedef.internal.U; + +/** + * Simple BLOB implementation. Actually there is no such entity as BLOB in Ignite. So using arrays is preferable way + * to work with binary objects. + * + * This implementation can be useful for reading binary fields of objects through JDBC. + */ +public class JdbcBlob implements Blob { + /** Byte array. */ + private byte[] arr; + + /** + * @param arr Byte array. + */ + public JdbcBlob(byte[] arr) { + this.arr = arr; + } + + /** {@inheritDoc} */ + @Override public long length() throws SQLException { + ensureNotClosed(); + + return arr.length; + } + + /** {@inheritDoc} */ + @Override public byte[] getBytes(long pos, int len) throws SQLException { + ensureNotClosed(); + + if (pos < 1 || arr.length - pos < 0 || len < 0) + throw new SQLException("Invalid argument. Position can't be less than 1 or " + + "greater than size of underlying byte array. Requested length also can't be negative " + "" + + "[pos=" + pos + ", len=" + len +']'); + + int idx = (int)(pos - 1); + + int size = len > arr.length - idx ? arr.length - idx : len; + + byte[] res = new byte[size]; + + U.arrayCopy(arr, idx, res, 0, size); + + return res; + } + + /** {@inheritDoc} */ + @Override public InputStream getBinaryStream() throws SQLException { + ensureNotClosed(); + + return new ByteArrayInputStream(arr); + } + + /** {@inheritDoc} */ + @Override public InputStream getBinaryStream(long pos, long len) throws SQLException { + ensureNotClosed(); + + if (pos < 1 || len < 1 || pos > arr.length || len > arr.length - pos + 1) + throw new SQLException("Invalid argument. Position can't be less than 1 or " + + "greater than size of underlying byte array. Requested length can't be negative and can't be " + + "greater than available bytes from given position [pos=" + pos + ", len=" + len +']'); + + + return new ByteArrayInputStream(arr, (int)(pos - 1), (int)len); + } + + /** {@inheritDoc} */ + @Override public long position(byte[] ptrn, long start) throws SQLException { + ensureNotClosed(); + + if (start < 1 || start > arr.length || ptrn.length == 0 || ptrn.length > arr.length) + return -1; + + for(int i = 0, pos = (int)(start - 1); pos < arr.length;) { + if (arr[pos] == ptrn[i]) { + pos++; + + i++; + + if (i == ptrn.length) + return pos - ptrn.length + 1; + } + else { + pos = pos - i + 1; + + i = 0; + } + } + + return -1; + } + + /** {@inheritDoc} */ + @Override public long position(Blob ptrn, long start) throws SQLException { + ensureNotClosed(); + + if (start < 1 || start > arr.length || ptrn.length() == 0 || ptrn.length() > arr.length) + return -1; + + return position(ptrn.getBytes(1, (int)ptrn.length()), start); + } + + /** {@inheritDoc} */ + @Override public int setBytes(long pos, byte[] bytes) throws SQLException { + return setBytes(pos, bytes, 0, bytes.length); + } + + /** {@inheritDoc} */ + @Override public int setBytes(long pos, byte[] bytes, int off, int len) throws SQLException { + ensureNotClosed(); + + if (pos < 1) + throw new SQLException("Invalid argument. Position can't be less than 1 [pos=" + pos + ']'); + + int idx = (int)(pos - 1); + + byte[] dst = arr; + + if (idx + len > arr.length) { + dst = new byte[arr.length + (len - (arr.length - idx))]; + + U.arrayCopy(arr, 0, dst, 0, idx); + + arr = dst; + } + + U.arrayCopy(bytes, off, dst, idx, len); + + return len; + } + + /** {@inheritDoc} */ + @Override public OutputStream setBinaryStream(long pos) throws SQLException { + throw new SQLFeatureNotSupportedException(); + } + + /** {@inheritDoc} */ + @Override public void truncate(long len) throws SQLException { + ensureNotClosed(); + + if (len < 0 || len > arr.length) + throw new SQLException("Invalid argument. Length can't be " + + "less than zero or greater than Blob length [len=" + len + ']'); + + arr = Arrays.copyOf(arr, (int)len); + + } + + /** {@inheritDoc} */ + @Override public void free() throws SQLException { + if (arr != null) + arr = null; + } + + /** + * + */ + private void ensureNotClosed() throws SQLException { + if (arr == null) + throw new SQLException("Blob instance can't be used after free() has been called."); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java index dc3fe7f454624..3810117a785ee 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java @@ -550,7 +550,7 @@ private IgniteConfiguration loadConfiguration(String cfgUrl) { @Override public Blob createBlob() throws SQLException { ensureNotClosed(); - throw new SQLFeatureNotSupportedException("SQL-specific types are not supported."); + return new JdbcBlob(new byte[0]); } /** {@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 57badd2df728e..371c009df0b8b 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 @@ -227,9 +227,7 @@ public class JdbcPreparedStatement extends JdbcStatement implements PreparedStat /** {@inheritDoc} */ @Override public void setBlob(int paramIdx, Blob x) throws SQLException { - ensureNotClosed(); - - throw new SQLFeatureNotSupportedException("SQL-specific types are not supported."); + setBytes(paramIdx, x.getBytes(1, (int)x.length())); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcResultSet.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcResultSet.java index b53521e4cfa4d..187930ec4cc66 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcResultSet.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcResultSet.java @@ -1012,9 +1012,7 @@ void closeInternal() throws SQLException { /** {@inheritDoc} */ @Override public Blob getBlob(int colIdx) throws SQLException { - ensureNotClosed(); - - throw new SQLFeatureNotSupportedException("SQL-specific types are not supported."); + return new JdbcBlob(getBytes(colIdx)); } /** {@inheritDoc} */ @@ -1045,9 +1043,7 @@ void closeInternal() throws SQLException { /** {@inheritDoc} */ @Override public Blob getBlob(String colLb) throws SQLException { - ensureNotClosed(); - - throw new SQLFeatureNotSupportedException("SQL-specific types are not supported."); + return new JdbcBlob(getBytes(colLb)); } /** {@inheritDoc} */ From d77a134fffee431cd7fa0bae2349419bc97ec1cf Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Tue, 30 May 2017 19:00:47 +0300 Subject: [PATCH 093/357] 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 d0186c368b12ab6893d0985e3fb92600708d5b65 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Tue, 30 May 2017 19:00:47 +0300 Subject: [PATCH 094/357] IGNITE-5342 - Skip permission check for TASK_EXECUTE for service jobs (cherry picked from commit d77a134) --- .../processors/affinity/GridAffinityProcessor.java | 2 +- .../internal/processors/cache/GridCacheAdapter.java | 5 ++--- .../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, 26 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 f10406fce2a80..d84060db4ce58 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,7 +84,6 @@ 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; @@ -3512,14 +3511,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 6bda8446b9b04..98439d4c4ec4b 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 d2e96bac52c0c..d16a4c48dcf86 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 @@ -192,8 +192,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 2ec63df4ed83c..f0e56c731fd2c 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 @@ -37,5 +37,8 @@ public enum GridTaskThreadContextKey { TC_SUBJ_ID, /** IO manager policy. */ - TC_IO_POLICY + TC_IO_POLICY, + + /** Skip authorization for the task. */ + TC_SKIP_AUTH } 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 0a0c5be1eb4793bc838da3184591d91537bd9cad Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Wed, 31 May 2017 15:24:14 +0300 Subject: [PATCH 095/357] Merge compilation fix. --- .../apache/ignite/testsuites/IgniteBinaryObjectsTestSuite.java | 1 + 1 file changed, 1 insertion(+) 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 0b3f2e318e3d5..9c72da5a76077 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 @@ -30,6 +30,7 @@ import org.apache.ignite.internal.binary.BinaryFooterOffsetsHeapSelfTest; import org.apache.ignite.internal.binary.BinaryFooterOffsetsOffheapSelfTest; import org.apache.ignite.internal.binary.BinaryIdentityResolverConfigurationSelfTest; +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; From bc6538e5052ff0109fa8a57e51e4678a74e89ad3 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Tue, 30 May 2017 19:00:47 +0300 Subject: [PATCH 096/357] IGNITE-5342 - Skip permission check for TASK_EXECUTE for service jobs (cherry picked from commit d77a134) --- .../processors/affinity/GridAffinityProcessor.java | 2 +- .../internal/processors/cache/GridCacheAdapter.java | 4 ++-- .../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, 26 insertions(+), 16 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 67ae9608b0f65..e17bf08dd3db6 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 @@ -3555,14 +3555,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 d864d3c01b988..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 @@ -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 61ed8a0b9e929..31cf0393e056d 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 6bda8446b9b04..98439d4c4ec4b 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 d2e96bac52c0c..d16a4c48dcf86 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 @@ -192,8 +192,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 2ec63df4ed83c..f0e56c731fd2c 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 @@ -37,5 +37,8 @@ public enum GridTaskThreadContextKey { TC_SUBJ_ID, /** IO manager policy. */ - TC_IO_POLICY + TC_IO_POLICY, + + /** Skip authorization for the task. */ + TC_SKIP_AUTH } 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 b52ea9299a007b0e5b7c0724e7756e012a6232ea Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Wed, 31 May 2017 15:35:34 +0300 Subject: [PATCH 097/357] IGNITE-5210 - If enabled security authentication, server is unable to restart if client tries to reconnect (cherry picked from commit a7e5660) --- .../ignite/spi/discovery/tcp/ClientImpl.java | 12 ++- .../discovery/AuthenticationRestartTest.java | 88 +++++++++++++++++++ .../IgniteSpiDiscoverySelfTestSuite.java | 2 + 3 files changed, 99 insertions(+), 3 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/spi/discovery/AuthenticationRestartTest.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 dcda742a764a2..a5a4e88990c4a 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 @@ -634,9 +634,12 @@ else if (addrs.isEmpty()) { if (!recon) { TcpDiscoveryNode node = locNode; - if (locNode.order() > 0) + if (locNode.order() > 0) { node = locNode.clientReconnectNode(spi.spiCtx.nodeAttributes()); + marshalCredentials(node); + } + msg = new TcpDiscoveryJoinRequestMessage(node, spi.collectExchangeData(getLocalNodeId())); } else @@ -729,8 +732,11 @@ private void marshalCredentials(TcpDiscoveryNode node) throws IgniteSpiException // Use security-unsafe getter. Map attrs = new HashMap<>(node.getAttributes()); - attrs.put(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS, - U.marshal(spi.marshaller(), attrs.get(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS))); + Object creds = attrs.get(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS); + + assert !(creds instanceof byte[]); + + attrs.put(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS, U.marshal(spi.marshaller(), creds)); node.setAttributes(attrs); } diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/AuthenticationRestartTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/AuthenticationRestartTest.java new file mode 100644 index 0000000000000..5841094bf2749 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/AuthenticationRestartTest.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery; + +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.util.lang.GridAbsPredicate; +import org.apache.ignite.lang.IgniteFuture; +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.testframework.junits.common.GridCommonAbstractTest; + +import static org.apache.ignite.testframework.GridTestUtils.waitForCondition; + +/** + * Checks whether client is able to reconnect to restarted cluster with + * enabled security. + */ +public class AuthenticationRestartTest extends GridCommonAbstractTest { + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + cfg.setClientMode(igniteInstanceName.contains("client")); + + ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setJoinTimeout(1120_000); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + TestReconnectPluginProvider.enabled = true; + TestReconnectProcessor.enabled = true; + + startGrid("server"); + startGrid("client"); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + TestReconnectPluginProvider.enabled = false; + TestReconnectProcessor.enabled = false; + + stopAllGrids(); + } + + /** + * @throws Exception If failed. + */ + public void testClientReconnect() throws Exception { + stopGrid("server"); + + final IgniteEx client = grid("client"); + + waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + return client.cluster().clientReconnectFuture() != null; + } + }, 10_000); + + startGrid("server"); + + IgniteFuture fut = client.cluster().clientReconnectFuture(); + + assertNotNull(fut); + + fut.get(); + + assertEquals(2, client.cluster().nodes().size()); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java index e6b39f740a0dd..12871492d0979 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java @@ -19,6 +19,7 @@ import junit.framework.TestSuite; import org.apache.ignite.spi.GridTcpSpiForwardingSelfTest; +import org.apache.ignite.spi.discovery.AuthenticationRestartTest; import org.apache.ignite.spi.discovery.tcp.TcpClientDiscoveryMarshallerCheckSelfTest; import org.apache.ignite.spi.discovery.tcp.TcpClientDiscoverySpiFailureTimeoutSelfTest; import org.apache.ignite.spi.discovery.tcp.TcpClientDiscoverySpiMulticastTest; @@ -87,6 +88,7 @@ public static TestSuite suite() throws Exception { suite.addTest(new TestSuite(TcpDiscoveryMultiThreadedTest.class)); suite.addTest(new TestSuite(TcpDiscoveryNodeAttributesUpdateOnReconnectTest.class)); + suite.addTest(new TestSuite(AuthenticationRestartTest.class)); // SSL. suite.addTest(new TestSuite(TcpDiscoverySslSelfTest.class)); From 10edc63b624685cf13c932497e38dad6f7a30a9f Mon Sep 17 00:00:00 2001 From: Sergi Vladykin Date: Tue, 9 May 2017 18:17:40 +0300 Subject: [PATCH 098/357] master - minor fix for subqueries with aggregates --- .../internal/processors/query/h2/sql/GridSqlQuerySplitter.java | 2 +- .../internal/processors/query/IgniteSqlSplitterSelfTest.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java index 5bd7f2b9772b6..35b30055795c2 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java @@ -1760,7 +1760,7 @@ private static boolean hasAggregates(GridSqlAst el) { // If in SELECT clause we have a subquery expression with aggregate, // we should not split it. Run the whole subquery on MAP stage. - if (el instanceof GridSqlQuery) + if (el instanceof GridSqlSubquery) return false; for (int i = 0; i < el.size(); i++) { diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java index 8daf97d50b69a..5144755b9aa70 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java @@ -198,7 +198,7 @@ public void testSubQueryWithAggregate() { c1.put(new AffinityKey<>(2, orgId), new Person2(orgId, "Another Vasya")); List> rs = c1.query(new SqlFieldsQuery("select name, " + - "(select count(1) from Person2 q where q.orgId = p.orgId) " + + "select count(1) from Person2 q where q.orgId = p.orgId " + "from Person2 p order by name desc")).getAll(); assertEquals(2, rs.size()); From 28a411653e826cebba5c2c396a6be681e36f08e4 Mon Sep 17 00:00:00 2001 From: Sergi Vladykin Date: Thu, 11 May 2017 16:01:38 +0300 Subject: [PATCH 099/357] IGNITE-5190 - ArrayIndexOutOfBoundsException in GridMergeIndexSorted --- .../h2/twostep/GridMergeIndexSorted.java | 3 + .../query/IgniteSqlSplitterSelfTest.java | 68 +++++++++++++++++++ 2 files changed, 71 insertions(+) diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndexSorted.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndexSorted.java index 361bb2d5878c0..ce01fefec9b27 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndexSorted.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndexSorted.java @@ -250,6 +250,9 @@ private void goFirst() { * */ private void goNext() { + if (off == streams.length) + return; // All streams are done. + if (streams[off].next()) bubbleUp(streams, off, streamCmp); else diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java index 5144755b9aa70..b9fc22f5c617d 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java @@ -1424,6 +1424,37 @@ public void testImplicitJoinConditionGeneration() { } } + /** + * @throws Exception If failed. + */ + public void testJoinWithSubquery() throws Exception { + IgniteCache c1 = ignite(0).createCache( + cacheConfig("Contract", true, + Integer.class, Contract.class)); + + IgniteCache c2 = ignite(0).createCache( + cacheConfig("PromoContract", true, + Integer.class, PromoContract.class)); + + for (int i = 0; i < 100; i++) { + int coId = i % 10; + int cust = i / 10; + c1.put( i, new Contract(coId, cust)); + } + + for (int i = 0; i < 10; i++) + c2.put(i, new PromoContract((i % 5) + 1, i)); + + final List> res = c2.query(new SqlFieldsQuery("SELECT CO.CO_ID \n" + + "FROM PromoContract PMC \n" + + "INNER JOIN \"Contract\".Contract CO ON PMC.CO_ID = 5 \n" + + "AND PMC.CO_ID = CO.CO_ID \n" + + "INNER JOIN (SELECT CO_ID FROM PromoContract EBP WHERE EBP.CO_ID = 5 LIMIT 1) VPMC \n" + + "ON PMC.CO_ID = VPMC.CO_ID ")).getAll(); + + assertFalse(res.isEmpty()); + } + /** @throws Exception if failed. */ public void testDistributedAggregates() throws Exception { final String cacheName = "ints"; @@ -1943,4 +1974,41 @@ private static class OrderGood implements Serializable { @QuerySqlField private int goodId; } + + /** */ + private static class Contract implements Serializable { + /** */ + @QuerySqlField(index = true) + private final int CO_ID; + + /** */ + @QuerySqlField(index = true) + private final int CUSTOMER_ID; + + /** */ + public Contract(final int CO_ID, final int CUSTOMER_ID) { + this.CO_ID = CO_ID; + this.CUSTOMER_ID = CUSTOMER_ID; + } + + } + + /** */ + public class PromoContract implements Serializable { + /** */ + @QuerySqlField(index = true, orderedGroups = { + @QuerySqlField.Group(name = "myIdx", order = 1)}) + private final int CO_ID; + + /** */ + @QuerySqlField(index = true, orderedGroups = { + @QuerySqlField.Group(name = "myIdx", order = 0)}) + private final int OFFER_ID; + + /** */ + public PromoContract(final int co_Id, final int offer_Id) { + this.CO_ID = co_Id; + this.OFFER_ID = offer_Id; + } + } } From 950a07b48d0d1f7203834fff3306018cf4ff0cf4 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Thu, 1 Jun 2017 14:17:52 +0300 Subject: [PATCH 100/357] Minor fix --- .../internal/processors/query/h2/opt/GridH2IndexBase.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 efa95a8979b34..1389460489f8f 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 @@ -316,7 +316,8 @@ protected T threadLocalSnapshot() { public void releaseSnapshot() { Object s = snapshot.get(); - assert s != null; + if (s == null) + return; // Nothing to do. snapshot.remove(); From abf4f325217c27cb5399aedc517e763092174d79 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Thu, 1 Jun 2017 14:49:20 +0300 Subject: [PATCH 101/357] Revert snapshot fix and fix tests. --- .../query/h2/opt/GridH2IndexBase.java | 3 +- .../query/h2/opt/GridH2TableSelfTest.java | 324 ++++++++++-------- 2 files changed, 184 insertions(+), 143 deletions(-) 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 1389460489f8f..efa95a8979b34 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 @@ -316,8 +316,7 @@ protected T threadLocalSnapshot() { public void releaseSnapshot() { Object s = snapshot.get(); - if (s == null) - return; // Nothing to do. + assert s != null; snapshot.remove(); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TableSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TableSelfTest.java index 8408ba06839ac..21515f9b2208e 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TableSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TableSelfTest.java @@ -128,7 +128,6 @@ private GridH2Row row(UUID id, long t, String str, long x) { ValueLong.get(x)); } - /** * Simple table test. * @@ -140,137 +139,147 @@ public void testTable() throws Exception { Random rnd = new Random(); - while(x-- > 0) { - UUID id = UUID.randomUUID(); + GridH2QueryContext qctx = new GridH2QueryContext(UUID.randomUUID(), UUID.randomUUID(), + Thread.currentThread().getId(), GridH2QueryType.LOCAL); - GridH2Row row = row(id, System.currentTimeMillis(), rnd.nextBoolean() ? id.toString() : - UUID.randomUUID().toString(), rnd.nextInt(100)); + try { + GridH2QueryContext.set(qctx); - tbl.doUpdate(row, false); - } + while (x-- > 0) { + UUID id = UUID.randomUUID(); - assertEquals(MAX_X, tbl.getRowCountApproximation()); - assertEquals(MAX_X, tbl.getRowCount(null)); + GridH2Row row = row(id, System.currentTimeMillis(), rnd.nextBoolean() ? id.toString() : + UUID.randomUUID().toString(), rnd.nextInt(100)); - for (GridH2IndexBase idx : tbl.indexes()) { - assertEquals(MAX_X, idx.getRowCountApproximation()); - assertEquals(MAX_X, idx.getRowCount(null)); - } + tbl.doUpdate(row, false); + } - // Check correct rows order. - checkOrdered((GridH2TreeIndex)tbl.indexes().get(0), new Comparator() { - @Override public int compare(SearchRow o1, SearchRow o2) { - UUID id1 = (UUID)o1.getValue(0).getObject(); - UUID id2 = (UUID)o2.getValue(0).getObject(); + assertEquals(MAX_X, tbl.getRowCountApproximation()); + assertEquals(MAX_X, tbl.getRowCount(null)); - return id1.compareTo(id2); + for (GridH2IndexBase idx : tbl.indexes()) { + assertEquals(MAX_X, idx.getRowCountApproximation()); + assertEquals(MAX_X, idx.getRowCount(null)); } - }); - checkOrdered((GridH2TreeIndex)tbl.indexes().get(1), new Comparator() { - @Override public int compare(SearchRow o1, SearchRow o2) { - Long x1 = (Long)o1.getValue(3).getObject(); - Long x2 = (Long)o2.getValue(3).getObject(); + // Check correct rows order. + checkOrdered((GridH2TreeIndex)tbl.indexes().get(0), new Comparator() { + @Override public int compare(SearchRow o1, SearchRow o2) { + UUID id1 = (UUID)o1.getValue(0).getObject(); + UUID id2 = (UUID)o2.getValue(0).getObject(); - int c = x2.compareTo(x1); + return id1.compareTo(id2); + } + }); - if (c != 0) - return c; + checkOrdered((GridH2TreeIndex)tbl.indexes().get(1), new Comparator() { + @Override public int compare(SearchRow o1, SearchRow o2) { + Long x1 = (Long)o1.getValue(3).getObject(); + Long x2 = (Long)o2.getValue(3).getObject(); - Timestamp t1 = (Timestamp)o1.getValue(1).getObject(); - Timestamp t2 = (Timestamp)o2.getValue(1).getObject(); + int c = x2.compareTo(x1); - return t1.compareTo(t2); - } - }); + if (c != 0) + return c; - checkOrdered((GridH2TreeIndex)tbl.indexes().get(2), new Comparator() { - @Override public int compare(SearchRow o1, SearchRow o2) { - String s1 = (String)o1.getValue(2).getObject(); - String s2 = (String)o2.getValue(2).getObject(); + Timestamp t1 = (Timestamp)o1.getValue(1).getObject(); + Timestamp t2 = (Timestamp)o2.getValue(1).getObject(); - return s2.compareTo(s1); - } - }); + return t1.compareTo(t2); + } + }); - // Indexes data consistency. - ArrayList idxs = tbl.indexes(); + checkOrdered((GridH2TreeIndex)tbl.indexes().get(2), new Comparator() { + @Override public int compare(SearchRow o1, SearchRow o2) { + String s1 = (String)o1.getValue(2).getObject(); + String s2 = (String)o2.getValue(2).getObject(); - checkIndexesConsistent((ArrayList)idxs, null); + return s2.compareTo(s1); + } + }); - // Check unique index. - UUID id = UUID.randomUUID(); - UUID id2 = UUID.randomUUID(); + // Indexes data consistency. + ArrayList idxs = tbl.indexes(); - assertTrue(tbl.doUpdate(row(id, System.currentTimeMillis(), id.toString(), rnd.nextInt(100)), false)); - assertTrue(tbl.doUpdate(row(id2, System.currentTimeMillis(), id2.toString(), rnd.nextInt(100)), false)); + checkIndexesConsistent((ArrayList)idxs, null); - // Check index selection. - checkQueryPlan(conn, "SELECT * FROM T", SCAN_IDX_NAME); + // Check unique index. + UUID id = UUID.randomUUID(); + UUID id2 = UUID.randomUUID(); - checkQueryPlan(conn, "SELECT * FROM T WHERE ID IS NULL", PK_NAME); - checkQueryPlan(conn, "SELECT * FROM T WHERE ID = RANDOM_UUID()", PK_NAME); - checkQueryPlan(conn, "SELECT * FROM T WHERE ID > RANDOM_UUID()", PK_NAME); - checkQueryPlan(conn, "SELECT * FROM T ORDER BY ID", PK_NAME); + assertTrue(tbl.doUpdate(row(id, System.currentTimeMillis(), id.toString(), rnd.nextInt(100)), false)); + assertTrue(tbl.doUpdate(row(id2, System.currentTimeMillis(), id2.toString(), rnd.nextInt(100)), false)); - checkQueryPlan(conn, "SELECT * FROM T WHERE STR IS NULL", STR_IDX_NAME); - checkQueryPlan(conn, "SELECT * FROM T WHERE STR = 'aaaa'", STR_IDX_NAME); - checkQueryPlan(conn, "SELECT * FROM T WHERE STR > 'aaaa'", STR_IDX_NAME); - checkQueryPlan(conn, "SELECT * FROM T ORDER BY STR DESC", STR_IDX_NAME); + // Check index selection. + checkQueryPlan(conn, "SELECT * FROM T", SCAN_IDX_NAME); - checkQueryPlan(conn, "SELECT * FROM T WHERE X IS NULL", NON_UNIQUE_IDX_NAME); - checkQueryPlan(conn, "SELECT * FROM T WHERE X = 10000", NON_UNIQUE_IDX_NAME); - checkQueryPlan(conn, "SELECT * FROM T WHERE X > 10000", NON_UNIQUE_IDX_NAME); - checkQueryPlan(conn, "SELECT * FROM T ORDER BY X DESC", NON_UNIQUE_IDX_NAME); - checkQueryPlan(conn, "SELECT * FROM T ORDER BY X DESC, T", NON_UNIQUE_IDX_NAME); + checkQueryPlan(conn, "SELECT * FROM T WHERE ID IS NULL", PK_NAME); + checkQueryPlan(conn, "SELECT * FROM T WHERE ID = RANDOM_UUID()", PK_NAME); + checkQueryPlan(conn, "SELECT * FROM T WHERE ID > RANDOM_UUID()", PK_NAME); + checkQueryPlan(conn, "SELECT * FROM T ORDER BY ID", PK_NAME); - checkQueryPlan(conn, "SELECT * FROM T ORDER BY T, X DESC", SCAN_IDX_NAME); + checkQueryPlan(conn, "SELECT * FROM T WHERE STR IS NULL", STR_IDX_NAME); + checkQueryPlan(conn, "SELECT * FROM T WHERE STR = 'aaaa'", STR_IDX_NAME); + checkQueryPlan(conn, "SELECT * FROM T WHERE STR > 'aaaa'", STR_IDX_NAME); + checkQueryPlan(conn, "SELECT * FROM T ORDER BY STR DESC", STR_IDX_NAME); - // Simple queries. + checkQueryPlan(conn, "SELECT * FROM T WHERE X IS NULL", NON_UNIQUE_IDX_NAME); + checkQueryPlan(conn, "SELECT * FROM T WHERE X = 10000", NON_UNIQUE_IDX_NAME); + checkQueryPlan(conn, "SELECT * FROM T WHERE X > 10000", NON_UNIQUE_IDX_NAME); + checkQueryPlan(conn, "SELECT * FROM T ORDER BY X DESC", NON_UNIQUE_IDX_NAME); + checkQueryPlan(conn, "SELECT * FROM T ORDER BY X DESC, T", NON_UNIQUE_IDX_NAME); - Statement s = conn.createStatement(); + checkQueryPlan(conn, "SELECT * FROM T ORDER BY T, X DESC", SCAN_IDX_NAME); - ResultSet rs = s.executeQuery("select id from t where x between 0 and 100"); + // Simple queries. - int i = 0; - while (rs.next()) - i++; + Statement s = conn.createStatement(); - assertEquals(MAX_X + 2, i); + ResultSet rs = s.executeQuery("select id from t where x between 0 and 100"); - // ----- + int i = 0; + while (rs.next()) + i++; - rs = s.executeQuery("select id from t where t is not null"); + assertEquals(MAX_X + 2, i); - i = 0; - while (rs.next()) - i++; + // ----- - assertEquals(MAX_X + 2, i); + rs = s.executeQuery("select id from t where t is not null"); - // ---- + i = 0; + while (rs.next()) + i++; - int cnt = 10 + rnd.nextInt(25); + assertEquals(MAX_X + 2, i); - long t = System.currentTimeMillis(); + // ---- - for (i = 0; i < cnt; i++) { - id = UUID.randomUUID(); + int cnt = 10 + rnd.nextInt(25); - assertTrue(tbl.doUpdate(row(id, t, id.toString(), 51), false)); - } + long t = System.currentTimeMillis(); - rs = s.executeQuery("select x, id from t where x = 51 limit " + cnt); + for (i = 0; i < cnt; i++) { + id = UUID.randomUUID(); - i = 0; + assertTrue(tbl.doUpdate(row(id, t, id.toString(), 51), false)); + } - while (rs.next()) { - assertEquals(51, rs.getInt(1)); + rs = s.executeQuery("select x, id from t where x = 51 limit " + cnt); - i++; - } + i = 0; + + while (rs.next()) { + assertEquals(51, rs.getInt(1)); + + i++; + } - assertEquals(cnt, i); + assertEquals(cnt, i); + } + finally { + qctx.clearContext(true); + } } /** @@ -296,6 +305,9 @@ public void testIndexesMultiThreadedConsistency() throws Exception { multithreaded(new Callable() { @Override public Void call() throws Exception { + GridH2QueryContext.set(new GridH2QueryContext(UUID.randomUUID(), UUID.randomUUID(), + Thread.currentThread().getId(), GridH2QueryType.LOCAL)); + Random rnd = new Random(); PreparedStatement ps1 = null; @@ -361,7 +373,7 @@ public void testIndexesMultiThreadedConsistency() throws Exception { rs = ps1.executeQuery(); - for (;;) { + for (; ; ) { assertTrue(rs.next()); if (rs.getObject(1).equals(id)) @@ -383,8 +395,8 @@ public void testIndexesMultiThreadedConsistency() throws Exception { * @throws Exception If failed. */ @SuppressWarnings("InfiniteLoopStatement") - public static void main(String ... args) throws Exception { - for (int i = 0;;) { + public static void main(String... args) throws Exception { + for (int i = 0; ; ) { GridH2TableSelfTest t = new GridH2TableSelfTest(); t.beforeTest(); @@ -398,7 +410,7 @@ public static void main(String ... args) throws Exception { } /** - * @throws Exception If failed. + * @throws Exception If failed. */ public void testRangeQuery() throws Exception { int rows = 3000; @@ -408,29 +420,39 @@ public void testRangeQuery() throws Exception { Random rnd = new Random(); - for (int i = 0 ; i < rows; i++) { - UUID id = UUID.randomUUID(); + GridH2QueryContext qctx = new GridH2QueryContext(UUID.randomUUID(), UUID.randomUUID(), + Thread.currentThread().getId(), GridH2QueryType.LOCAL); - GridH2Row row = row(id, t++, id.toString(), rnd.nextInt(xs)); + try { + GridH2QueryContext.set(qctx); - assertTrue(tbl.doUpdate(row, false)); - } + for (int i = 0; i < rows; i++) { + UUID id = UUID.randomUUID(); - PreparedStatement ps = conn.prepareStatement("select count(*) from t where x = ?"); + GridH2Row row = row(id, t++, id.toString(), rnd.nextInt(xs)); - int cnt = 0; + assertTrue(tbl.doUpdate(row, false)); + } - for (int x = 0; x < xs; x++) { - ps.setInt(1, x); + PreparedStatement ps = conn.prepareStatement("select count(*) from t where x = ?"); - ResultSet rs = ps.executeQuery(); + int cnt = 0; - assertTrue(rs.next()); + for (int x = 0; x < xs; x++) { + ps.setInt(1, x); - cnt += rs.getInt(1); - } + ResultSet rs = ps.executeQuery(); + + assertTrue(rs.next()); - assertEquals(rows, cnt); + cnt += rs.getInt(1); + } + + assertEquals(rows, cnt); + } + finally { + qctx.clearContext(false); + } } /** @@ -453,62 +475,82 @@ public void testDataLoss() throws Exception { multithreaded(new Callable() { @Override public Void call() throws Exception { - Random rnd = new Random(); + GridH2QueryContext qctx = new GridH2QueryContext(UUID.randomUUID(), UUID.randomUUID(), + Thread.currentThread().getId(), GridH2QueryType.LOCAL); + + try { + GridH2QueryContext.set(qctx); + + Random rnd = new Random(); - int offset = cntr.getAndIncrement() * iterations; + int offset = cntr.getAndIncrement() * iterations; - synchronized (ids[offset]) { - for (int i = 0; i < iterations; i++) { - UUID id = ids[offset + i]; + synchronized (ids[offset]) { + for (int i = 0; i < iterations; i++) { + UUID id = ids[offset + i]; - int x = rnd.nextInt(50); + int x = rnd.nextInt(50); - GridH2Row row = row(id, t, id.toString(), x); + GridH2Row row = row(id, t, id.toString(), x); - assertTrue(tbl.doUpdate(row, false)); + assertTrue(tbl.doUpdate(row, false)); + } } - } - offset = (offset + iterations) % ids.length; + offset = (offset + iterations) % ids.length; - synchronized (ids[offset]) { - for (int i = 0; i < iterations; i += 2) { - UUID id = ids[offset + i]; + synchronized (ids[offset]) { + for (int i = 0; i < iterations; i += 2) { + UUID id = ids[offset + i]; - int x = rnd.nextInt(50); + int x = rnd.nextInt(50); - GridH2Row row = row(id, t, id.toString(), x); + GridH2Row row = row(id, t, id.toString(), x); - if (tbl.doUpdate(row, true)) - deleted.incrementAndGet(); + if (tbl.doUpdate(row, true)) + deleted.incrementAndGet(); + } } - } - return null; + return null; + } + finally { + qctx.clearContext(false); + } } }, threads); assertTrue(deleted.get() > 0); - PreparedStatement p = conn.prepareStatement("select count(*) from t where id = ?"); + GridH2QueryContext qctx = new GridH2QueryContext(UUID.randomUUID(), UUID.randomUUID(), + Thread.currentThread().getId(), GridH2QueryType.LOCAL); - for (int i = 1; i < ids.length; i += 2) { - p.setObject(1, ids[i]); + try { + GridH2QueryContext.set(qctx); - ResultSet rs = p.executeQuery(); + PreparedStatement p = conn.prepareStatement("select count(*) from t where id = ?"); - assertTrue(rs.next()); + for (int i = 1; i < ids.length; i += 2) { + p.setObject(1, ids[i]); - assertEquals(1, rs.getInt(1)); - } + ResultSet rs = p.executeQuery(); + + assertTrue(rs.next()); - Statement s = conn.createStatement(); + assertEquals(1, rs.getInt(1)); + } + + Statement s = conn.createStatement(); - ResultSet rs = s.executeQuery("select count(*) from t"); + ResultSet rs = s.executeQuery("select count(*) from t"); - assertTrue(rs.next()); + assertTrue(rs.next()); - assertEquals(ids.length - deleted.get(), rs.getInt(1)); + assertEquals(ids.length - deleted.get(), rs.getInt(1)); + } + finally { + qctx.clearContext(false); + } } /** @@ -525,7 +567,7 @@ public void testRebuildIndexes() throws Exception { UUID id = UUID.randomUUID(); GridH2Row row = row(id, System.currentTimeMillis(), rnd.nextBoolean() ? id.toString() : - UUID.randomUUID().toString(), rnd.nextInt(100)); + UUID.randomUUID().toString(), rnd.nextInt(100)); tbl.doUpdate(row, false); } @@ -569,7 +611,7 @@ private void checkQueryPlan(Connection conn, String sql, String search) throws S String plan = r.getString(1); assertTrue("Execution plan for '" + sql + "' query should contain '" + search + "'", - plan.contains(search)); + plan.contains(search)); } } } @@ -588,7 +630,7 @@ private Set checkIndexesConsistent(ArrayList idxs, @Nullable Set iter = ((GridH2TreeIndex)idx).rows(); - while(iter.hasNext()) + while (iter.hasNext()) assertTrue(set.add(iter.next())); //((GridH2SnapTreeSet)((GridH2Index)idx).tree).print(); From 705d9cfca519183fa9e79e0686c1db48c0f5afe2 Mon Sep 17 00:00:00 2001 From: rfqu Date: Thu, 1 Jun 2017 19:31:11 +0300 Subject: [PATCH 102/357] ticket fixed: IGN-7062 (TcpDiscoverySpi ignores maxMissedClientHeartbeats property) --- .../ignite/spi/discovery/tcp/ClientImpl.java | 19 +++++- .../ignite/spi/discovery/tcp/ServerImpl.java | 58 ++++++++++++++++++- 2 files changed, 72 insertions(+), 5 deletions(-) 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..2412624e900fe 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 @@ -366,7 +366,7 @@ else if (state == DISCONNECTED) { else { final GridFutureAdapter finalFut = fut; - timer.schedule(new TimerTask() { + TimerTask task = new TimerTask() { @Override public void run() { if (pingFuts.remove(nodeId, finalFut)) { if (ClientImpl.this.state == DISCONNECTED) @@ -376,7 +376,13 @@ else if (state == DISCONNECTED) { finalFut.onDone(false); } } - }, spi.netTimeout); + }; + + try { + timer.schedule(task, spi.netTimeout); + } catch (IllegalStateException e) { + return false; // timer is cancelled because this client node is dying + } sockWriter.sendMessage(new TcpDiscoveryClientPingRequest(getLocalNodeId(), nodeId)); } @@ -803,7 +809,12 @@ private NavigableSet allVisibleNodes() { U.warn(log, "Simulating client node failure: " + getLocalNodeId()); U.interrupt(sockWriter); + U.interrupt(sockReader); U.interrupt(msgWorker); + try { + timer.cancel(); + } catch (Throwable e) { + } U.join(sockWriter, log); U.join( @@ -870,6 +881,10 @@ private class HeartbeatSender extends TimerTask { msg.client(true); sockWriter.sendMessage(msg); + + if (log.isDebugEnabled()) + log.debug("*** Send heartbeat message from node: "+msg.creatorNodeId()); + } } } 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..2769fae3cc5ef 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 @@ -6088,9 +6088,11 @@ private void processClientHeartbeatMessage(TcpDiscoveryClientHeartbeatMessage ms assert msg.client(); ClientMessageWorker wrk = clientMsgWorkers.get(msg.creatorNodeId()); + if (wrk != null) { + msg.verify(getLocalNodeId()); - if (wrk != null) - wrk.metrics(msg.metrics()); + wrk.addMessage(msg); + } else if (log.isDebugEnabled()) log.debug("Received heartbeat message from unknown client node: " + msg); } @@ -6223,6 +6225,9 @@ private class StatisticsPrinter extends IgniteSpiThread { /** */ private class ClientMessageWorker extends MessageWorkerAdapter> { + /** minimal period of time which can be used as heartbeat timeout */ + public static final int minHeartbeaTimeout = 10; // ms + /** Node ID. */ private final UUID clientNodeId; @@ -6238,15 +6243,25 @@ private class ClientMessageWorker extends MessageWorkerAdapter= heartBeatTimeOut) { + if (log.isInfoEnabled()) + log.info("### No heartbeat message from node:" + clientNodeId + "; timeOut=" + heartBeatTimeOut + "; period=" + period); + + TcpDiscoveryAbstractMessage msg = new TcpDiscoveryNodeLeftMessage(clientNodeId); + + msg.senderNodeId(getLocalNodeId()); + + msgWorker.addMessage(msg); + + clientMsgWorkers.remove(clientNodeId, this); + + U.interrupt(this); + + U.closeQuiet(sock); + } + } + /** {@inheritDoc} */ @Override protected void processMessage(T2 msgT) { boolean success = false; @@ -6333,6 +6375,16 @@ else if (msgLog.isDebugEnabled()) spi.failureDetectionTimeout() : spi.getSocketTimeout()); } } + else if (msg instanceof TcpDiscoveryClientHeartbeatMessage) { + TcpDiscoveryClientHeartbeatMessage hbmsg = (TcpDiscoveryClientHeartbeatMessage)msg; + + if (log.isDebugEnabled()) // TODO turn to debug + log.debug("### Received heartbeat message from node:" + hbmsg.creatorNodeId()); + + this.metrics = hbmsg.metrics(); + + this.lastHeartBeatTime=U.currentTimeMillis(); + } else { if (msgLog.isDebugEnabled()) msgLog.debug("Redirecting message to client [sock=" + sock + ", locNodeId=" From 95d55954de4d60cb1f1f46f19f1aa0b8a9821f16 Mon Sep 17 00:00:00 2001 From: Evgenii Zhuravlev Date: Thu, 1 Jun 2017 19:56:34 +0300 Subject: [PATCH 103/357] SSL fix --- .../ignite/spi/communication/tcp/TcpCommunicationSpi.java | 4 ++-- 1 file changed, 2 insertions(+), 2 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 d50bc425e4569..c319848f1b9a7 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 @@ -3198,10 +3198,10 @@ else if (log.isDebugEnabled()) assert sslHnd != null; buf = ByteBuffer.allocate(1000); + buf.order(ByteOrder.nativeOrder()); ByteBuffer decode = ByteBuffer.allocate(2 * buf.capacity()); - - buf.order(ByteOrder.nativeOrder()); + decode.order(ByteOrder.nativeOrder()); for (int i = 0; i < 9; ) { int read = ch.read(buf); From 5f9dc362f82bc6351d01a77418e42c01db9391cf Mon Sep 17 00:00:00 2001 From: rfqu Date: Fri, 2 Jun 2017 12:11:40 +0300 Subject: [PATCH 104/357] code style fixed --- .../java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java | 1 - .../java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java | 5 +++-- 2 files changed, 3 insertions(+), 3 deletions(-) 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 2412624e900fe..b83a3809aa26d 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 @@ -884,7 +884,6 @@ private class HeartbeatSender extends TimerTask { if (log.isDebugEnabled()) log.debug("*** Send heartbeat message from node: "+msg.creatorNodeId()); - } } } 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 2769fae3cc5ef..178d22b577fe3 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 @@ -6243,7 +6243,8 @@ private class ClientMessageWorker extends MessageWorkerAdapter= heartBeatTimeOut) { if (log.isInfoEnabled()) - log.info("### No heartbeat message from node:" + clientNodeId + "; timeOut=" + heartBeatTimeOut + "; period=" + period); + log.info("Heartbeat timeout for node:" + clientNodeId + "; timeOut=" + heartBeatTimeOut + "; period=" + period); TcpDiscoveryAbstractMessage msg = new TcpDiscoveryNodeLeftMessage(clientNodeId); From 05c639f041ffbbc53678addaf46fc806fb7c168c Mon Sep 17 00:00:00 2001 From: rfqu Date: Fri, 2 Jun 2017 12:25:53 +0300 Subject: [PATCH 105/357] coding style fixed --- .../java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java | 1 + .../java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java | 1 - 2 files changed, 1 insertion(+), 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 b83a3809aa26d..9225a922b5dd3 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 @@ -814,6 +814,7 @@ private NavigableSet allVisibleNodes() { try { timer.cancel(); } catch (Throwable e) { + // No-op. } U.join(sockWriter, log); 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 178d22b577fe3..c7e6d9619ed54 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 @@ -6383,7 +6383,6 @@ else if (msg instanceof TcpDiscoveryClientHeartbeatMessage) { log.debug("### Received heartbeat message from node:" + hbmsg.creatorNodeId()); this.metrics = hbmsg.metrics(); - this.lastHeartBeatTime=U.currentTimeMillis(); } else { From 4d2c9ef1cd50be0a73cd8ac4a0edc809631b23a2 Mon Sep 17 00:00:00 2001 From: rfqu Date: Fri, 2 Jun 2017 13:49:31 +0300 Subject: [PATCH 106/357] test added, taken from tiket IGNITE-5103 --- .../ignite/spi/discovery/tcp/ServerImpl.java | 3 +- .../TcpDiscoveryClientSuspensionSelfTest.java | 116 ++++++++++++++++++ 2 files changed, 117 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 c7e6d9619ed54..318cc23f16101 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 @@ -6316,8 +6316,7 @@ void addMessage(TcpDiscoveryAbstractMessage msg, @Nullable byte[] msgBytes) { * Check the last time a heartbeat message received. * In case of timeout, expel client node from the topology */ - @Override - protected void noMessageLoop() { + @Override protected void noMessageLoop() { long period = U.currentTimeMillis() - lastHeartBeatTime; if (period >= heartBeatTimeOut) { 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..9dc6e506cc74a --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryClientSuspensionSelfTest.java @@ -0,0 +1,116 @@ +/* + * 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.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 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"); + + impl.simulateNodeFailure(); + + //Timer timer = U.field(impl, "timer"); timer.cancel(); -- client node successfully reconnects + } +} From 744a81ba937ba83ecdefa7c71f198d92d21527bb Mon Sep 17 00:00:00 2001 From: Anton Vinogradov Date: Wed, 31 May 2017 15:27:33 +0300 Subject: [PATCH 107/357] 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 108/357] 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 374cba8a2b0d4438b46258a4ea89e43ab1e7989c Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Tue, 6 Jun 2017 16:17:01 +0300 Subject: [PATCH 109/357] IGNITE-5259 Minor serialization fix --- .../discovery/GridDiscoveryManager.java | 6 ++--- .../processors/security/SecurityUtils.java | 23 +++++++++++++++++-- .../ignite/spi/discovery/tcp/ClientImpl.java | 17 +++++++++++++- 3 files changed, 40 insertions(+), 6 deletions(-) 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 dfd4c87ec9cae..c27f2a35b92c7 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 @@ -138,8 +138,8 @@ 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.internal.processors.security.SecurityUtils.isServiceSecuritySupported; import static org.apache.ignite.plugin.segmentation.SegmentationPolicy.NOOP; /** @@ -1163,7 +1163,7 @@ else if (Boolean.FALSE.equals(locSrvcCompatibilityEnabled)) { ", locNodeId=" + locNode.id() + ", rmtNodeId=" + n.id() + ']'); } - if (n.version().compareToIgnoreTimestamp(SERVICE_PERMISSIONS_SINCE) >= 0 + if (isServiceSecuritySupported(n.version()) && ctx.security().enabled() // Matters only if security enabled. ) { Boolean rmtSecurityCompatibilityEnabled = n.attribute(ATTR_SECURITY_COMPATIBILITY_MODE); @@ -1181,7 +1181,7 @@ else if (Boolean.FALSE.equals(locSrvcCompatibilityEnabled)) { } } - if (n.version().compareToIgnoreTimestamp(SERVICE_PERMISSIONS_SINCE) < 0 + if (!isServiceSecuritySupported(n.version()) && ctx.security().enabled() // Matters only if security enabled. && (locSecurityCompatibilityEnabled == null || !locSecurityCompatibilityEnabled)) { throw new IgniteCheckedException("Remote node does not support service security permissions. " + 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 index 1016335888b3d..c84e53f729b66 100644 --- 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 @@ -29,8 +29,12 @@ * Security utilities. */ public class SecurityUtils { - /** Version since service security supported. */ - public static final IgniteProductVersion SERVICE_PERMISSIONS_SINCE = IgniteProductVersion.fromString("1.7.11"); + /** Versions since service security supported. */ + private static final IgniteProductVersion[] SERVICE_PERMISSIONS_SINCE = { + IgniteProductVersion.fromString("1.7.11"), + IgniteProductVersion.fromString("1.8.7"), + IgniteProductVersion.fromString("1.9.3") + }; /** Default serialization version. */ private final static int DFLT_SERIALIZE_VERSION = isSecurityCompatibilityMode() ? 1 : 2; @@ -89,4 +93,19 @@ public static Map> compatibleServicePermi return srvcPerms; } + + /** + * Checks whether provided release supports service security permissions. + * + * @param ver Version to ckeck. + * @return {@code True} if passed release supports service security permissions. + */ + public static boolean isServiceSecuritySupported(IgniteProductVersion ver) { + for (IgniteProductVersion v : SERVICE_PERMISSIONS_SINCE) { + if (v.major() == ver.major() && v.minor() == ver.minor()) + return ver.compareToIgnoreTimestamp(v) >= 0; + } + + return ver.compareToIgnoreTimestamp(SERVICE_PERMISSIONS_SINCE[SERVICE_PERMISSIONS_SINCE.length]) >= 0; + } } 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 a5a4e88990c4a..c73fa3987ece3 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(); @@ -975,6 +985,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())); } @@ -1000,6 +1012,9 @@ private void forceStopRead() throws InterruptedException { continue; } + finally { + SecurityUtils.restoreDefaultSerializeVersion(); + } msg.senderNodeId(rmtNodeId); From 5cb580ad7043f27e4a0396aea1f877c21d49078e Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Tue, 6 Jun 2017 16:17:01 +0300 Subject: [PATCH 110/357] IGNITE-5259 Minor serialization fix (cherry picked from commit 374cba8) --- .../discovery/GridDiscoveryManager.java | 6 ++--- .../processors/security/SecurityUtils.java | 22 +++++++++++++++++-- .../ignite/spi/discovery/tcp/ClientImpl.java | 17 +++++++++++++- 3 files changed, 39 insertions(+), 6 deletions(-) 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 fb72989eb81ba..71b7217bd6afc 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 @@ -138,8 +138,8 @@ 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.internal.processors.security.SecurityUtils.isServiceSecuritySupported; import static org.apache.ignite.plugin.segmentation.SegmentationPolicy.NOOP; /** @@ -1163,7 +1163,7 @@ else if (Boolean.FALSE.equals(locSrvcCompatibilityEnabled)) { ", locNodeId=" + locNode.id() + ", rmtNodeId=" + n.id() + ']'); } - if (n.version().compareToIgnoreTimestamp(SERVICE_PERMISSIONS_SINCE) >= 0 + if (isServiceSecuritySupported(n.version()) && ctx.security().enabled() // Matters only if security enabled. ) { Boolean rmtSecurityCompatibilityEnabled = n.attribute(ATTR_SECURITY_COMPATIBILITY_MODE); @@ -1181,7 +1181,7 @@ else if (Boolean.FALSE.equals(locSrvcCompatibilityEnabled)) { } } - if (n.version().compareToIgnoreTimestamp(SERVICE_PERMISSIONS_SINCE) < 0 + if (!isServiceSecuritySupported(n.version()) && ctx.security().enabled() // Matters only if security enabled. && (locSecurityCompatibilityEnabled == null || !locSecurityCompatibilityEnabled)) { throw new IgniteCheckedException("Remote node does not support service security permissions. " + 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 index 1016335888b3d..306d8de58d87f 100644 --- 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 @@ -29,8 +29,11 @@ * Security utilities. */ public class SecurityUtils { - /** Version since service security supported. */ - public static final IgniteProductVersion SERVICE_PERMISSIONS_SINCE = IgniteProductVersion.fromString("1.7.11"); + /** Versions since service security supported. */ + private static final IgniteProductVersion[] SERVICE_PERMISSIONS_SINCE = { + IgniteProductVersion.fromString("1.7.11"), + IgniteProductVersion.fromString("1.8.7") + }; /** Default serialization version. */ private final static int DFLT_SERIALIZE_VERSION = isSecurityCompatibilityMode() ? 1 : 2; @@ -89,4 +92,19 @@ public static Map> compatibleServicePermi return srvcPerms; } + + /** + * Checks whether provided release supports service security permissions. + * + * @param ver Version to ckeck. + * @return {@code True} if passed release supports service security permissions. + */ + public static boolean isServiceSecuritySupported(IgniteProductVersion ver) { + for (IgniteProductVersion v : SERVICE_PERMISSIONS_SINCE) { + if (v.major() == ver.major() && v.minor() == ver.minor()) + return ver.compareToIgnoreTimestamp(v) >= 0; + } + + return ver.compareToIgnoreTimestamp(SERVICE_PERMISSIONS_SINCE[SERVICE_PERMISSIONS_SINCE.length]) >= 0; + } } 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 9225a922b5dd3..e7bc81e5bc55f 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; @@ -522,7 +523,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(); @@ -984,6 +994,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())); } @@ -1009,6 +1021,9 @@ private void forceStopRead() throws InterruptedException { continue; } + finally { + SecurityUtils.restoreDefaultSerializeVersion(); + } msg.senderNodeId(rmtNodeId); From f03252f9b2c6f0e777f307fd85cc8bd20ab21423 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Tue, 6 Jun 2017 16:17:01 +0300 Subject: [PATCH 111/357] 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 d2bf9619aaf867f251bc193d913dd4cc174a33a3 Mon Sep 17 00:00:00 2001 From: Ivan Veselovskiy Date: Tue, 6 Jun 2017 16:56:09 +0300 Subject: [PATCH 112/357] IGNITE-5410: Fixed assertion in HadoopDataOutStream. This closes #2084. --- .../processors/hadoop/shuffle/streams/HadoopOffheapBuffer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/HadoopOffheapBuffer.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/HadoopOffheapBuffer.java index acc9be62cf677..655346eaae93a 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/HadoopOffheapBuffer.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/HadoopOffheapBuffer.java @@ -92,7 +92,7 @@ public void pointer(long ptr) { * @return Old position pointer or {@code 0} if move goes beyond the end of the buffer. */ public long move(long size) { - assert size > 0 : size; + assert size >= 0 : size; long oldPos = posPtr; long newPos = oldPos + size; From 77ff30cc08dae653c0b914167088e9e90cdadd32 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Tue, 6 Jun 2017 17:12:27 +0300 Subject: [PATCH 113/357] IGNITE-5259 Minor serialization fix --- .../ignite/internal/processors/security/SecurityUtils.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 index c84e53f729b66..3d8b8ccad2c92 100644 --- 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 @@ -106,6 +106,6 @@ public static boolean isServiceSecuritySupported(IgniteProductVersion ver) { return ver.compareToIgnoreTimestamp(v) >= 0; } - return ver.compareToIgnoreTimestamp(SERVICE_PERMISSIONS_SINCE[SERVICE_PERMISSIONS_SINCE.length]) >= 0; + return ver.compareToIgnoreTimestamp(SERVICE_PERMISSIONS_SINCE[SERVICE_PERMISSIONS_SINCE.length - 1]) >= 0; } } From 3a1d560cd8741de9e7a6dd1110b42814d0ccff6b Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Tue, 6 Jun 2017 17:13:52 +0300 Subject: [PATCH 114/357] IGNITE-5259 Minor serialization fix --- .../ignite/internal/processors/security/SecurityUtils.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 index 306d8de58d87f..ca256c922f545 100644 --- 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 @@ -105,6 +105,6 @@ public static boolean isServiceSecuritySupported(IgniteProductVersion ver) { return ver.compareToIgnoreTimestamp(v) >= 0; } - return ver.compareToIgnoreTimestamp(SERVICE_PERMISSIONS_SINCE[SERVICE_PERMISSIONS_SINCE.length]) >= 0; + return ver.compareToIgnoreTimestamp(SERVICE_PERMISSIONS_SINCE[SERVICE_PERMISSIONS_SINCE.length - 1]) >= 0; } } From 56d4ce8a042238654ab96235d1a2969107b8881c Mon Sep 17 00:00:00 2001 From: devozerov Date: Tue, 6 Jun 2017 17:39:33 +0300 Subject: [PATCH 115/357] GG-12244: Fixed a bug in GridH2IndexRangeRequest serialization mechanics. --- .../query/h2/twostep/msg/GridH2IndexRangeRequest.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeRequest.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeRequest.java index b2548cc27a401..be188b04dd2a9 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeRequest.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeRequest.java @@ -174,6 +174,8 @@ public int batchLookupId() { if (!writer.writeInt("segmentId", segmentId)) return false; + writer.incrementState(); + case 5: if (!writer.writeInt("originSegId", originSegmentId)) return false; From cfbe8da934741e76c8964af87671a38ec7b6c9a3 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Tue, 6 Jun 2017 19:15:59 +0300 Subject: [PATCH 116/357] IGNITE-5103 Rolled back due to test failings. --- .../ignite/spi/discovery/tcp/ClientImpl.java | 19 +-- .../ignite/spi/discovery/tcp/ServerImpl.java | 57 +-------- .../TcpDiscoveryClientSuspensionSelfTest.java | 116 ------------------ 3 files changed, 5 insertions(+), 187 deletions(-) delete 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/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java index e7bc81e5bc55f..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 @@ -367,7 +367,7 @@ else if (state == DISCONNECTED) { else { final GridFutureAdapter finalFut = fut; - TimerTask task = new TimerTask() { + timer.schedule(new TimerTask() { @Override public void run() { if (pingFuts.remove(nodeId, finalFut)) { if (ClientImpl.this.state == DISCONNECTED) @@ -377,13 +377,7 @@ else if (state == DISCONNECTED) { finalFut.onDone(false); } } - }; - - try { - timer.schedule(task, spi.netTimeout); - } catch (IllegalStateException e) { - return false; // timer is cancelled because this client node is dying - } + }, spi.netTimeout); sockWriter.sendMessage(new TcpDiscoveryClientPingRequest(getLocalNodeId(), nodeId)); } @@ -819,13 +813,7 @@ private NavigableSet allVisibleNodes() { U.warn(log, "Simulating client node failure: " + getLocalNodeId()); U.interrupt(sockWriter); - U.interrupt(sockReader); U.interrupt(msgWorker); - try { - timer.cancel(); - } catch (Throwable e) { - // No-op. - } U.join(sockWriter, log); U.join( @@ -892,9 +880,6 @@ private class HeartbeatSender extends TimerTask { msg.client(true); sockWriter.sendMessage(msg); - - if (log.isDebugEnabled()) - log.debug("*** Send heartbeat message from node: "+msg.creatorNodeId()); } } } 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 a8feab41abd9a..c91954cb18c92 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 @@ -6165,11 +6165,9 @@ private void processClientHeartbeatMessage(TcpDiscoveryClientHeartbeatMessage ms assert msg.client(); ClientMessageWorker wrk = clientMsgWorkers.get(msg.creatorNodeId()); - if (wrk != null) { - msg.verify(getLocalNodeId()); - wrk.addMessage(msg); - } + if (wrk != null) + wrk.metrics(msg.metrics()); else if (log.isDebugEnabled()) log.debug("Received heartbeat message from unknown client node: " + msg); } @@ -6302,9 +6300,6 @@ private class StatisticsPrinter extends IgniteSpiThread { /** */ private class ClientMessageWorker extends MessageWorkerAdapter> { - /** minimal period of time which can be used as heartbeat timeout */ - public static final int minHeartbeaTimeout = 10; // ms - /** Node ID. */ private final UUID clientNodeId; @@ -6320,26 +6315,15 @@ private class ClientMessageWorker extends MessageWorkerAdapter= heartBeatTimeOut) { - if (log.isInfoEnabled()) - log.info("Heartbeat timeout for node:" + clientNodeId + "; timeOut=" + heartBeatTimeOut + "; period=" + period); - - TcpDiscoveryAbstractMessage msg = new TcpDiscoveryNodeLeftMessage(clientNodeId); - - msg.senderNodeId(getLocalNodeId()); - - msgWorker.addMessage(msg); - - clientMsgWorkers.remove(clientNodeId, this); - - U.interrupt(this); - - U.closeQuiet(sock); - } - } - /** {@inheritDoc} */ @Override protected void processMessage(T2 msgT) { boolean success = false; @@ -6452,15 +6410,6 @@ else if (msgLog.isDebugEnabled()) spi.failureDetectionTimeout() : spi.getSocketTimeout()); } } - else if (msg instanceof TcpDiscoveryClientHeartbeatMessage) { - TcpDiscoveryClientHeartbeatMessage hbmsg = (TcpDiscoveryClientHeartbeatMessage)msg; - - if (log.isDebugEnabled()) // TODO turn to debug - log.debug("### Received heartbeat message from node:" + hbmsg.creatorNodeId()); - - this.metrics = hbmsg.metrics(); - this.lastHeartBeatTime=U.currentTimeMillis(); - } else { if (msgLog.isDebugEnabled()) msgLog.debug("Redirecting message to client [sock=" + sock + ", locNodeId=" 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 deleted file mode 100644 index 9dc6e506cc74a..0000000000000 --- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryClientSuspensionSelfTest.java +++ /dev/null @@ -1,116 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.spi.discovery.tcp; - -import java.util.Timer; -import org.apache.ignite.Ignite; -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 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"); - - impl.simulateNodeFailure(); - - //Timer timer = U.field(impl, "timer"); timer.cancel(); -- client node successfully reconnects - } -} From 83307da08289c873c5c2eb02d5eb314018bc5c13 Mon Sep 17 00:00:00 2001 From: Ivan Veselovskiy Date: Tue, 6 Jun 2017 16:56:09 +0300 Subject: [PATCH 117/357] IGNITE-5410: Fixed assertion in HadoopDataOutStream. This closes #2084. (cherry picked from commit d2bf961) --- .../processors/hadoop/shuffle/streams/HadoopOffheapBuffer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/HadoopOffheapBuffer.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/HadoopOffheapBuffer.java index acc9be62cf677..655346eaae93a 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/HadoopOffheapBuffer.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/HadoopOffheapBuffer.java @@ -92,7 +92,7 @@ public void pointer(long ptr) { * @return Old position pointer or {@code 0} if move goes beyond the end of the buffer. */ public long move(long size) { - assert size > 0 : size; + assert size >= 0 : size; long oldPos = posPtr; long newPos = oldPos + size; From e95626d609ee225918b49653b7981b180e5d4e49 Mon Sep 17 00:00:00 2001 From: Evgenii Zhuravlev Date: Thu, 1 Jun 2017 19:56:34 +0300 Subject: [PATCH 118/357] SSL fix (cherry picked from commit 95d5595) --- .../ignite/spi/communication/tcp/TcpCommunicationSpi.java | 4 ++-- 1 file changed, 2 insertions(+), 2 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 60d97348d42cf..dbad401e9d037 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 @@ -3198,10 +3198,10 @@ else if (log.isDebugEnabled()) assert sslHnd != null; buf = ByteBuffer.allocate(1000); + buf.order(ByteOrder.nativeOrder()); ByteBuffer decode = ByteBuffer.allocate(2 * buf.capacity()); - - buf.order(ByteOrder.nativeOrder()); + decode.order(ByteOrder.nativeOrder()); for (int i = 0; i < 9; ) { int read = ch.read(buf); From 340204637a03e5533685f1b11ca65c9121f6e193 Mon Sep 17 00:00:00 2001 From: Alexei Kaigorodov Date: Thu, 8 Jun 2017 19:37:40 +0300 Subject: [PATCH 119/357] IGNITE-5103 Rolled back due to test failings. (#69) --- .../ignite/spi/discovery/tcp/ClientImpl.java | 19 +-- .../ignite/spi/discovery/tcp/ServerImpl.java | 57 +-------- .../TcpDiscoveryClientSuspensionSelfTest.java | 116 ------------------ 3 files changed, 5 insertions(+), 187 deletions(-) delete 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/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java index f4262f197cb63..a5a4e88990c4a 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 @@ -366,7 +366,7 @@ else if (state == DISCONNECTED) { else { final GridFutureAdapter finalFut = fut; - TimerTask task = new TimerTask() { + timer.schedule(new TimerTask() { @Override public void run() { if (pingFuts.remove(nodeId, finalFut)) { if (ClientImpl.this.state == DISCONNECTED) @@ -376,13 +376,7 @@ else if (state == DISCONNECTED) { finalFut.onDone(false); } } - }; - - try { - timer.schedule(task, spi.netTimeout); - } catch (IllegalStateException e) { - return false; // timer is cancelled because this client node is dying - } + }, spi.netTimeout); sockWriter.sendMessage(new TcpDiscoveryClientPingRequest(getLocalNodeId(), nodeId)); } @@ -815,13 +809,7 @@ private NavigableSet allVisibleNodes() { U.warn(log, "Simulating client node failure: " + getLocalNodeId()); U.interrupt(sockWriter); - U.interrupt(sockReader); U.interrupt(msgWorker); - try { - timer.cancel(); - } catch (Throwable e) { - // No-op. - } U.join(sockWriter, log); U.join( @@ -888,9 +876,6 @@ private class HeartbeatSender extends TimerTask { msg.client(true); sockWriter.sendMessage(msg); - - if (log.isDebugEnabled()) - log.debug("*** Send heartbeat message from node: "+msg.creatorNodeId()); } } } 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 211265b712524..eef8f45a3064e 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 @@ -6164,11 +6164,9 @@ private void processClientHeartbeatMessage(TcpDiscoveryClientHeartbeatMessage ms assert msg.client(); ClientMessageWorker wrk = clientMsgWorkers.get(msg.creatorNodeId()); - if (wrk != null) { - msg.verify(getLocalNodeId()); - wrk.addMessage(msg); - } + if (wrk != null) + wrk.metrics(msg.metrics()); else if (log.isDebugEnabled()) log.debug("Received heartbeat message from unknown client node: " + msg); } @@ -6301,9 +6299,6 @@ private class StatisticsPrinter extends IgniteSpiThread { /** */ private class ClientMessageWorker extends MessageWorkerAdapter> { - /** minimal period of time which can be used as heartbeat timeout */ - public static final int minHeartbeaTimeout = 10; // ms - /** Node ID. */ private final UUID clientNodeId; @@ -6319,26 +6314,15 @@ private class ClientMessageWorker extends MessageWorkerAdapter= heartBeatTimeOut) { - if (log.isInfoEnabled()) - log.info("Heartbeat timeout for node:" + clientNodeId + "; timeOut=" + heartBeatTimeOut + "; period=" + period); - - TcpDiscoveryAbstractMessage msg = new TcpDiscoveryNodeLeftMessage(clientNodeId); - - msg.senderNodeId(getLocalNodeId()); - - msgWorker.addMessage(msg); - - clientMsgWorkers.remove(clientNodeId, this); - - U.interrupt(this); - - U.closeQuiet(sock); - } - } - /** {@inheritDoc} */ @Override protected void processMessage(T2 msgT) { boolean success = false; @@ -6451,15 +6409,6 @@ else if (msgLog.isDebugEnabled()) spi.failureDetectionTimeout() : spi.getSocketTimeout()); } } - else if (msg instanceof TcpDiscoveryClientHeartbeatMessage) { - TcpDiscoveryClientHeartbeatMessage hbmsg = (TcpDiscoveryClientHeartbeatMessage)msg; - - if (log.isDebugEnabled()) // TODO turn to debug - log.debug("### Received heartbeat message from node:" + hbmsg.creatorNodeId()); - - this.metrics = hbmsg.metrics(); - this.lastHeartBeatTime=U.currentTimeMillis(); - } else { if (msgLog.isDebugEnabled()) msgLog.debug("Redirecting message to client [sock=" + sock + ", locNodeId=" 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 deleted file mode 100644 index 9dc6e506cc74a..0000000000000 --- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryClientSuspensionSelfTest.java +++ /dev/null @@ -1,116 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.spi.discovery.tcp; - -import java.util.Timer; -import org.apache.ignite.Ignite; -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 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"); - - impl.simulateNodeFailure(); - - //Timer timer = U.field(impl, "timer"); timer.cancel(); -- client node successfully reconnects - } -} From f3f726e9059e492573dc5125fd5edb5d2f71e9d3 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Tue, 13 Jun 2017 14:11:17 +0300 Subject: [PATCH 120/357] 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 | 7 ++++++- 2 files changed, 14 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 e933d83dda840..88aeae6bfccd5 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -326,6 +326,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 8c5c2a364cabf..66fb7ae94f5be 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 @@ -176,6 +176,7 @@ 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; @@ -1797,11 +1798,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 c2c237d1222557d3e6b35d9a51a61a4c78e56782 Mon Sep 17 00:00:00 2001 From: Sergey Kalashnikov Date: Fri, 3 Feb 2017 11:41:14 +0300 Subject: [PATCH 121/357] 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 4a8f295f2f2f34e8472b1d1320f03744135b2504 Mon Sep 17 00:00:00 2001 From: Igor Sapego Date: Tue, 13 Jun 2017 19:47:00 +0300 Subject: [PATCH 122/357] IGNITE-5478: ODBC: SQLNumParams now returns number of required parameters. (cherry picked from commit b1c56a1) --- .../cpp/odbc-test/src/queries_test.cpp | 30 +++++++++++++++++ .../cpp/odbc/include/ignite/odbc/statement.h | 13 ++++++-- modules/platforms/cpp/odbc/src/odbc.cpp | 7 +++- modules/platforms/cpp/odbc/src/statement.cpp | 33 +++++++++++++++++-- 4 files changed, 76 insertions(+), 7 deletions(-) diff --git a/modules/platforms/cpp/odbc-test/src/queries_test.cpp b/modules/platforms/cpp/odbc-test/src/queries_test.cpp index 52b885d1d2298..6dc5d4f63cf3c 100644 --- a/modules/platforms/cpp/odbc-test/src/queries_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/queries_test.cpp @@ -273,6 +273,25 @@ struct QueriesTestSuiteFixture BOOST_CHECK(ret == SQL_NO_DATA); } + void CheckParamsNum(const std::string& req, SQLSMALLINT expectedParamsNum) + { + std::vector req0(req.begin(), req.end()); + + SQLRETURN ret = SQLPrepare(stmt, &req0[0], static_cast(req0.size())); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + SQLSMALLINT paramsNum = -1; + + ret = SQLNumParams(stmt, ¶msNum); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECK_EQUAL(paramsNum, expectedParamsNum); + } + int CountRows(SQLHSTMT stmt) { int res = 0; @@ -1342,4 +1361,15 @@ BOOST_AUTO_TEST_CASE(TestInsertMergeSelect) BOOST_CHECK_EQUAL(recordsNum, selectedRecordsNum); } +BOOST_AUTO_TEST_CASE(TestParamsNum) +{ + Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;SCHEMA=cache"); + + CheckParamsNum("SELECT * FROM TestType", 0); + CheckParamsNum("SELECT * FROM TestType WHERE _key=?", 1); + CheckParamsNum("SELECT * FROM TestType WHERE _key=? AND _val=?", 2); + CheckParamsNum("INSERT INTO TestType(_key, strField) VALUES(1, 'some')", 0); + CheckParamsNum("INSERT INTO TestType(_key, strField) VALUES(?, ?)", 2); +} + BOOST_AUTO_TEST_SUITE_END() diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h b/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h index db566609d1b86..1ee56197ce81b 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h @@ -137,11 +137,11 @@ namespace ignite void GetAttribute(int attr, void* buf, SQLINTEGER bufLen, SQLINTEGER *valueLen); /** - * Get number of binded parameters. + * Get number parameters required by the prepared statement. * - * @return Number of binded parameters. + * @param paramNum Number of parameters. */ - uint16_t GetParametersNumber(); + void GetParametersNumber(uint16_t& paramNum); /** * Set parameter binding offset pointer. @@ -394,6 +394,13 @@ namespace ignite */ SqlResult InternalGetAttribute(int attr, void* buf, SQLINTEGER bufLen, SQLINTEGER* valueLen); + /** + * Get number parameters required by the prepared statement. + * + * @param paramNum Number of parameters. + */ + SqlResult InternalGetParametersNumber(uint16_t& paramNum); + /** * Get value of the column in the result set. * diff --git a/modules/platforms/cpp/odbc/src/odbc.cpp b/modules/platforms/cpp/odbc/src/odbc.cpp index 0b18a116f4993..caf31d91860b1 100644 --- a/modules/platforms/cpp/odbc/src/odbc.cpp +++ b/modules/platforms/cpp/odbc/src/odbc.cpp @@ -929,7 +929,12 @@ namespace ignite return SQL_INVALID_HANDLE; if (paramCnt) - *paramCnt = static_cast(statement->GetParametersNumber()); + { + uint16_t paramNum = 0; + statement->GetParametersNumber(paramNum); + + *paramCnt = static_cast(paramNum); + } return statement->GetDiagnosticRecords().GetReturnCode(); } diff --git a/modules/platforms/cpp/odbc/src/statement.cpp b/modules/platforms/cpp/odbc/src/statement.cpp index 02c6dd972154b..9aca8c9164209 100644 --- a/modules/platforms/cpp/odbc/src/statement.cpp +++ b/modules/platforms/cpp/odbc/src/statement.cpp @@ -296,11 +296,38 @@ namespace ignite return SQL_RESULT_SUCCESS; } - uint16_t Statement::GetParametersNumber() + void Statement::GetParametersNumber(uint16_t& paramNum) { - IGNITE_ODBC_API_CALL_ALWAYS_SUCCESS; + IGNITE_ODBC_API_CALL(InternalGetParametersNumber(paramNum)); + } + + SqlResult Statement::InternalGetParametersNumber(uint16_t& paramNum) + { + if (!currentQuery.get()) + { + AddStatusRecord(SQL_STATE_HY010_SEQUENCE_ERROR, "Query is not prepared."); + + return SQL_RESULT_ERROR; + } + + if (currentQuery->GetType() != query::Query::DATA) + { + paramNum = 0; - return static_cast(paramBindings.size()); + return SQL_RESULT_SUCCESS; + } + + if (paramTypes.empty()) + { + SqlResult res = UpdateParamsMeta(); + + if (res != SQL_RESULT_SUCCESS) + return res; + } + + paramNum = static_cast(paramTypes.size()); + + return SQL_RESULT_SUCCESS; } void Statement::SetParamBindOffsetPtr(int* ptr) From a2a4ec1ee9794cb542f146a07c6c67002cad444e Mon Sep 17 00:00:00 2001 From: Igor Sapego Date: Wed, 14 Jun 2017 12:16:43 +0300 Subject: [PATCH 123/357] IGNITE-5478: Fix for cherry pick --- modules/platforms/cpp/odbc-test/src/queries_test.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/platforms/cpp/odbc-test/src/queries_test.cpp b/modules/platforms/cpp/odbc-test/src/queries_test.cpp index 6dc5d4f63cf3c..3a3858802459e 100644 --- a/modules/platforms/cpp/odbc-test/src/queries_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/queries_test.cpp @@ -1363,7 +1363,7 @@ BOOST_AUTO_TEST_CASE(TestInsertMergeSelect) BOOST_AUTO_TEST_CASE(TestParamsNum) { - Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;SCHEMA=cache"); + Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache"); CheckParamsNum("SELECT * FROM TestType", 0); CheckParamsNum("SELECT * FROM TestType WHERE _key=?", 1); From d268b32cb252a5f06887d2b803d27ddc20ded95f Mon Sep 17 00:00:00 2001 From: Igor Sapego Date: Fri, 16 Jun 2017 12:27:35 +0300 Subject: [PATCH 124/357] IGNITE-4370: Implemented writing of batch of parameters for ODBC. (cherry picked from commit c10be5780589cc84e7929e234e4411d515166e0b) --- .../processors/odbc/OdbcMessageParser.java | 57 ++- .../processors/odbc/OdbcNioListener.java | 3 +- .../odbc/OdbcQueryExecuteBatchRequest.java | 95 ++++ .../odbc/OdbcQueryExecuteBatchResult.java | 75 ++++ .../odbc/OdbcQueryExecuteRequest.java | 6 +- .../odbc/OdbcQueryExecuteResult.java | 2 +- .../internal/processors/odbc/OdbcRequest.java | 3 + .../processors/odbc/OdbcRequestHandler.java | 106 ++++- .../cpp/core-test/src/test_utils.cpp | 0 .../src/application_data_buffer_test.cpp | 146 ++----- .../cpp/odbc-test/src/column_test.cpp | 12 +- .../cpp/odbc-test/src/queries_test.cpp | 386 ++++++++++++++++- .../platforms/cpp/odbc-test/src/row_test.cpp | 9 +- modules/platforms/cpp/odbc/Makefile.am | 3 + .../platforms/cpp/odbc/include/Makefile.am | 2 + .../ignite/odbc/app/application_data_buffer.h | 38 +- .../odbc/include/ignite/odbc/app/parameter.h | 20 +- .../include/ignite/odbc/app/parameter_set.h | 268 ++++++++++++ .../odbc/include/ignite/odbc/common_types.h | 8 +- .../ignite/odbc/diagnostic/diagnosable.h | 4 +- .../cpp/odbc/include/ignite/odbc/message.h | 408 +++++++----------- .../include/ignite/odbc/query/batch_query.h | 160 +++++++ .../include/ignite/odbc/query/data_query.h | 20 +- .../odbc/include/ignite/odbc/query/query.h | 3 + .../ignite/odbc/query/type_info_query.h | 2 +- .../cpp/odbc/include/ignite/odbc/statement.h | 59 ++- .../cpp/odbc/project/vs/odbc.vcxproj | 5 + .../cpp/odbc/project/vs/odbc.vcxproj.filters | 15 + .../odbc/src/app/application_data_buffer.cpp | 94 +++- .../platforms/cpp/odbc/src/app/parameter.cpp | 13 +- .../cpp/odbc/src/app/parameter_set.cpp | 242 +++++++++++ .../cpp/odbc/src/config/connection_info.cpp | 68 ++- .../odbc/src/diagnostic/diagnostic_record.cpp | 12 + modules/platforms/cpp/odbc/src/message.cpp | 366 ++++++++++++++++ modules/platforms/cpp/odbc/src/odbc.cpp | 23 +- .../cpp/odbc/src/query/batch_query.cpp | 197 +++++++++ .../cpp/odbc/src/query/data_query.cpp | 15 +- modules/platforms/cpp/odbc/src/statement.cpp | 260 +++++++---- 38 files changed, 2621 insertions(+), 584 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteBatchRequest.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteBatchResult.java create mode 100644 modules/platforms/cpp/core-test/src/test_utils.cpp create mode 100644 modules/platforms/cpp/odbc/include/ignite/odbc/app/parameter_set.h create mode 100644 modules/platforms/cpp/odbc/include/ignite/odbc/query/batch_query.h create mode 100644 modules/platforms/cpp/odbc/src/app/parameter_set.cpp create mode 100644 modules/platforms/cpp/odbc/src/message.cpp create mode 100644 modules/platforms/cpp/odbc/src/query/batch_query.cpp diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcMessageParser.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcMessageParser.java index 7b863d6b2af21..0cb89d71f626b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcMessageParser.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcMessageParser.java @@ -29,6 +29,7 @@ import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl; import java.util.Collection; +import org.jetbrains.annotations.NotNull; /** * ODBC message parser. @@ -99,23 +100,37 @@ public OdbcRequest decode(byte[] msg) { } OdbcRequest res; - + switch (cmd) { case OdbcRequest.EXECUTE_SQL_QUERY: { String cache = reader.readString(); String sql = reader.readString(); - int argsNum = reader.readInt(); - - Object[] params = new Object[argsNum]; + int paramNum = reader.readInt(); - for (int i = 0; i < argsNum; ++i) - params[i] = reader.readObjectDetached(); + Object[] params = readParameterRow(reader, paramNum); res = new OdbcQueryExecuteRequest(cache, sql, params); break; } + case OdbcRequest.QRY_EXEC_BATCH: { + String schema = reader.readString(); + String sql = reader.readString(); + int paramRowLen = reader.readInt(); + int rowNum = reader.readInt(); + boolean last = reader.readBoolean(); + + Object[][] params = new Object[rowNum][]; + + for (int i = 0; i < rowNum; ++i) + params[i] = readParameterRow(reader, paramRowLen); + + res = new OdbcQueryExecuteBatchRequest(schema, sql, last, params); + + break; + } + case OdbcRequest.FETCH_SQL_QUERY: { long queryId = reader.readLong(); int pageSize = reader.readInt(); @@ -170,6 +185,21 @@ public OdbcRequest decode(byte[] msg) { return res; } + /** + * Read row of parameters using reader. + * @param reader reader + * @param paramNum Number of parameters in a row + * @return Parameters array. + */ + @NotNull private static Object[] readParameterRow(BinaryReaderExImpl reader, int paramNum) { + Object[] params = new Object[paramNum]; + + for (int i = 0; i < paramNum; ++i) + params[i] = reader.readObjectDetached(); + + return params; + } + /** * Encode OdbcResponse to byte array. * @@ -218,9 +248,9 @@ else if (res0 instanceof OdbcQueryExecuteResult) { OdbcQueryExecuteResult res = (OdbcQueryExecuteResult) res0; if (log.isDebugEnabled()) - log.debug("Resulting query ID: " + res.getQueryId()); + log.debug("Resulting query ID: " + res.queryId()); - writer.writeLong(res.getQueryId()); + writer.writeLong(res.queryId()); Collection metas = res.getColumnsMetadata(); @@ -231,6 +261,17 @@ else if (res0 instanceof OdbcQueryExecuteResult) { for (OdbcColumnMeta meta : metas) meta.write(writer); } + else if (res0 instanceof OdbcQueryExecuteBatchResult) { + OdbcQueryExecuteBatchResult res = (OdbcQueryExecuteBatchResult) res0; + + writer.writeBoolean(res.errorMessage() == null); + writer.writeLong(res.rowsAffected()); + + if (res.errorMessage() != null) { + writer.writeLong(res.errorSetIdx()); + writer.writeString(res.errorMessage()); + } + } else if (res0 instanceof OdbcQueryFetchResult) { OdbcQueryFetchResult res = (OdbcQueryFetchResult) res0; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcNioListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcNioListener.java index 378e5f25ebebc..a68cf888af48b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcNioListener.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcNioListener.java @@ -23,6 +23,7 @@ import org.apache.ignite.internal.util.nio.GridNioServerListenerAdapter; import org.apache.ignite.internal.util.nio.GridNioSession; import org.apache.ignite.internal.util.nio.GridNioSessionMetaKey; +import org.apache.ignite.internal.util.typedef.internal.U; import org.jetbrains.annotations.Nullable; import java.util.concurrent.atomic.AtomicLong; @@ -100,7 +101,7 @@ public OdbcNioListener(GridKernalContext ctx, GridSpinBusyLock busyLock, int max req = parser.decode(msg); } catch (Exception e) { - log.error("Failed to parse message [id=" + reqId + ", err=" + e + ']'); + U.error(log, "Failed to parse message [id=" + reqId + ']', e); ses.close(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteBatchRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteBatchRequest.java new file mode 100644 index 0000000000000..0ace947126349 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteBatchRequest.java @@ -0,0 +1,95 @@ +package org.apache.ignite.internal.processors.odbc; + +/* + * 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. + */ + +import org.apache.ignite.internal.util.tostring.GridToStringExclude; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.jetbrains.annotations.Nullable; + +/** + * ODBC query execute request with the batch of parameters. + */ +public class OdbcQueryExecuteBatchRequest extends OdbcRequest { + /** Schema. */ + @GridToStringInclude(sensitive = true) + private final String schema; + + /** Sql query. */ + @GridToStringInclude(sensitive = true) + private final String sqlQry; + + /** Last param page flag. */ + private final boolean last; + + /** Sql query arguments. */ + @GridToStringExclude + private final Object[][] args; + + /** + * @param schema Schema. + * @param sqlQry SQL query. + * @param last Last page flag. + * @param args Arguments list. + */ + public OdbcQueryExecuteBatchRequest(@Nullable String schema, String sqlQry, boolean last, Object[][] args) { + super(QRY_EXEC_BATCH); + + assert sqlQry != null : "SQL query should not be null"; + assert args != null : "Parameters should not be null"; + + this.schema = schema; + this.sqlQry = sqlQry; + this.last = last; + this.args = args; + } + + /** + * @return Sql query. + */ + public String sqlQuery() { + return sqlQry; + } + + /** + * @return Sql query arguments. + */ + public Object[][] arguments() { + return args; + } + + /** + * @return Schema. + */ + @Nullable + public String schema() { + return schema; + } + + /** + * @return Last page flag. + */ + public boolean last() { + return last; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(OdbcQueryExecuteBatchRequest.class, this, "args", args, true); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteBatchResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteBatchResult.java new file mode 100644 index 0000000000000..6fc3873dea267 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteBatchResult.java @@ -0,0 +1,75 @@ +/* + * 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.odbc; + +import org.jetbrains.annotations.Nullable; + +/** + * ODBC query execute with batch of parameters result. + */ +public class OdbcQueryExecuteBatchResult { + /** Rows affected. */ + private final long rowsAffected; + + /** Index of the set which caused an error. */ + private final long errorSetIdx; + + /** Error message. */ + private final String errorMessage; + + /** + * @param rowsAffected Number of rows affected by the query. + */ + public OdbcQueryExecuteBatchResult(long rowsAffected) { + this.rowsAffected = rowsAffected; + this.errorSetIdx = -1; + this.errorMessage = null; + } + + /** + * @param rowsAffected Number of rows affected by the query. + * @param errorSetIdx Sets processed. + * @param errorMessage Error message. + */ + public OdbcQueryExecuteBatchResult(long rowsAffected, long errorSetIdx, String errorMessage) { + this.rowsAffected = rowsAffected; + this.errorSetIdx = errorSetIdx; + this.errorMessage = errorMessage; + } + + /** + * @return Number of rows affected by the query. + */ + public long rowsAffected() { + return rowsAffected; + } + + /** + * @return Index of the set which caused an error or -1 if no error occurred. + */ + public long errorSetIdx() { + return errorSetIdx; + } + + /** + * @return Error message. + */ + @Nullable public String errorMessage() { + return errorMessage; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteRequest.java index c0d1c601c92b7..029135cd52e80 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteRequest.java @@ -42,10 +42,12 @@ public class OdbcQueryExecuteRequest extends OdbcRequest { * @param sqlQry SQL query. * @param args Arguments list. */ - public OdbcQueryExecuteRequest(String cacheName, String sqlQry, Object[] args) { + public OdbcQueryExecuteRequest(@Nullable String cacheName, String sqlQry, Object[] args) { super(EXECUTE_SQL_QUERY); - this.cacheName = cacheName.isEmpty() ? null : cacheName; + assert sqlQry != null : "SQL query should not be null"; + + this.cacheName = cacheName; this.sqlQry = sqlQry; this.args = args; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteResult.java index a4038123d2d4d..efa432b615614 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteResult.java @@ -41,7 +41,7 @@ public OdbcQueryExecuteResult(long queryId, Collection columnsMe /** * @return Query ID. */ - public long getQueryId() { + public long queryId() { return queryId; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequest.java index f056fedf079bd..3c0c12e09cd28 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequest.java @@ -42,6 +42,9 @@ public class OdbcRequest { /** Get parameters meta. */ public static final int GET_PARAMS_META = 7; + /** Execute sql query with the batch of parameters. */ + public static final int QRY_EXEC_BATCH = 8; + /** Command. */ private final int cmd; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java index c6b41d2d75d36..06430834b2d8b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java @@ -18,6 +18,8 @@ package org.apache.ignite.internal.processors.odbc; import org.apache.ignite.IgniteCache; + +import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.cache.query.QueryCursor; import org.apache.ignite.cache.query.SqlFieldsQuery; @@ -36,7 +38,11 @@ import java.sql.ParameterMetaData; import java.sql.PreparedStatement; import java.sql.Types; -import java.util.*; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicLong; @@ -104,6 +110,9 @@ public OdbcResponse handle(long reqId, OdbcRequest req) { case HANDSHAKE: return performHandshake(reqId, (OdbcHandshakeRequest)req); + case QRY_EXEC_BATCH: + return executeBatchQuery((OdbcQueryExecuteBatchRequest)req); + case EXECUTE_SQL_QUERY: return executeQuery(reqId, (OdbcQueryExecuteRequest)req); @@ -229,6 +238,101 @@ private OdbcResponse executeQuery(long reqId, OdbcQueryExecuteRequest req) { } } + /** + * {@link OdbcQueryExecuteBatchRequest} command handler. + * + * @param req Execute query request. + * @return Response. + */ + private OdbcResponse executeBatchQuery(OdbcQueryExecuteBatchRequest req) { + long rowsAffected = 0; + int currentSet = 0; + + try { + String sql = OdbcEscapeUtils.parse(req.sqlQuery()); + + if (log.isDebugEnabled()) + log.debug("ODBC query parsed [original=" + req.sqlQuery() + ", parsed=" + sql + ']'); + + SqlFieldsQuery qry = new SqlFieldsQuery(sql); + + qry.setDistributedJoins(distributedJoins); + qry.setEnforceJoinOrder(enforceJoinOrder); + + IgniteCache cache0 = ctx.grid().cache(req.schema()); + + if (cache0 == null) + return new OdbcResponse(OdbcResponse.STATUS_FAILED, + "Cache doesn't exist (did you configure it?): " + req.schema()); + + IgniteCache cache = cache0.withKeepBinary(); + + if (cache == null) + return new OdbcResponse(OdbcResponse.STATUS_FAILED, + "Can not get cache with keep binary: " + req.schema()); + + Object[][] paramSet = req.arguments(); + + if (paramSet.length <= 0) + throw new IgniteException("Batch execute request with non-positive batch length. [len=" + + paramSet.length + ']'); + + // Getting meta and do the checks for the first execution. + qry.setArgs(paramSet[0]); + + QueryCursorImpl> qryCur = (QueryCursorImpl>)cache.query(qry); + + if (qryCur.isQuery()) + throw new IgniteException("Batching of parameters only supported for DML statements. [query=" + + req.sqlQuery() + ']'); + + rowsAffected += getRowsAffected(qryCur); + + for (currentSet = 1; currentSet < paramSet.length; ++currentSet) + { + qry.setArgs(paramSet[currentSet]); + + QueryCursor> cur = cache.query(qry); + + rowsAffected += getRowsAffected(cur); + } + + OdbcQueryExecuteBatchResult res = new OdbcQueryExecuteBatchResult(rowsAffected); + + return new OdbcResponse(res); + } + catch (Exception e) { + U.error(log, "Failed to execute SQL query [req=" + req + ']', e); + + OdbcQueryExecuteBatchResult res = new OdbcQueryExecuteBatchResult(rowsAffected, currentSet, + e.getMessage()); + + return new OdbcResponse(res); + } + } + + /** + * Get affected rows for DML statement. + * @param qryCur Cursor. + * @return Number of table rows affected. + */ + private static long getRowsAffected(QueryCursor> qryCur) { + Iterator> iter = qryCur.iterator(); + + if (iter.hasNext()) { + List res = iter.next(); + + if (res.size() > 0) { + Long affected = (Long) res.get(0); + + if (affected != null) + return affected; + } + } + + return 0; + } + /** * {@link OdbcQueryCloseRequest} command handler. * diff --git a/modules/platforms/cpp/core-test/src/test_utils.cpp b/modules/platforms/cpp/core-test/src/test_utils.cpp new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/modules/platforms/cpp/odbc-test/src/application_data_buffer_test.cpp b/modules/platforms/cpp/odbc-test/src/application_data_buffer_test.cpp index fe502950a2898..fcc744ac89107 100644 --- a/modules/platforms/cpp/odbc-test/src/application_data_buffer_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/application_data_buffer_test.cpp @@ -44,9 +44,8 @@ BOOST_AUTO_TEST_CASE(TestPutIntToString) { char buffer[1024]; SqlLen reslen = 0; - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, buffer, sizeof(buffer), &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, buffer, sizeof(buffer), &reslen); appBuf.PutInt8(12); BOOST_CHECK(!strcmp(buffer, "12")); @@ -77,9 +76,8 @@ BOOST_AUTO_TEST_CASE(TestPutFloatToString) { char buffer[1024]; SqlLen reslen = 0; - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, buffer, sizeof(buffer), &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, buffer, sizeof(buffer), &reslen); appBuf.PutFloat(12.42f); BOOST_CHECK(!strcmp(buffer, "12.42")); @@ -102,9 +100,8 @@ BOOST_AUTO_TEST_CASE(TestPutGuidToString) { char buffer[1024]; SqlLen reslen = 0; - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, buffer, sizeof(buffer), &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, buffer, sizeof(buffer), &reslen); ignite::Guid guid(0x1da1ef8f39ff4d62ULL, 0x8b72e8e9f3371801ULL); @@ -119,7 +116,7 @@ BOOST_AUTO_TEST_CASE(TestGetGuidFromString) char buffer[] = "1da1ef8f-39ff-4d62-8b72-e8e9f3371801"; SqlLen reslen = sizeof(buffer) - 1; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, buffer, sizeof(buffer) - 1, &reslen, 0); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, buffer, sizeof(buffer) - 1, &reslen); ignite::Guid guid = appBuf.GetGuid(); @@ -130,9 +127,8 @@ BOOST_AUTO_TEST_CASE(TestPutBinaryToString) { char buffer[1024]; SqlLen reslen = 0; - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, buffer, sizeof(buffer), &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, buffer, sizeof(buffer), &reslen); uint8_t binary[] = { 0x21, 0x84, 0xF4, 0xDC, 0x01, 0x00, 0xFF, 0xF0 }; @@ -146,9 +142,8 @@ BOOST_AUTO_TEST_CASE(TestPutStringToString) { char buffer[1024]; SqlLen reslen = 0; - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, buffer, sizeof(buffer), &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, buffer, sizeof(buffer), &reslen); std::string testString("Test string"); @@ -162,9 +157,8 @@ BOOST_AUTO_TEST_CASE(TestPutStringToWstring) { wchar_t buffer[1024]; SqlLen reslen = 0; - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_WCHAR, buffer, sizeof(buffer), &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_WCHAR, buffer, sizeof(buffer), &reslen); std::string testString("Test string"); @@ -176,9 +170,8 @@ BOOST_AUTO_TEST_CASE(TestPutStringToLong) { long numBuf; SqlLen reslen = 0; - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_SIGNED_LONG, &numBuf, sizeof(numBuf), &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_SIGNED_LONG, &numBuf, sizeof(numBuf), &reslen); appBuf.PutString("424242424"); BOOST_CHECK(numBuf == 424242424L); @@ -191,9 +184,8 @@ BOOST_AUTO_TEST_CASE(TestPutStringToTiny) { int8_t numBuf; SqlLen reslen = 0; - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_SIGNED_TINYINT, &numBuf, sizeof(numBuf), &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_SIGNED_TINYINT, &numBuf, sizeof(numBuf), &reslen); appBuf.PutString("12"); BOOST_CHECK(numBuf == 12); @@ -206,9 +198,8 @@ BOOST_AUTO_TEST_CASE(TestPutStringToFloat) { float numBuf; SqlLen reslen = 0; - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_FLOAT, &numBuf, sizeof(numBuf), &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_FLOAT, &numBuf, sizeof(numBuf), &reslen); appBuf.PutString("12.21"); BOOST_CHECK_CLOSE_FRACTION(numBuf, 12.21, FLOAT_PRECISION); @@ -221,9 +212,8 @@ BOOST_AUTO_TEST_CASE(TestPutIntToFloat) { float numBuf; SqlLen reslen = 0; - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_FLOAT, &numBuf, sizeof(numBuf), &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_FLOAT, &numBuf, sizeof(numBuf), &reslen); appBuf.PutInt8(5); BOOST_CHECK_CLOSE_FRACTION(numBuf, 5.0, FLOAT_PRECISION); @@ -248,9 +238,8 @@ BOOST_AUTO_TEST_CASE(TestPutFloatToShort) { short numBuf; SqlLen reslen = 0; - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_SIGNED_SHORT, &numBuf, sizeof(numBuf), &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_SIGNED_SHORT, &numBuf, sizeof(numBuf), &reslen); appBuf.PutDouble(5.42); BOOST_CHECK(numBuf == 5); @@ -270,7 +259,7 @@ BOOST_AUTO_TEST_CASE(TestPutDecimalToDouble) double numBuf; SqlLen reslen = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_DOUBLE, &numBuf, sizeof(numBuf), &reslen, 0); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_DOUBLE, &numBuf, sizeof(numBuf), &reslen); common::Decimal decimal; @@ -299,7 +288,7 @@ BOOST_AUTO_TEST_CASE(TestPutDecimalToLong) long numBuf; SqlLen reslen = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_SIGNED_LONG, &numBuf, sizeof(numBuf), &reslen, 0); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_SIGNED_LONG, &numBuf, sizeof(numBuf), &reslen); common::Decimal decimal; @@ -326,7 +315,7 @@ BOOST_AUTO_TEST_CASE(TestPutDecimalToString) char strBuf[64]; SqlLen reslen = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &strBuf, sizeof(strBuf), &reslen, 0); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &strBuf, sizeof(strBuf), &reslen); common::Decimal decimal; @@ -353,7 +342,7 @@ BOOST_AUTO_TEST_CASE(TestPutDecimalToNumeric) SQL_NUMERIC_STRUCT buf; SqlLen reslen = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_NUMERIC, &buf, sizeof(buf), &reslen, 0); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_NUMERIC, &buf, sizeof(buf), &reslen); common::Decimal decimal; @@ -404,7 +393,7 @@ BOOST_AUTO_TEST_CASE(TestPutDateToString) char strBuf[64] = { 0 }; SqlLen reslen = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &strBuf, sizeof(strBuf), &reslen, 0); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &strBuf, sizeof(strBuf), &reslen); Date date = BinaryUtils::MakeDateGmt(1999, 2, 22); @@ -418,7 +407,7 @@ BOOST_AUTO_TEST_CASE(TestPutTimestampToString) char strBuf[64] = { 0 }; SqlLen reslen = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &strBuf, sizeof(strBuf), &reslen, 0); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &strBuf, sizeof(strBuf), &reslen); Timestamp date = BinaryUtils::MakeTimestampGmt(2018, 11, 1, 17, 45, 59); @@ -432,10 +421,7 @@ BOOST_AUTO_TEST_CASE(TestPutDateToDate) SQL_DATE_STRUCT buf = { 0 }; SqlLen reslen = sizeof(buf); - int offset = 0; - int* offsetPtr = &offset; - - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_TDATE, &buf, sizeof(buf), &reslen, &offsetPtr); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_TDATE, &buf, sizeof(buf), &reslen); Date date = BinaryUtils::MakeDateGmt(1984, 5, 27); @@ -451,10 +437,7 @@ BOOST_AUTO_TEST_CASE(TestPutTimestampToDate) SQL_DATE_STRUCT buf = { 0 }; SqlLen reslen = sizeof(buf); - int offset = 0; - int* offsetPtr = &offset; - - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_TDATE, &buf, sizeof(buf), &reslen, &offsetPtr); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_TDATE, &buf, sizeof(buf), &reslen); Timestamp ts = BinaryUtils::MakeTimestampGmt(2004, 8, 14, 6, 34, 51, 573948623); @@ -470,10 +453,7 @@ BOOST_AUTO_TEST_CASE(TestPutTimestampToTimestamp) SQL_TIMESTAMP_STRUCT buf = { 0 }; SqlLen reslen = sizeof(buf); - int offset = 0; - int* offsetPtr = &offset; - - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_TTIMESTAMP, &buf, sizeof(buf), &reslen, &offsetPtr); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_TTIMESTAMP, &buf, sizeof(buf), &reslen); Timestamp ts = BinaryUtils::MakeTimestampGmt(2004, 8, 14, 6, 34, 51, 573948623); @@ -494,10 +474,7 @@ BOOST_AUTO_TEST_CASE(TestPutDateToTimestamp) SqlLen reslen = sizeof(buf); - int offset = 0; - int* offsetPtr = &offset; - - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_TTIMESTAMP, &buf, sizeof(buf), &reslen, &offsetPtr); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_TTIMESTAMP, &buf, sizeof(buf), &reslen); Date date = BinaryUtils::MakeDateGmt(1984, 5, 27); @@ -516,9 +493,8 @@ BOOST_AUTO_TEST_CASE(TestGetStringFromLong) { long numBuf = 42; SqlLen reslen = sizeof(numBuf); - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_SIGNED_LONG, &numBuf, reslen, &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_SIGNED_LONG, &numBuf, reslen, &reslen); std::string res = appBuf.GetString(32); @@ -535,9 +511,8 @@ BOOST_AUTO_TEST_CASE(TestGetStringFromDouble) { double numBuf = 43.36; SqlLen reslen = sizeof(numBuf); - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_DOUBLE, &numBuf, reslen, &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_DOUBLE, &numBuf, reslen, &reslen); std::string res = appBuf.GetString(32); @@ -554,9 +529,8 @@ BOOST_AUTO_TEST_CASE(TestGetStringFromString) { char buf[] = "Some data 32d2d5hs"; SqlLen reslen = sizeof(buf); - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &buf, reslen, &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &buf, reslen, &reslen); std::string res = appBuf.GetString(reslen); @@ -567,9 +541,8 @@ BOOST_AUTO_TEST_CASE(TestGetFloatFromUshort) { unsigned short numBuf = 7162; SqlLen reslen = sizeof(numBuf); - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_UNSIGNED_SHORT, &numBuf, reslen, &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_UNSIGNED_SHORT, &numBuf, reslen, &reslen); float resFloat = appBuf.GetFloat(); @@ -584,9 +557,8 @@ BOOST_AUTO_TEST_CASE(TestGetFloatFromString) { char buf[] = "28.562"; SqlLen reslen = sizeof(buf); - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &buf, reslen, &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &buf, reslen, &reslen); float resFloat = appBuf.GetFloat(); @@ -601,9 +573,8 @@ BOOST_AUTO_TEST_CASE(TestGetFloatFromFloat) { float buf = 207.49f; SqlLen reslen = sizeof(buf); - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_FLOAT, &buf, reslen, &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_FLOAT, &buf, reslen, &reslen); float resFloat = appBuf.GetFloat(); @@ -618,9 +589,8 @@ BOOST_AUTO_TEST_CASE(TestGetFloatFromDouble) { double buf = 893.162; SqlLen reslen = sizeof(buf); - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_DOUBLE, &buf, reslen, &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_DOUBLE, &buf, reslen, &reslen); float resFloat = appBuf.GetFloat(); @@ -635,9 +605,8 @@ BOOST_AUTO_TEST_CASE(TestGetIntFromString) { char buf[] = "39"; SqlLen reslen = sizeof(buf); - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &buf, reslen, &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &buf, reslen, &reslen); int64_t resInt64 = appBuf.GetInt64(); @@ -660,9 +629,8 @@ BOOST_AUTO_TEST_CASE(TestGetIntFromFloat) { float buf = -107.49f; SqlLen reslen = sizeof(buf); - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_FLOAT, &buf, reslen, &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_FLOAT, &buf, reslen, &reslen); int64_t resInt64 = appBuf.GetInt64(); @@ -685,9 +653,8 @@ BOOST_AUTO_TEST_CASE(TestGetIntFromDouble) { double buf = 42.97f; SqlLen reslen = sizeof(buf); - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_DOUBLE, &buf, reslen, &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_DOUBLE, &buf, reslen, &reslen); int64_t resInt64 = appBuf.GetInt64(); @@ -710,9 +677,8 @@ BOOST_AUTO_TEST_CASE(TestGetIntFromBigint) { uint64_t buf = 19; SqlLen reslen = sizeof(buf); - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_UNSIGNED_BIGINT, &buf, reslen, &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_UNSIGNED_BIGINT, &buf, reslen, &reslen); int64_t resInt64 = appBuf.GetInt64(); @@ -744,22 +710,19 @@ BOOST_AUTO_TEST_CASE(TestGetIntWithOffset) { 42, sizeof(uint64_t) } }; - int offset = 0; - int* offsetPtr = &offset; - - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_UNSIGNED_BIGINT, &buf[0].val, sizeof(buf[0].val), &buf[0].reslen, &offsetPtr); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_UNSIGNED_BIGINT, &buf[0].val, sizeof(buf[0].val), &buf[0].reslen); int64_t val = appBuf.GetInt64(); BOOST_CHECK(val == 12); - offset += sizeof(TestStruct); + appBuf.SetByteOffset(sizeof(TestStruct)); val = appBuf.GetInt64(); BOOST_CHECK(val == 42); - offsetPtr = 0; + appBuf.SetByteOffset(0); val = appBuf.GetInt64(); @@ -779,10 +742,7 @@ BOOST_AUTO_TEST_CASE(TestSetStringWithOffset) { "", 0 } }; - int offset = 0; - int* offsetPtr = &offset; - - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &buf[0].val, sizeof(buf[0].val), &buf[0].reslen, &offsetPtr); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &buf[0].val, sizeof(buf[0].val), &buf[0].reslen); appBuf.PutString("Hello Ignite!"); @@ -792,7 +752,7 @@ BOOST_AUTO_TEST_CASE(TestSetStringWithOffset) BOOST_CHECK(res == "Hello Ignite!"); BOOST_CHECK(res.size() == strlen("Hello Ignite!")); - offset += sizeof(TestStruct); + appBuf.SetByteOffset(sizeof(TestStruct)); appBuf.PutString("Hello with offset!"); @@ -814,10 +774,7 @@ BOOST_AUTO_TEST_CASE(TestGetDateFromString) char buf[] = "1999-02-22"; SqlLen reslen = sizeof(buf); - int offset = 0; - int* offsetPtr = &offset; - - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &buf[0], sizeof(buf), &reslen, &offsetPtr); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &buf[0], sizeof(buf), &reslen); Date date = appBuf.GetDate(); @@ -840,10 +797,7 @@ BOOST_AUTO_TEST_CASE(TestGetTimestampFromString) char buf[] = "2018-11-01 17:45:59"; SqlLen reslen = sizeof(buf); - int offset = 0; - int* offsetPtr = &offset; - - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &buf[0], sizeof(buf), &reslen, &offsetPtr); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &buf[0], sizeof(buf), &reslen); Timestamp date = appBuf.GetTimestamp(); @@ -871,10 +825,7 @@ BOOST_AUTO_TEST_CASE(TestGetDateFromDate) SqlLen reslen = sizeof(buf); - int offset = 0; - int* offsetPtr = &offset; - - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_TDATE, &buf, sizeof(buf), &reslen, &offsetPtr); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_TDATE, &buf, sizeof(buf), &reslen); Date date = appBuf.GetDate(); @@ -902,10 +853,7 @@ BOOST_AUTO_TEST_CASE(TestGetTimestampFromDate) SqlLen reslen = sizeof(buf); - int offset = 0; - int* offsetPtr = &offset; - - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_TDATE, &buf, sizeof(buf), &reslen, &offsetPtr); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_TDATE, &buf, sizeof(buf), &reslen); Timestamp ts = appBuf.GetTimestamp(); @@ -937,10 +885,7 @@ BOOST_AUTO_TEST_CASE(TestGetTimestampFromTimestamp) SqlLen reslen = sizeof(buf); - int offset = 0; - int* offsetPtr = &offset; - - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_TTIMESTAMP, &buf, sizeof(buf), &reslen, &offsetPtr); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_TTIMESTAMP, &buf, sizeof(buf), &reslen); Timestamp ts = appBuf.GetTimestamp(); @@ -973,10 +918,7 @@ BOOST_AUTO_TEST_CASE(TestGetDateFromTimestamp) SqlLen reslen = sizeof(buf); - int offset = 0; - int* offsetPtr = &offset; - - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_TTIMESTAMP, &buf, sizeof(buf), &reslen, &offsetPtr); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_TTIMESTAMP, &buf, sizeof(buf), &reslen); Date date = appBuf.GetDate(); diff --git a/modules/platforms/cpp/odbc-test/src/column_test.cpp b/modules/platforms/cpp/odbc-test/src/column_test.cpp index 6cbea8b648f4b..a5b0d0854103a 100644 --- a/modules/platforms/cpp/odbc-test/src/column_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/column_test.cpp @@ -66,9 +66,8 @@ BOOST_AUTO_TEST_CASE(TestColumnShort) short shortBuf = 0; SqlLen reslen = 0; - int* offset = 0; - ApplicationDataBuffer appBuf(type_traits::IGNITE_ODBC_C_TYPE_SIGNED_SHORT, &shortBuf, sizeof(shortBuf), &reslen, &offset); + ApplicationDataBuffer appBuf(type_traits::IGNITE_ODBC_C_TYPE_SIGNED_SHORT, &shortBuf, sizeof(shortBuf), &reslen); BOOST_REQUIRE(column.ReadToBuffer(reader, appBuf) == SQL_RESULT_SUCCESS); @@ -114,9 +113,8 @@ BOOST_AUTO_TEST_CASE(TestColumnString) char strBuf[1024] = {}; SqlLen reslen = 0; - int* offset = 0; - ApplicationDataBuffer appBuf(type_traits::IGNITE_ODBC_C_TYPE_CHAR, &strBuf, sizeof(strBuf), &reslen, &offset); + ApplicationDataBuffer appBuf(type_traits::IGNITE_ODBC_C_TYPE_CHAR, &strBuf, sizeof(strBuf), &reslen); BOOST_REQUIRE(column.ReadToBuffer(reader, appBuf) == SQL_RESULT_SUCCESS); @@ -164,9 +162,8 @@ BOOST_AUTO_TEST_CASE(TestColumnStringSeveral) std::string strBuf(data.size() / 3 + 2, 0); SqlLen reslen = 0; - int* offset = 0; - ApplicationDataBuffer appBuf(type_traits::IGNITE_ODBC_C_TYPE_CHAR, &strBuf[0], strBuf.size(), &reslen, &offset); + ApplicationDataBuffer appBuf(type_traits::IGNITE_ODBC_C_TYPE_CHAR, &strBuf[0], strBuf.size(), &reslen); BOOST_REQUIRE(column.ReadToBuffer(reader, appBuf) == SQL_RESULT_SUCCESS); @@ -246,9 +243,8 @@ BOOST_AUTO_TEST_CASE(TestColumnMultiString) char strBuf[1024] = {}; SqlLen reslen = 0; - int* offset = 0; - ApplicationDataBuffer appBuf(type_traits::IGNITE_ODBC_C_TYPE_CHAR, &strBuf, sizeof(strBuf), &reslen, &offset); + ApplicationDataBuffer appBuf(type_traits::IGNITE_ODBC_C_TYPE_CHAR, &strBuf, sizeof(strBuf), &reslen); BOOST_REQUIRE(column1.ReadToBuffer(reader, appBuf) == SQL_RESULT_SUCCESS); diff --git a/modules/platforms/cpp/odbc-test/src/queries_test.cpp b/modules/platforms/cpp/odbc-test/src/queries_test.cpp index 3a3858802459e..9aca77de784ed 100644 --- a/modules/platforms/cpp/odbc-test/src/queries_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/queries_test.cpp @@ -33,6 +33,7 @@ #include #include "ignite/ignite.h" +#include "ignite/common/fixed_size_array.h" #include "ignite/ignition.h" #include "ignite/impl/binary/binary_utils.h" @@ -324,10 +325,10 @@ struct QueriesTestSuiteFixture } /** - * Insert requested number of TestType vlaues with all defaults except + * Insert requested number of TestType values with all defaults except * for the strFields, which are generated using getTestString(). * - * @param num Number of records to insert. + * @param recordsNum Number of records to insert. * @param merge Set to true to use merge instead. */ void InsertTestStrings(int recordsNum, bool merge = false) @@ -385,6 +386,317 @@ struct QueriesTestSuiteFixture BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); } + /** + * Insert requested number of TestType values in a batch. + * + * @param from Index to start from. + * @param to Index to stop. + * @param expectedToAffect Expected number of affected records. + * @param merge Set to true to use merge instead of insert. + * @return Records inserted. + */ + int InsertTestBatch(int from, int to, int expectedToAffect, bool merge = false) + { + SQLCHAR insertReq[] = "INSERT " + "INTO TestType(_key, i8Field, i16Field, i32Field, strField, floatField, doubleField, boolField, dateField, " + "timestampField) VALUES(?, ?, ?, ?, ?, ?, ?, ?, ?, ?)"; + + SQLCHAR mergeReq[] = "MERGE " + "INTO TestType(_key, i8Field, i16Field, i32Field, strField, floatField, doubleField, boolField, dateField, " + "timestampField) VALUES(?, ?, ?, ?, ?, ?, ?, ?, ?, ?)"; + + SQLRETURN ret; + + int recordsNum = to - from; + + ret = SQLPrepare(stmt, merge ? mergeReq : insertReq, SQL_NTS); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + FixedSizeArray keys(recordsNum); + FixedSizeArray i8Fields(recordsNum); + FixedSizeArray i16Fields(recordsNum); + FixedSizeArray i32Fields(recordsNum); + FixedSizeArray strFields(recordsNum * 1024); + FixedSizeArray floatFields(recordsNum); + FixedSizeArray doubleFields(recordsNum); + FixedSizeArray boolFields(recordsNum); + FixedSizeArray dateFields(recordsNum); + FixedSizeArray timeFields(recordsNum); + FixedSizeArray timestampFields(recordsNum); + + FixedSizeArray strFieldsLen(recordsNum); + + BOOST_CHECKPOINT("Filling param data"); + + for (int i = 0; i < recordsNum; ++i) + { + int seed = from + i; + + keys[i] = seed; + i8Fields[i] = seed * 8; + i16Fields[i] = seed * 16; + i32Fields[i] = seed * 32; + + std::string val = getTestString(seed); + strncpy(strFields.GetData() + 1024 * i, val.c_str(), 1023); + strFieldsLen[i] = val.size(); + + floatFields[i] = seed * 0.5f; + doubleFields[i] = seed * 0.25f; + boolFields[i] = seed % 2 == 0; + + dateFields[i].year = 2017 + seed / 365; + dateFields[i].month = ((seed / 28) % 12) + 1; + dateFields[i].day = (seed % 28) + 1; + + timeFields[i].hour = (seed / 3600) % 24; + timeFields[i].minute = (seed / 60) % 60; + timeFields[i].second = seed % 60; + + timestampFields[i].year = dateFields[i].year; + timestampFields[i].month = dateFields[i].month; + timestampFields[i].day = dateFields[i].day; + timestampFields[i].hour = timeFields[i].hour; + timestampFields[i].minute = timeFields[i].minute; + timestampFields[i].second = timeFields[i].second; + timestampFields[i].fraction = std::abs(seed * 914873) % 1000000000; + } + + SQLULEN setsProcessed = 0; + + BOOST_CHECKPOINT("Setting processed pointer"); + ret = SQLSetStmtAttr(stmt, SQL_ATTR_PARAMS_PROCESSED_PTR, &setsProcessed, SQL_IS_POINTER); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECKPOINT("Binding keys"); + ret = SQLBindParameter(stmt, 1, SQL_PARAM_INPUT, SQL_C_SBIGINT, SQL_BIGINT, 0, 0, keys.GetData(), 0, 0); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECKPOINT("Binding i8Fields"); + ret = SQLBindParameter(stmt, 2, SQL_PARAM_INPUT, SQL_C_STINYINT, SQL_TINYINT, 0, 0, i8Fields.GetData(), 0, 0); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECKPOINT("Binding i16Fields"); + ret = SQLBindParameter(stmt, 3, SQL_PARAM_INPUT, SQL_C_SSHORT, SQL_SMALLINT, 0, 0, i16Fields.GetData(), 0, 0); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECKPOINT("Binding i32Fields"); + ret = SQLBindParameter(stmt, 4, SQL_PARAM_INPUT, SQL_C_SLONG, SQL_INTEGER, 0, 0, i32Fields.GetData(), 0, 0); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECKPOINT("Binding strFields"); + ret = SQLBindParameter(stmt, 5, SQL_PARAM_INPUT, SQL_C_CHAR, SQL_VARCHAR, 1024, 0, strFields.GetData(), 1024, strFieldsLen.GetData()); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECKPOINT("Binding floatFields"); + ret = SQLBindParameter(stmt, 6, SQL_PARAM_INPUT, SQL_C_FLOAT, SQL_FLOAT, 0, 0, floatFields.GetData(), 0, 0); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECKPOINT("Binding doubleFields"); + ret = SQLBindParameter(stmt, 7, SQL_PARAM_INPUT, SQL_C_DOUBLE, SQL_DOUBLE, 0, 0, doubleFields.GetData(), 0, 0); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECKPOINT("Binding boolFields"); + ret = SQLBindParameter(stmt, 8, SQL_PARAM_INPUT, SQL_C_BIT, SQL_BIT, 0, 0, boolFields.GetData(), 0, 0); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECKPOINT("Binding dateFields"); + ret = SQLBindParameter(stmt, 9, SQL_PARAM_INPUT, SQL_C_DATE, SQL_DATE, 0, 0, dateFields.GetData(), 0, 0); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECKPOINT("Binding timestampFields"); + ret = SQLBindParameter(stmt, 10, SQL_PARAM_INPUT, SQL_C_TIMESTAMP, SQL_TIMESTAMP, 0, 0, timestampFields.GetData(), 0, 0); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECKPOINT("Setting paramset size"); + ret = SQLSetStmtAttr(stmt, SQL_ATTR_PARAMSET_SIZE, reinterpret_cast(recordsNum), 0); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECKPOINT("Executing query"); + ret = SQLExecute(stmt); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + SQLLEN affected = 0; + ret = SQLRowCount(stmt, &affected); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECK_EQUAL(affected, expectedToAffect); + + BOOST_CHECKPOINT("Getting next result set"); + ret = SQLMoreResults(stmt); + + if (ret != SQL_NO_DATA) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECKPOINT("Resetting parameters."); + ret = SQLFreeStmt(stmt, SQL_RESET_PARAMS); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECKPOINT("Setting paramset size"); + ret = SQLSetStmtAttr(stmt, SQL_ATTR_PARAMSET_SIZE, reinterpret_cast(1), 0); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + return static_cast(setsProcessed); + } + + void InsertBatchSelect(int recordsNum) + { + Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache"); + + // Inserting values. + int inserted = InsertTestBatch(0, recordsNum, recordsNum); + + BOOST_REQUIRE_EQUAL(inserted, recordsNum); + + int64_t key = 0; + char strField[1024] = { 0 }; + SQLLEN strFieldLen = 0; + + // Binding columns. + SQLRETURN ret = SQLBindCol(stmt, 1, SQL_C_SLONG, &key, 0, 0); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + // Binding columns. + ret = SQLBindCol(stmt, 2, SQL_C_CHAR, &strField, sizeof(strField), &strFieldLen); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + // Just selecting everything to make sure everything is OK + SQLCHAR selectReq[] = "SELECT _key, strField FROM TestType ORDER BY _key"; + + ret = SQLExecDirect(stmt, selectReq, sizeof(selectReq)); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + int selectedRecordsNum = 0; + + ret = SQL_SUCCESS; + + while (ret == SQL_SUCCESS) + { + ret = SQLFetch(stmt); + + if (ret == SQL_NO_DATA) + break; + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + std::string expectedStr = getTestString(selectedRecordsNum); + int64_t expectedKey = selectedRecordsNum; + + BOOST_CHECK_EQUAL(key, expectedKey); + + BOOST_CHECK_EQUAL(std::string(strField, strFieldLen), expectedStr); + + ++selectedRecordsNum; + } + + BOOST_CHECK_EQUAL(recordsNum, selectedRecordsNum); + } + + void InsertNonFullBatchSelect(int recordsNum, int splitAt) + { + Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache"); + + // Inserting values. + int inserted = InsertTestBatch(splitAt, recordsNum, recordsNum - splitAt); + + BOOST_REQUIRE_EQUAL(inserted, recordsNum - splitAt); + + inserted = InsertTestBatch(0, recordsNum, splitAt); + + BOOST_REQUIRE_EQUAL(inserted, splitAt); + + int64_t key = 0; + char strField[1024] = { 0 }; + SQLLEN strFieldLen = 0; + + // Binding columns. + SQLRETURN ret = SQLBindCol(stmt, 1, SQL_C_SLONG, &key, 0, 0); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + // Binding columns. + ret = SQLBindCol(stmt, 2, SQL_C_CHAR, &strField, sizeof(strField), &strFieldLen); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + // Just selecting everything to make sure everything is OK + SQLCHAR selectReq[] = "SELECT _key, strField FROM TestType ORDER BY _key"; + + ret = SQLExecDirect(stmt, selectReq, sizeof(selectReq)); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + int selectedRecordsNum = 0; + + ret = SQL_SUCCESS; + + while (ret == SQL_SUCCESS) + { + ret = SQLFetch(stmt); + + if (ret == SQL_NO_DATA) + break; + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + std::string expectedStr = getTestString(selectedRecordsNum); + int64_t expectedKey = selectedRecordsNum; + + BOOST_CHECK_EQUAL(key, expectedKey); + + BOOST_CHECK_EQUAL(std::string(strField, strFieldLen), expectedStr); + + ++selectedRecordsNum; + } + + BOOST_CHECK_EQUAL(recordsNum, selectedRecordsNum); + } + /** Node started during the test. */ Ignite grid; @@ -1361,6 +1673,76 @@ BOOST_AUTO_TEST_CASE(TestInsertMergeSelect) BOOST_CHECK_EQUAL(recordsNum, selectedRecordsNum); } +BOOST_AUTO_TEST_CASE(TestInsertBatchSelect2) +{ + InsertBatchSelect(2); +} + +BOOST_AUTO_TEST_CASE(TestInsertBatchSelect100) +{ + InsertBatchSelect(100); +} + +BOOST_AUTO_TEST_CASE(TestInsertBatchSelect1000) +{ + InsertBatchSelect(1000); +} + +BOOST_AUTO_TEST_CASE(TestInsertBatchSelect1023) +{ + InsertBatchSelect(1024); +} + +BOOST_AUTO_TEST_CASE(TestInsertBatchSelect1024) +{ + InsertBatchSelect(1024); +} + +BOOST_AUTO_TEST_CASE(TestInsertBatchSelect1025) +{ + InsertBatchSelect(1025); +} + +BOOST_AUTO_TEST_CASE(TestInsertBatchSelect2000) +{ + InsertBatchSelect(2000); +} + +BOOST_AUTO_TEST_CASE(TestInsertBatchSelect2047) +{ + InsertBatchSelect(2048); +} + +BOOST_AUTO_TEST_CASE(TestInsertBatchSelect2048) +{ + InsertBatchSelect(2048); +} + +BOOST_AUTO_TEST_CASE(TestInsertBatchSelect2049) +{ + InsertBatchSelect(2049); +} + +BOOST_AUTO_TEST_CASE(TestNotFullInsertBatchSelect900) +{ + InsertNonFullBatchSelect(900, 42); +} + +BOOST_AUTO_TEST_CASE(TestNotFullInsertBatchSelect1500) +{ + InsertNonFullBatchSelect(1500, 100); +} + +BOOST_AUTO_TEST_CASE(TestNotFullInsertBatchSelect4500) +{ + InsertNonFullBatchSelect(4500, 1500); +} + +BOOST_AUTO_TEST_CASE(TestNotFullInsertBatchSelect4096) +{ + InsertNonFullBatchSelect(4096, 1024); +} + BOOST_AUTO_TEST_CASE(TestParamsNum) { Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache"); diff --git a/modules/platforms/cpp/odbc-test/src/row_test.cpp b/modules/platforms/cpp/odbc-test/src/row_test.cpp index f38e9c5b58f1d..1ae34c12feadf 100644 --- a/modules/platforms/cpp/odbc-test/src/row_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/row_test.cpp @@ -82,12 +82,11 @@ void CheckRowData(Row& row, size_t rowIdx) char strBuf[1024]; SQLGUID guidBuf; char bitBuf; - int* offset = 0; - ApplicationDataBuffer appLongBuf(type_traits::IGNITE_ODBC_C_TYPE_SIGNED_LONG, &longBuf, sizeof(longBuf), &reslen, &offset); - ApplicationDataBuffer appStrBuf(type_traits::IGNITE_ODBC_C_TYPE_CHAR, &strBuf, sizeof(strBuf), &reslen, &offset); - ApplicationDataBuffer appGuidBuf(type_traits::IGNITE_ODBC_C_TYPE_GUID, &guidBuf, sizeof(guidBuf), &reslen, &offset); - ApplicationDataBuffer appBitBuf(type_traits::IGNITE_ODBC_C_TYPE_BIT, &bitBuf, sizeof(bitBuf), &reslen, &offset); + ApplicationDataBuffer appLongBuf(type_traits::IGNITE_ODBC_C_TYPE_SIGNED_LONG, &longBuf, sizeof(longBuf), &reslen); + ApplicationDataBuffer appStrBuf(type_traits::IGNITE_ODBC_C_TYPE_CHAR, &strBuf, sizeof(strBuf), &reslen); + ApplicationDataBuffer appGuidBuf(type_traits::IGNITE_ODBC_C_TYPE_GUID, &guidBuf, sizeof(guidBuf), &reslen); + ApplicationDataBuffer appBitBuf(type_traits::IGNITE_ODBC_C_TYPE_BIT, &bitBuf, sizeof(bitBuf), &reslen); // Checking size. BOOST_REQUIRE(row.GetSize() == 4); diff --git a/modules/platforms/cpp/odbc/Makefile.am b/modules/platforms/cpp/odbc/Makefile.am index b0cc5f87133fc..c2fb42e19edff 100644 --- a/modules/platforms/cpp/odbc/Makefile.am +++ b/modules/platforms/cpp/odbc/Makefile.am @@ -52,6 +52,7 @@ libignite_odbc_la_SOURCES = \ os/linux/src/system/socket_client.cpp \ src/app/application_data_buffer.cpp \ src/app/parameter.cpp \ + src/app/parameter_set.cpp \ src/common_types.cpp \ src/config/configuration.cpp \ src/config/connection_info.cpp \ @@ -68,6 +69,7 @@ libignite_odbc_la_SOURCES = \ src/dsn_config.cpp \ src/query/column_metadata_query.cpp \ src/query/data_query.cpp \ + src/query/batch_query.cpp \ src/query/foreign_keys_query.cpp \ src/query/primary_keys_query.cpp \ src/query/table_metadata_query.cpp \ @@ -76,6 +78,7 @@ libignite_odbc_la_SOURCES = \ src/protocol_version.cpp \ src/result_page.cpp \ src/row.cpp \ + src/message.cpp \ src/column.cpp \ src/statement.cpp \ src/type_traits.cpp \ diff --git a/modules/platforms/cpp/odbc/include/Makefile.am b/modules/platforms/cpp/odbc/include/Makefile.am index 073dcaa1898ba..1408aee3cb52b 100644 --- a/modules/platforms/cpp/odbc/include/Makefile.am +++ b/modules/platforms/cpp/odbc/include/Makefile.am @@ -22,6 +22,7 @@ noinst_HEADERS = \ ignite/odbc/query/table_metadata_query.h \ ignite/odbc/query/special_columns_query.h \ ignite/odbc/query/type_info_query.h \ + ignite/odbc/query/batch_query.h \ ignite/odbc/query/data_query.h \ ignite/odbc/query/foreign_keys_query.h \ ignite/odbc/query/column_metadata_query.h \ @@ -35,6 +36,7 @@ noinst_HEADERS = \ ignite/odbc/parser.h \ ignite/odbc/app/application_data_buffer.h \ ignite/odbc/app/parameter.h \ + ignite/odbc/app/parameter_set.h \ ignite/odbc/row.h \ ignite/odbc/utility.h \ ignite/odbc/environment.h \ diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/app/application_data_buffer.h b/modules/platforms/cpp/odbc/include/ignite/odbc/app/application_data_buffer.h index 18ac36aead536..b7989c55982fa 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/app/application_data_buffer.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/app/application_data_buffer.h @@ -54,10 +54,9 @@ namespace ignite * @param buffer Data buffer pointer. * @param buflen Data buffer length. * @param reslen Resulting data length. - * @param offset Pointer to buffer and reslen offset pointer. */ ApplicationDataBuffer(type_traits::IgniteSqlType type, void* buffer, - SqlLen buflen, SqlLen* reslen, int** offset = 0); + SqlLen buflen, SqlLen* reslen); /** * Copy constructor. @@ -80,13 +79,23 @@ namespace ignite ApplicationDataBuffer& operator=(const ApplicationDataBuffer& other); /** - * Set pointer to offset pointer. + * Set offset in bytes for all bound pointers. * - * @param offset Pointer to offset pointer. + * @param offset Offset. */ - void SetPtrToOffsetPtr(int** offset) + void SetByteOffset(int offset) { - this->offset = offset; + this->byteOffset = offset; + } + + /** + * Set offset in elements for all bound pointers. + * + * @param + */ + void SetElementOffset(SqlUlen idx) + { + this->elementOffset = idx; } /** @@ -313,6 +322,13 @@ namespace ignite */ SqlLen GetDataAtExecSize() const; + /** + * Get single element size. + * + * @return Size of the single element. + */ + SqlLen GetElementSize() const; + /** * Get size of the input buffer. * @@ -392,10 +408,11 @@ namespace ignite * Apply buffer offset to pointer. * Adds offset to pointer if offset pointer is not null. * @param ptr Pointer. + * @param elemSize Element size. * @return Pointer with applied offset. */ template - T* ApplyOffset(T* ptr) const; + T* ApplyOffset(T* ptr, size_t elemSize) const; /** Underlying data type. */ type_traits::IgniteSqlType type; @@ -409,8 +426,11 @@ namespace ignite /** Result length. */ SqlLen* reslen; - /** Pointer to implementation pointer to application offset */ - int** offset; + /** Current byte offset */ + int byteOffset; + + /** Current element offset. */ + SqlUlen elementOffset; }; /** Column binging map type alias. */ diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/app/parameter.h b/modules/platforms/cpp/odbc/include/ignite/odbc/app/parameter.h index 1cf85b514aa59..cdaaead40ee3d 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/app/parameter.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/app/parameter.h @@ -22,12 +22,10 @@ #include -#include #include #include #include "ignite/odbc/app/application_data_buffer.h" -#include "ignite/odbc/type_traits.h" namespace ignite { @@ -70,7 +68,7 @@ namespace ignite ~Parameter(); /** - * Copy assigment operator. + * Assignment operator. * * @param other Other instance. * @return This. @@ -78,10 +76,12 @@ namespace ignite Parameter& operator=(const Parameter& other); /** - * Write request using provided writer. + * Write parameter using provided writer. * @param writer Writer. + * @param offset Offset for the buffer. + * @param idx Index for the array-of-parameters case. */ - void Write(impl::binary::BinaryWriterImpl& writer) const; + void Write(impl::binary::BinaryWriterImpl& writer, int offset = 0, SqlUlen idx = 0) const; /** * Get data buffer. @@ -90,6 +90,13 @@ namespace ignite */ ApplicationDataBuffer& GetBuffer(); + /** + * Get data buffer. + * + * @return underlying ApplicationDataBuffer instance. + */ + const ApplicationDataBuffer& GetBuffer() const; + /** * Reset stored at-execution data. */ @@ -128,9 +135,6 @@ namespace ignite /** Stored at-execution data. */ std::vector storedData; }; - - /** Parameter binging map type alias. */ - typedef std::map ParameterBindingMap; } } } diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/app/parameter_set.h b/modules/platforms/cpp/odbc/include/ignite/odbc/app/parameter_set.h new file mode 100644 index 0000000000000..2ab55808c073c --- /dev/null +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/app/parameter_set.h @@ -0,0 +1,268 @@ +/* + * 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. + */ + +#ifndef _IGNITE_ODBC_APP_PARAMETER_SET +#define _IGNITE_ODBC_APP_PARAMETER_SET + +#include + +#include + +#include +#include + +#include "ignite/odbc/app/parameter.h" + +namespace ignite +{ + namespace odbc + { + namespace app + { + /** + * Parameter set. + */ + class ParameterSet + { + /** Parameter binging map type alias. */ + typedef std::map ParameterBindingMap; + + /** Parameter meta vector. */ + typedef std::vector ParameterTypeVector; + public: + /** + * Default constructor. + */ + ParameterSet(); + + /** + * Destructor. + */ + ~ParameterSet() + { + // No-op. + } + + /** + * Set parameters set size. + * + * @param size Size of the parameter set. + */ + void SetParamSetSize(SqlUlen size); + + /** + * Bind parameter. + * + * @param paramIdx Parameter index. + * @param param Parameter. + */ + void BindParameter(uint16_t paramIdx, const Parameter& param); + + /** + * Unbind specified parameter. + * + * @param paramIdx Parameter index. + */ + void UnbindParameter(uint16_t paramIdx); + + /** + * Unbind all parameters. + */ + void UnbindAll(); + + /** + * Get number of binded parameters. + * + * @return Number of binded parameters. + */ + uint16_t GetParametersNumber() const; + + /** + * Set parameter binding offset pointer. + * + * @param ptr Parameter binding offset pointer. + */ + void SetParamBindOffsetPtr(int* ptr); + + /** + * Get parameter binding offset pointer. + * + * @return Parameter binding offset pointer. + */ + int* GetParamBindOffsetPtr(); + + /** + * Prepare parameters set for statement execution. + */ + void Prepare(); + + /** + * Check if the data at-execution is needed. + * + * @return True if the data at execution is needed. + */ + bool IsDataAtExecNeeded() const; + + /** + * Update parameter types metadata. + * + * @param meta Types metadata. + */ + void UpdateParamsTypes(const ParameterTypeVector& meta); + + /** + * Get type id of the parameter. + * + * @param idx Parameter index. + * @param dflt Default value to return if the type can not be found. + * @return Type ID of the parameter or dflt, if the type can not be returned. + */ + int8_t GetParamType(int16_t idx, int8_t dflt); + + /** + * Get expected parameters number. + * Using metadata. If metadata was not updated returns zero. + * + * @return Expected parameters number. + */ + uint16_t GetExpectedParamNum(); + + /** + * Check if the metadata was set for the parameter set. + * + * @return True if the metadata was set for the parameter set. + */ + bool IsMetadataSet() const; + + /** + * Check if the parameter selected for putting data at-execution. + * + * @return True if the parameter selected for putting data at-execution. + */ + bool IsParameterSelected() const; + + /** + * Get parameter by index. + * + * @param idx Index. + * @return Parameter or null, if parameter is not bound. + */ + Parameter* GetParameter(uint16_t idx); + + /** + * Get selected parameter. + * + * @return Parameter or null, if parameter is not bound. + */ + Parameter* GetSelectedParameter(); + + /** + * Internally selects next parameter for putting data at-execution. + * + * @return Parameter if found and null otherwise. + */ + Parameter* SelectNextParameter(); + + /** + * Write only first row of the param set using provided writer. + * @param writer Writer. + */ + void Write(impl::binary::BinaryWriterImpl& writer) const; + + /** + * Write rows of the param set in interval [begin, end) using provided writer. + * @param writer Writer. + * @param begin Beginng of the interval. + * @param end End of the interval. + * @param last Last page flag. + */ + void Write(impl::binary::BinaryWriterImpl& writer, SqlUlen begin, SqlUlen end, bool last) const; + + /** + * Calculate row length. + * + * @return Row length. + */ + int32_t CalculateRowLen() const; + + /** + * Get parameter set size. + * + * @return Number of rows in set. + */ + int32_t GetParamSetSize() const; + + /** + * Set number of parameters processed in batch. + * + * @param processed Processed. + */ + void SetParamsProcessed(SqlUlen processed) const; + + /** + * Number of processed params should be written using provided address. + * + * @param ptr Pointer. + */ + void SetParamsProcessedPtr(SqlUlen* ptr); + + /** + * Get pointer to write number of parameters processed in batch. + * + * @return Pointer to write number of parameters processed in batch. + */ + SqlUlen* GetParamsProcessedPtr(); + + private: + /** + * Write single row of the param set using provided writer. + * @param writer Writer. + * @param idx Row index. + */ + void WriteRow(impl::binary::BinaryWriterImpl& writer, SqlUlen idx) const; + + IGNITE_NO_COPY_ASSIGNMENT(ParameterSet); + + /** Parameters. */ + ParameterBindingMap parameters; + + /** Parameter types. */ + ParameterTypeVector paramTypes; + + /** Offset added to pointers to change binding of parameters. */ + int* paramBindOffset; + + /** Processed parameters. */ + SqlUlen* processedParamRows; + + /** Parameter set size. */ + SqlUlen paramSetSize; + + /** Current position in parametor set. */ + SqlUlen paramSetPos; + + /** Index of the parameter, which is currently being set. */ + uint16_t currentParamIdx; + + /** Parameter types are set. */ + bool typesSet; + }; + } + } +} + +#endif //_IGNITE_ODBC_APP_PARAMETER_SET diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h b/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h index 517fe4e0be17d..408816a6d8f54 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h @@ -132,7 +132,13 @@ namespace ignite * The connection timeout period expired before the data source * responded to the request. */ - SQL_STATE_HYT01_CONNECTIOIN_TIMEOUT + SQL_STATE_HYT01_CONNECTIOIN_TIMEOUT, + + /** Invalid buffer type. */ + SQL_STATE_HY003_INVALID_APPLICATION_BUFFER_TYPE, + + /** Invalid parameter type. */ + SQL_STATE_HY105_INVALID_PARAMETER_TYPE }; /** diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnosable.h b/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnosable.h index 6937fcc6f3196..95e4f548c1f9c 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnosable.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnosable.h @@ -45,14 +45,14 @@ namespace ignite * * @return Diagnostic record. */ - virtual const diagnostic::DiagnosticRecordStorage& GetDiagnosticRecords() const = 0; + virtual const DiagnosticRecordStorage& GetDiagnosticRecords() const = 0; /** * Get diagnostic record. * * @return Diagnostic record. */ - virtual diagnostic::DiagnosticRecordStorage& GetDiagnosticRecords() = 0; + virtual DiagnosticRecordStorage& GetDiagnosticRecords() = 0; /** * Add new status record. diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/message.h b/modules/platforms/cpp/odbc/include/ignite/odbc/message.h index a2bbd99adce3c..cecdac67f9d36 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/message.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/message.h @@ -24,11 +24,11 @@ #include "ignite/impl/binary/binary_writer_impl.h" #include "ignite/impl/binary/binary_reader_impl.h" -#include "ignite/odbc/utility.h" #include "ignite/odbc/result_page.h" +#include "ignite/odbc/protocol_version.h" #include "ignite/odbc/meta/column_meta.h" #include "ignite/odbc/meta/table_meta.h" -#include "ignite/odbc/app/parameter.h" +#include "ignite/odbc/app/parameter_set.h" namespace ignite { @@ -48,7 +48,9 @@ namespace ignite REQUEST_TYPE_GET_TABLES_METADATA = 6, - REQUEST_TYPE_GET_PARAMS_METADATA = 7 + REQUEST_TYPE_GET_PARAMS_METADATA = 7, + + REQUEST_TYPE_EXECUTE_SQL_QUERY_BATCH = 8 }; enum ResponseStatus @@ -71,35 +73,18 @@ namespace ignite * @param distributedJoins Distributed joins flag. * @param enforceJoinOrder Enforce join order flag. */ - HandshakeRequest(int64_t version, bool distributedJoins, bool enforceJoinOrder) : - version(version), - distributedJoins(distributedJoins), - enforceJoinOrder(enforceJoinOrder) - { - // No-op. - } + HandshakeRequest(int64_t version, bool distributedJoins, bool enforceJoinOrder); /** * Destructor. */ - ~HandshakeRequest() - { - // No-op. - } + ~HandshakeRequest(); /** * Write request using provided writer. * @param writer Writer. */ - void Write(ignite::impl::binary::BinaryWriterImpl& writer) const - { - writer.WriteInt8(REQUEST_TYPE_HANDSHAKE); - - writer.WriteInt64(version); - - writer.WriteBool(distributedJoins); - writer.WriteBool(enforceJoinOrder); - } + void Write(impl::binary::BinaryWriterImpl& writer) const; private: /** Protocol version. */ @@ -123,65 +108,80 @@ namespace ignite * * @param cache Cache name. * @param sql SQL query. - * @param argsNum Number of arguments. + * @param params Number of arguments. */ - QueryExecuteRequest(const std::string& cache, const std::string& sql, - const app::ParameterBindingMap& params) : - cache(cache), - sql(sql), - params(params) - { - // No-op. - } + QueryExecuteRequest(const std::string& cache, const std::string& sql, const app::ParameterSet& params); /** * Destructor. */ - ~QueryExecuteRequest() - { - // No-op. - } + ~QueryExecuteRequest(); /** * Write request using provided writer. * @param writer Writer. */ - void Write(ignite::impl::binary::BinaryWriterImpl& writer) const - { - writer.WriteInt8(REQUEST_TYPE_EXECUTE_SQL_QUERY); - utility::WriteString(writer, cache); - utility::WriteString(writer, sql); + void Write(impl::binary::BinaryWriterImpl& writer) const; - writer.WriteInt32(static_cast(params.size())); + private: + /** Cache name. */ + std::string cache; - app::ParameterBindingMap::const_iterator i; - uint16_t prev = 0; + /** SQL query. */ + std::string sql; - for (i = params.begin(); i != params.end(); ++i) { - uint16_t current = i->first; + /** Parameters bindings. */ + const app::ParameterSet& params; + }; - while ((current - prev) > 1) { - writer.WriteNull(); - ++prev; - } + /** + * Query execute batch request. + */ + class QueryExecuteBatchtRequest + { + public: + /** + * Constructor. + * + * @param schema Schema. + * @param sql SQL query. + * @param params Query arguments. + * @param begin Beginng of the interval. + * @param end End of the interval. + */ + QueryExecuteBatchtRequest(const std::string& schema, const std::string& sql, + const app::ParameterSet& params, SqlUlen begin, SqlUlen end, bool last); - i->second.Write(writer); + /** + * Destructor. + */ + ~QueryExecuteBatchtRequest(); - prev = current; - } - } + /** + * Write request using provided writer. + * @param writer Writer. + */ + void Write(impl::binary::BinaryWriterImpl& writer) const; private: - /** Cache name. */ - std::string cache; + /** Schema name. */ + std::string schema; /** SQL query. */ std::string sql; /** Parameters bindings. */ - const app::ParameterBindingMap& params; - }; + const app::ParameterSet& params; + /** Beginng of the interval. */ + SqlUlen begin; + + /** End of the interval. */ + SqlUlen end; + + /** Last page flag. */ + bool last; + }; /** * Query close request. @@ -194,28 +194,18 @@ namespace ignite * * @param queryId Query ID. */ - QueryCloseRequest(int64_t queryId) : queryId(queryId) - { - // No-op. - } + QueryCloseRequest(int64_t queryId); /** * Destructor. */ - ~QueryCloseRequest() - { - // No-op. - } + ~QueryCloseRequest(); /** * Write request using provided writer. * @param writer Writer. */ - void Write(ignite::impl::binary::BinaryWriterImpl& writer) const - { - writer.WriteInt8(REQUEST_TYPE_CLOSE_SQL_QUERY); - writer.WriteInt64(queryId); - } + void Write(impl::binary::BinaryWriterImpl& writer) const; private: /** Query ID. */ @@ -234,31 +224,18 @@ namespace ignite * @param queryId Query ID. * @param pageSize Required page size. */ - QueryFetchRequest(int64_t queryId, int32_t pageSize) : - queryId(queryId), - pageSize(pageSize) - { - // No-op. - } + QueryFetchRequest(int64_t queryId, int32_t pageSize); /** * Destructor. */ - ~QueryFetchRequest() - { - // No-op. - } + ~QueryFetchRequest(); /** * Write request using provided writer. * @param writer Writer. */ - void Write(ignite::impl::binary::BinaryWriterImpl& writer) const - { - writer.WriteInt8(REQUEST_TYPE_FETCH_SQL_QUERY); - writer.WriteInt64(queryId); - writer.WriteInt32(pageSize); - } + void Write(impl::binary::BinaryWriterImpl& writer) const; private: /** Query ID. */ @@ -281,34 +258,18 @@ namespace ignite * @param table Table name. * @param column Column name. */ - QueryGetColumnsMetaRequest(const std::string& schema, const std::string& table, const std::string& column) : - schema(schema), - table(table), - column(column) - { - // No-op. - } + QueryGetColumnsMetaRequest(const std::string& schema, const std::string& table, const std::string& column); /** * Destructor. */ - ~QueryGetColumnsMetaRequest() - { - // No-op. - } + ~QueryGetColumnsMetaRequest(); /** * Write request using provided writer. * @param writer Writer. */ - void Write(ignite::impl::binary::BinaryWriterImpl& writer) const - { - writer.WriteInt8(REQUEST_TYPE_GET_COLUMNS_METADATA); - - utility::WriteString(writer, schema); - utility::WriteString(writer, table); - utility::WriteString(writer, column); - } + void Write(impl::binary::BinaryWriterImpl& writer) const; private: /** Schema search pattern. */ @@ -336,36 +297,18 @@ namespace ignite * @param tableTypes Table types search pattern. */ QueryGetTablesMetaRequest(const std::string& catalog, const std::string& schema, - const std::string& table, const std::string& tableTypes) : - catalog(catalog), - schema(schema), - table(table), - tableTypes(tableTypes) - { - // No-op. - } + const std::string& table, const std::string& tableTypes); /** * Destructor. */ - ~QueryGetTablesMetaRequest() - { - // No-op. - } + ~QueryGetTablesMetaRequest(); /** * Write request using provided writer. * @param writer Writer. */ - void Write(ignite::impl::binary::BinaryWriterImpl& writer) const - { - writer.WriteInt8(REQUEST_TYPE_GET_TABLES_METADATA); - - utility::WriteString(writer, catalog); - utility::WriteString(writer, schema); - utility::WriteString(writer, table); - utility::WriteString(writer, tableTypes); - } + void Write(impl::binary::BinaryWriterImpl& writer) const; private: /** Column search pattern. */ @@ -414,13 +357,7 @@ namespace ignite * Write request using provided writer. * @param writer Writer. */ - void Write(ignite::impl::binary::BinaryWriterImpl& writer) const - { - writer.WriteInt8(REQUEST_TYPE_GET_PARAMS_METADATA); - - utility::WriteString(writer, cacheName); - utility::WriteString(writer, sqlQuery); - } + void Write(impl::binary::BinaryWriterImpl& writer) const; private: /** Cache name. */ @@ -439,33 +376,19 @@ namespace ignite /** * Constructor. */ - Response() : status(RESPONSE_STATUS_FAILED), error() - { - // No-op. - } + Response(); /** * Destructor. */ - virtual ~Response() - { - // No-op. - } + virtual ~Response(); /** * Read response using provided reader. * @param reader Reader. */ - void Read(ignite::impl::binary::BinaryReaderImpl& reader) - { - status = reader.ReadInt8(); + void Read(impl::binary::BinaryReaderImpl& reader); - if (status == RESPONSE_STATUS_SUCCESS) - ReadOnSuccess(reader); - else - utility::ReadString(reader, error);; - } - /** * Get request processing status. * @return Status. @@ -488,10 +411,7 @@ namespace ignite /** * Read data if response status is RESPONSE_STATUS_SUCCESS. */ - virtual void ReadOnSuccess(ignite::impl::binary::BinaryReaderImpl&) - { - // No-op. - } + virtual void ReadOnSuccess(impl::binary::BinaryReaderImpl&); private: /** Request processing status. */ @@ -510,21 +430,12 @@ namespace ignite /** * Constructor. */ - HandshakeResponse() : - accepted(false), - protoVerSince(), - currentVer() - { - // No-op. - } + HandshakeResponse(); /** * Destructor. */ - ~HandshakeResponse() - { - // No-op. - } + ~HandshakeResponse(); /** * Check if the handshake has been accepted. @@ -558,16 +469,7 @@ namespace ignite * Read response using provided reader. * @param reader Reader. */ - virtual void ReadOnSuccess(ignite::impl::binary::BinaryReaderImpl& reader) - { - accepted = reader.ReadBool(); - - if (!accepted) - { - utility::ReadString(reader, protoVerSince); - utility::ReadString(reader, currentVer); - } - } + virtual void ReadOnSuccess(impl::binary::BinaryReaderImpl& reader); /** Handshake accepted. */ bool accepted; @@ -588,18 +490,12 @@ namespace ignite /** * Constructor. */ - QueryCloseResponse() : queryId(0) - { - // No-op. - } + QueryCloseResponse(); /** * Destructor. */ - ~QueryCloseResponse() - { - // No-op. - } + virtual ~QueryCloseResponse(); /** * Get query ID. @@ -615,10 +511,7 @@ namespace ignite * Read response using provided reader. * @param reader Reader. */ - virtual void ReadOnSuccess(ignite::impl::binary::BinaryReaderImpl& reader) - { - queryId = reader.ReadInt64(); - } + virtual void ReadOnSuccess(impl::binary::BinaryReaderImpl& reader); /** Query ID. */ int64_t queryId; @@ -633,18 +526,12 @@ namespace ignite /** * Constructor. */ - QueryExecuteResponse() : queryId(0), meta() - { - // No-op. - } + QueryExecuteResponse(); /** * Destructor. */ - ~QueryExecuteResponse() - { - // No-op. - } + virtual ~QueryExecuteResponse(); /** * Get query ID. @@ -669,12 +556,7 @@ namespace ignite * Read response using provided reader. * @param reader Reader. */ - virtual void ReadOnSuccess(ignite::impl::binary::BinaryReaderImpl& reader) - { - queryId = reader.ReadInt64(); - - meta::ReadColumnMetaVector(reader, meta); - } + virtual void ReadOnSuccess(impl::binary::BinaryReaderImpl& reader); /** Query ID. */ int64_t queryId; @@ -684,28 +566,82 @@ namespace ignite }; /** - * Query fetch response. + * Query execute batch start response. */ - class QueryFetchResponse : public Response + class QueryExecuteBatchResponse : public Response { public: /** * Constructor. - * @param resultPage Result page. */ - QueryFetchResponse(ResultPage& resultPage) : queryId(0), resultPage(resultPage) + QueryExecuteBatchResponse(); + + /** + * Destructor. + */ + virtual ~QueryExecuteBatchResponse(); + + /** + * Affected rows. + * @return Affected rows. + */ + int64_t GetAffectedRows() const { - // No-op. + return affectedRows; } /** - * Destructor. + * Get index of the set which caused an error. + * @return Index of the set which caused an error. */ - ~QueryFetchResponse() + int64_t GetErrorSetIdx() const { - // No-op. + return affectedRows; + } + + /** + * Get error message. + * @return Error message. + */ + const std::string& GetErrorMessage() const + { + return errorMessage; } + private: + /** + * Read response using provided reader. + * @param reader Reader. + */ + virtual void ReadOnSuccess(impl::binary::BinaryReaderImpl& reader); + + /** Affected rows. */ + int64_t affectedRows; + + /** Index of the set which caused an error. */ + int64_t errorSetIdx; + + /** Error message. */ + std::string errorMessage; + }; + + /** + * Query fetch response. + */ + class QueryFetchResponse : public Response + { + public: + /** + * Constructor. + * @param resultPage Result page. + */ + QueryFetchResponse(ResultPage& resultPage); + + /** + * Destructor. + */ + virtual ~QueryFetchResponse(); + /** * Get query ID. * @return Query ID. @@ -720,12 +656,7 @@ namespace ignite * Read response using provided reader. * @param reader Reader. */ - virtual void ReadOnSuccess(ignite::impl::binary::BinaryReaderImpl& reader) - { - queryId = reader.ReadInt64(); - - resultPage.Read(reader); - } + virtual void ReadOnSuccess(impl::binary::BinaryReaderImpl& reader); /** Query ID. */ int64_t queryId; @@ -743,18 +674,12 @@ namespace ignite /** * Constructor. */ - QueryGetColumnsMetaResponse() - { - // No-op. - } + QueryGetColumnsMetaResponse(); /** * Destructor. */ - ~QueryGetColumnsMetaResponse() - { - // No-op. - } + virtual ~QueryGetColumnsMetaResponse(); /** * Get column metadata. @@ -770,10 +695,7 @@ namespace ignite * Read response using provided reader. * @param reader Reader. */ - virtual void ReadOnSuccess(ignite::impl::binary::BinaryReaderImpl& reader) - { - meta::ReadColumnMetaVector(reader, meta); - } + virtual void ReadOnSuccess(impl::binary::BinaryReaderImpl& reader); /** Columns metadata. */ meta::ColumnMetaVector meta; @@ -788,18 +710,12 @@ namespace ignite /** * Constructor. */ - QueryGetTablesMetaResponse() - { - // No-op. - } + QueryGetTablesMetaResponse(); /** * Destructor. */ - ~QueryGetTablesMetaResponse() - { - // No-op. - } + virtual ~QueryGetTablesMetaResponse(); /** * Get column metadata. @@ -815,10 +731,7 @@ namespace ignite * Read response using provided reader. * @param reader Reader. */ - virtual void ReadOnSuccess(ignite::impl::binary::BinaryReaderImpl& reader) - { - meta::ReadTableMetaVector(reader, meta); - } + virtual void ReadOnSuccess(impl::binary::BinaryReaderImpl& reader); /** Columns metadata. */ meta::TableMetaVector meta; @@ -833,18 +746,12 @@ namespace ignite /** * Constructor. */ - QueryGetParamsMetaResponse() - { - // No-op. - } + QueryGetParamsMetaResponse(); /** * Destructor. */ - ~QueryGetParamsMetaResponse() - { - // No-op. - } + virtual ~QueryGetParamsMetaResponse(); /** * Get parameter type IDs. @@ -860,10 +767,7 @@ namespace ignite * Read response using provided reader. * @param reader Reader. */ - virtual void ReadOnSuccess(ignite::impl::binary::BinaryReaderImpl& reader) - { - utility::ReadByteArray(reader, typeIds); - } + virtual void ReadOnSuccess(impl::binary::BinaryReaderImpl& reader); /** Columns metadata. */ std::vector typeIds; diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/query/batch_query.h b/modules/platforms/cpp/odbc/include/ignite/odbc/query/batch_query.h new file mode 100644 index 0000000000000..a691f7351d956 --- /dev/null +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/query/batch_query.h @@ -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. + */ + +#ifndef _IGNITE_ODBC_QUERY_BATCH_QUERY +#define _IGNITE_ODBC_QUERY_BATCH_QUERY + +#include "ignite/odbc/query/query.h" +#include "ignite/odbc/app/parameter_set.h" +#include "ignite/odbc/cursor.h" + +namespace ignite +{ + namespace odbc + { + /** Connection forward-declaration. */ + class Connection; + + namespace query + { + /** + * Query. + */ + class BatchQuery : public Query + { + public: + /** + * Constructor. + * + * @param diag Diagnostics collector. + * @param connection Associated connection. + * @param sql SQL query string. + * @param params SQL params. + */ + BatchQuery(diagnostic::Diagnosable& diag, Connection& connection, + const std::string& sql, const app::ParameterSet& params); + + /** + * Destructor. + */ + virtual ~BatchQuery(); + + /** + * Execute query. + * + * @return True on success. + */ + virtual SqlResult Execute(); + + /** + * Get column metadata. + * + * @return Column metadata. + */ + virtual const meta::ColumnMetaVector& GetMeta() const; + + /** + * Fetch next result row to application buffers. + * + * @param columnBindings Application buffers to put data to. + * @return Operation result. + */ + virtual SqlResult FetchNextRow(app::ColumnBindingMap& columnBindings); + + /** + * Get data of the specified column in the result set. + * + * @param columnIdx Column index. + * @param buffer Buffer to put column data to. + * @return Operation result. + */ + virtual SqlResult GetColumn(uint16_t columnIdx, app::ApplicationDataBuffer& buffer); + + /** + * Close query. + * + * @return Result. + */ + virtual SqlResult Close(); + + /** + * Check if data is available. + * + * @return True if data is available. + */ + virtual bool DataAvailable() const; + + /** + * Get number of rows affected by the statement. + * + * @return Number of rows affected by the statement. + */ + virtual int64_t AffectedRows() const; + + /** + * Get SQL query string. + * + * @return SQL query string. + */ + const std::string& GetSql() const + { + return sql; + } + + private: + IGNITE_NO_COPY_ASSIGNMENT(BatchQuery); + + /** + * Make query execute request and use response to set internal + * state. + * + * @param begin Paramset interval beginning. + * @param end Paramset interval end. + * @param last Last page flag. + * @return Result. + */ + SqlResult MakeRequestExecuteBatch(SqlUlen begin, SqlUlen end, bool last); + + /** Connection associated with the statement. */ + Connection& connection; + + /** SQL Query. */ + std::string sql; + + /** Parameter bindings. */ + const app::ParameterSet& params; + + /** Columns metadata. */ + meta::ColumnMetaVector resultMeta; + + /** Number of rows affected. */ + int64_t rowsAffected; + + /** Number of parameter sets successfully processed. */ + int64_t setsProcessed; + + /** Query executed. */ + bool executed; + + /** Data retrieved. */ + bool dataRetrieved; + }; + } + } +} + +#endif //_IGNITE_ODBC_QUERY_BATCH_QUERY diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/query/data_query.h b/modules/platforms/cpp/odbc/include/ignite/odbc/query/data_query.h index 68bb8776417c8..0424bf8364a6d 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/query/data_query.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/query/data_query.h @@ -19,7 +19,7 @@ #define _IGNITE_ODBC_QUERY_DATA_QUERY #include "ignite/odbc/query/query.h" -#include "ignite/odbc/app/parameter.h" +#include "ignite/odbc/app/parameter_set.h" #include "ignite/odbc/cursor.h" namespace ignite @@ -46,7 +46,7 @@ namespace ignite * @param params SQL params. */ DataQuery(diagnostic::Diagnosable& diag, Connection& connection, - const std::string& sql, const app::ParameterBindingMap& params); + const std::string& sql, const app::ParameterSet& params); /** * Destructor. @@ -122,24 +122,32 @@ namespace ignite * Make query execute request and use response to set internal * state. * - * @return True on success. + * @return Result. */ SqlResult MakeRequestExecute(); /** * Make query close request. * - * @return True on success. + * @return Result. */ SqlResult MakeRequestClose(); /** * Make data fetch request and use response to set internal state. * - * @return True on success. + * @return Result. */ SqlResult MakeRequestFetch(); + /** + * Close query. + * Non-virtual implementation. + * + * @return True on success. + */ + SqlResult InternalClose(); + /** Connection associated with the statement. */ Connection& connection; @@ -147,7 +155,7 @@ namespace ignite std::string sql; /** Parameter bindings. */ - const app::ParameterBindingMap& params; + const app::ParameterSet& params; /** Columns metadata. */ meta::ColumnMetaVector resultMeta; diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/query/query.h b/modules/platforms/cpp/odbc/include/ignite/odbc/query/query.h index 40be1ed4c8bd6..d8103af182a30 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/query/query.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/query/query.h @@ -47,6 +47,9 @@ namespace ignite /** Data query type. */ DATA, + /** Batch query type. */ + BATCH, + /** Foreign keys query type. */ FOREIGN_KEYS, diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/query/type_info_query.h b/modules/platforms/cpp/odbc/include/ignite/odbc/query/type_info_query.h index d337d0311b972..ee37bb012bb33 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/query/type_info_query.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/query/type_info_query.h @@ -95,7 +95,7 @@ namespace ignite * @return Number of rows affected by the statement. */ virtual int64_t AffectedRows() const; - + private: IGNITE_NO_COPY_ASSIGNMENT(TypeInfoQuery); diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h b/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h index 1ee56197ce81b..170f59998e461 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h @@ -23,19 +23,12 @@ #include #include -#include -#include -#include - #include "ignite/odbc/meta/column_meta.h" -#include "ignite/odbc/meta/table_meta.h" #include "ignite/odbc/query/query.h" #include "ignite/odbc/app/application_data_buffer.h" -#include "ignite/odbc/app/parameter.h" +#include "ignite/odbc/app/parameter_set.h" #include "ignite/odbc/diagnostic/diagnosable_adapter.h" #include "ignite/odbc/common_types.h" -#include "ignite/odbc/cursor.h" -#include "ignite/odbc/utility.h" namespace ignite { @@ -96,14 +89,23 @@ namespace ignite * @return Columns number. */ int32_t GetColumnNumber(); - + /** * Bind parameter. * * @param paramIdx Parameter index. - * @param param Parameter. + * @param ioType Type of the parameter (input/output). + * @param bufferType The data type of the parameter. + * @param paramSqlType The SQL data type of the parameter. + * @param columnSize The size of the column or expression of the corresponding parameter marker. + * @param decDigits The decimal digits of the column or expression of the corresponding parameter marker. + * @param buffer A pointer to a buffer for the parameter's data. + * @param bufferLen Length of the ParameterValuePtr buffer in bytes. + * @param resLen A pointer to a buffer for the parameter's length. + * @return Operation result. */ - void BindParameter(uint16_t paramIdx, const app::Parameter& param); + void BindParameter(uint16_t paramIdx, int16_t ioType, int16_t bufferType, int16_t paramSqlType, + SqlUlen columnSize, int16_t decDigits, void* buffer, SqlLen bufferLen, SqlLen* resLen); /** * Unbind specified parameter. @@ -150,13 +152,6 @@ namespace ignite */ void SetParamBindOffsetPtr(int* ptr); - /** - * Get parameter binding offset pointer. - * - * @return Parameter binding offset pointer. - */ - int* GetParamBindOffsetPtr(); - /** * Get value of the column in the result set. * @@ -171,7 +166,7 @@ namespace ignite * @param query SQL query. */ void PrepareSqlQuery(const std::string& query); - + /** * Execute SQL query. * @@ -366,10 +361,18 @@ namespace ignite * Bind parameter. * * @param paramIdx Parameter index. - * @param param Parameter. + * @param ioType Type of the parameter (input/output). + * @param bufferType The data type of the parameter. + * @param paramSqlType The SQL data type of the parameter. + * @param columnSize The size of the column or expression of the corresponding parameter marker. + * @param decDigits The decimal digits of the column or expression of the corresponding parameter marker. + * @param buffer A pointer to a buffer for the parameter's data. + * @param bufferLen Length of the ParameterValuePtr buffer in bytes. + * @param resLen A pointer to a buffer for the parameter's length. * @return Operation result. */ - SqlResult InternalBindParameter(uint16_t paramIdx, const app::Parameter& param); + SqlResult InternalBindParameter(uint16_t paramIdx, int16_t ioType, int16_t bufferType, int16_t paramSqlType, + SqlUlen columnSize, int16_t decDigits, void* buffer, SqlLen bufferLen, SqlLen* resLen); /** * Set statement attribute. @@ -425,7 +428,7 @@ namespace ignite * @return Operation result. */ SqlResult InternalPrepareSqlQuery(const std::string& query); - + /** * Execute SQL query. * @@ -611,12 +614,6 @@ namespace ignite /** Column bindings. */ app::ColumnBindingMap columnBindings; - /** Parameter bindings. */ - app::ParameterBindingMap paramBindings; - - /** Parameter meta. */ - std::vector paramTypes; - /** Underlying query. */ std::auto_ptr currentQuery; @@ -626,14 +623,10 @@ namespace ignite /** Array to store statuses of rows fetched by the last fetch. */ uint16_t* rowStatuses; - /** Offset added to pointers to change binding of parameters. */ - int* paramBindOffset; - /** Offset added to pointers to change binding of column data. */ int* columnBindOffset; - /** Index of the parameter, which is currently being set. */ - uint16_t currentParamIdx; + app::ParameterSet parameters; }; } } diff --git a/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj b/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj index 56358c5f8bf4f..280665984e50f 100644 --- a/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj +++ b/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj @@ -160,6 +160,7 @@ + @@ -172,10 +173,12 @@ + + @@ -196,6 +199,7 @@ + @@ -214,6 +218,7 @@ + diff --git a/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj.filters b/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj.filters index 58764e4043871..9aefc0c616ea1 100644 --- a/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj.filters +++ b/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj.filters @@ -136,6 +136,15 @@ Code + + Code\app + + + Code\query + + + Code + @@ -260,5 +269,11 @@ Code + + Code\app + + + Code\query + \ No newline at end of file diff --git a/modules/platforms/cpp/odbc/src/app/application_data_buffer.cpp b/modules/platforms/cpp/odbc/src/app/application_data_buffer.cpp index 71c5f39025796..c13857e815c51 100644 --- a/modules/platforms/cpp/odbc/src/app/application_data_buffer.cpp +++ b/modules/platforms/cpp/odbc/src/app/application_data_buffer.cpp @@ -33,35 +33,38 @@ namespace ignite { namespace app { - using ignite::impl::binary::BinaryUtils; + using impl::binary::BinaryUtils; ApplicationDataBuffer::ApplicationDataBuffer() : type(type_traits::IGNITE_ODBC_C_TYPE_UNSUPPORTED), buffer(0), buflen(0), reslen(0), - offset(0) + byteOffset(0), + elementOffset(0) { // No-op. } - ApplicationDataBuffer::ApplicationDataBuffer(type_traits::IgniteSqlType type, - void* buffer, SqlLen buflen, SqlLen* reslen, int** offset) : + ApplicationDataBuffer::ApplicationDataBuffer(type_traits::IgniteSqlType type, + void* buffer, SqlLen buflen, SqlLen* reslen) : type(type), buffer(buffer), buflen(buflen), reslen(reslen), - offset(offset) + byteOffset(0), + elementOffset(0) { // No-op. } - ApplicationDataBuffer::ApplicationDataBuffer(const ApplicationDataBuffer & other) : + ApplicationDataBuffer::ApplicationDataBuffer(const ApplicationDataBuffer& other) : type(other.type), buffer(other.buffer), buflen(other.buflen), reslen(other.reslen), - offset(other.offset) + byteOffset(other.byteOffset), + elementOffset(other.elementOffset) { // No-op. } @@ -77,7 +80,8 @@ namespace ignite buffer = other.buffer; buflen = other.buflen; reslen = other.reslen; - offset = other.offset; + byteOffset = other.byteOffset; + elementOffset = other.elementOffset; return *this; } @@ -1072,22 +1076,22 @@ namespace ignite const void* ApplicationDataBuffer::GetData() const { - return ApplyOffset(buffer); + return ApplyOffset(buffer, GetElementSize()); } const SqlLen* ApplicationDataBuffer::GetResLen() const { - return ApplyOffset(reslen); + return ApplyOffset(reslen, sizeof(*reslen)); } - void* ApplicationDataBuffer::GetData() + void* ApplicationDataBuffer::GetData() { - return ApplyOffset(buffer); + return ApplyOffset(buffer, GetElementSize()); } SqlLen* ApplicationDataBuffer::GetResLen() { - return ApplyOffset(reslen); + return ApplyOffset(reslen, sizeof(*reslen)); } template @@ -1407,12 +1411,12 @@ namespace ignite } template - T* ApplicationDataBuffer::ApplyOffset(T* ptr) const + T* ApplicationDataBuffer::ApplyOffset(T* ptr, size_t elemSize) const { - if (!ptr || !offset || !*offset) + if (!ptr) return ptr; - return utility::GetPointerWithOffset(ptr, **offset); + return utility::GetPointerWithOffset(ptr, byteOffset + elemSize * elementOffset); } bool ApplicationDataBuffer::IsDataAtExec() const @@ -1502,6 +1506,64 @@ namespace ignite return 0; } + SqlLen ApplicationDataBuffer::GetElementSize() const + { + using namespace type_traits; + + switch (type) + { + case IGNITE_ODBC_C_TYPE_WCHAR: + case IGNITE_ODBC_C_TYPE_CHAR: + case IGNITE_ODBC_C_TYPE_BINARY: + return buflen; + + case IGNITE_ODBC_C_TYPE_SIGNED_SHORT: + case IGNITE_ODBC_C_TYPE_UNSIGNED_SHORT: + return static_cast(sizeof(short)); + + case IGNITE_ODBC_C_TYPE_SIGNED_LONG: + case IGNITE_ODBC_C_TYPE_UNSIGNED_LONG: + return static_cast(sizeof(long)); + + case IGNITE_ODBC_C_TYPE_FLOAT: + return static_cast(sizeof(float)); + + case IGNITE_ODBC_C_TYPE_DOUBLE: + return static_cast(sizeof(double)); + + case IGNITE_ODBC_C_TYPE_BIT: + case IGNITE_ODBC_C_TYPE_SIGNED_TINYINT: + case IGNITE_ODBC_C_TYPE_UNSIGNED_TINYINT: + return static_cast(sizeof(char)); + + case IGNITE_ODBC_C_TYPE_SIGNED_BIGINT: + case IGNITE_ODBC_C_TYPE_UNSIGNED_BIGINT: + return static_cast(sizeof(SQLBIGINT)); + + case IGNITE_ODBC_C_TYPE_TDATE: + return static_cast(sizeof(SQL_DATE_STRUCT)); + + case IGNITE_ODBC_C_TYPE_TTIME: + return static_cast(sizeof(SQL_TIME_STRUCT)); + + case IGNITE_ODBC_C_TYPE_TTIMESTAMP: + return static_cast(sizeof(SQL_TIMESTAMP_STRUCT)); + + case IGNITE_ODBC_C_TYPE_NUMERIC: + return static_cast(sizeof(SQL_NUMERIC_STRUCT)); + + case IGNITE_ODBC_C_TYPE_GUID: + return static_cast(sizeof(SQLGUID)); + + case IGNITE_ODBC_C_TYPE_DEFAULT: + case IGNITE_ODBC_C_TYPE_UNSUPPORTED: + default: + break; + } + + return 0; + } + SqlLen ApplicationDataBuffer::GetInputSize() const { if (!IsDataAtExec()) diff --git a/modules/platforms/cpp/odbc/src/app/parameter.cpp b/modules/platforms/cpp/odbc/src/app/parameter.cpp index 937ef58e7c6c3..5ee132c1b7107 100644 --- a/modules/platforms/cpp/odbc/src/app/parameter.cpp +++ b/modules/platforms/cpp/odbc/src/app/parameter.cpp @@ -16,8 +16,6 @@ */ #include -#include -#include #include "ignite/odbc/system/odbc_constants.h" #include "ignite/odbc/app/parameter.h" @@ -78,7 +76,7 @@ namespace ignite return *this; } - void Parameter::Write(ignite::impl::binary::BinaryWriterImpl& writer) const + void Parameter::Write(impl::binary::BinaryWriterImpl& writer, int offset, SqlUlen idx) const { if (buffer.GetInputSize() == SQL_NULL_DATA) { @@ -89,6 +87,8 @@ namespace ignite // Buffer to use to get data. ApplicationDataBuffer buf(buffer); + buf.SetByteOffset(offset); + buf.SetElementOffset(idx); SqlLen storedDataLen = static_cast(storedData.size()); @@ -150,12 +150,14 @@ namespace ignite break; } + case SQL_TYPE_DATE: case SQL_DATE: { writer.WriteDate(buf.GetDate()); break; } + case SQL_TYPE_TIMESTAMP: case SQL_TIMESTAMP: { writer.WriteTimestamp(buf.GetTimestamp()); @@ -207,6 +209,11 @@ namespace ignite return buffer; } + const ApplicationDataBuffer& Parameter::GetBuffer() const + { + return buffer; + } + void Parameter::ResetStoredData() { storedData.clear(); diff --git a/modules/platforms/cpp/odbc/src/app/parameter_set.cpp b/modules/platforms/cpp/odbc/src/app/parameter_set.cpp new file mode 100644 index 0000000000000..c110d05dc5b38 --- /dev/null +++ b/modules/platforms/cpp/odbc/src/app/parameter_set.cpp @@ -0,0 +1,242 @@ +/* + * 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. + */ + +#include "ignite/odbc/app/parameter_set.h" + +namespace ignite +{ + namespace odbc + { + namespace app + { + ParameterSet::ParameterSet(): + parameters(), + paramTypes(), + paramBindOffset(0), + processedParamRows(0), + paramSetSize(1), + paramSetPos(0), + currentParamIdx(0), + typesSet(false) + { + // No-op. + } + + void ParameterSet::SetParamSetSize(SqlUlen size) + { + paramSetSize = size; + } + + void ParameterSet::BindParameter(uint16_t paramIdx, const Parameter& param) + { + parameters[paramIdx] = param; + } + + void ParameterSet::UnbindParameter(uint16_t paramIdx) + { + parameters.erase(paramIdx); + } + + void ParameterSet::UnbindAll() + { + parameters.clear(); + } + + uint16_t ParameterSet::GetParametersNumber() const + { + return static_cast(parameters.size()); + } + + void ParameterSet::SetParamBindOffsetPtr(int* ptr) + { + paramBindOffset = ptr; + } + + int* ParameterSet::GetParamBindOffsetPtr() + { + return paramBindOffset; + } + + void ParameterSet::Prepare() + { + paramTypes.clear(); + + typesSet = false; + + paramSetPos = 0; + + for (ParameterBindingMap::iterator it = parameters.begin(); it != parameters.end(); ++it) + it->second.ResetStoredData(); + } + + bool ParameterSet::IsDataAtExecNeeded() const + { + for (ParameterBindingMap::const_iterator it = parameters.begin(); it != parameters.end(); ++it) + { + if (!it->second.IsDataReady()) + return true; + } + + return false; + } + + void ParameterSet::SetParamsProcessedPtr(SqlUlen* ptr) + { + processedParamRows = ptr; + } + + SqlUlen* ParameterSet::GetParamsProcessedPtr() + { + return processedParamRows; + } + + void ParameterSet::SetParamsProcessed(SqlUlen processed) const + { + if (processedParamRows) + *processedParamRows = processed; + } + + void ParameterSet::UpdateParamsTypes(const ParameterTypeVector& meta) + { + paramTypes = meta; + + typesSet = true; + } + + int8_t ParameterSet::GetParamType(int16_t idx, int8_t dflt) + { + if (idx > 0 && static_cast(idx) <= paramTypes.size()) + return paramTypes[idx - 1]; + + return dflt; + } + + uint16_t ParameterSet::GetExpectedParamNum() + { + return static_cast(paramTypes.size()); + } + + bool ParameterSet::IsMetadataSet() const + { + return typesSet; + } + + bool ParameterSet::IsParameterSelected() const + { + return currentParamIdx != 0; + } + + Parameter* ParameterSet::GetParameter(uint16_t idx) + { + ParameterBindingMap::iterator it = parameters.find(currentParamIdx); + + if (it != parameters.end()) + return &it->second; + + return 0; + } + + Parameter* ParameterSet::GetSelectedParameter() + { + return GetParameter(currentParamIdx); + } + + Parameter* ParameterSet::SelectNextParameter() + { + for (ParameterBindingMap::iterator it = parameters.begin(); it != parameters.end(); ++it) + { + uint16_t paramIdx = it->first; + Parameter& param = it->second; + + if (!param.IsDataReady()) + { + currentParamIdx = paramIdx; + + return ¶m; + } + } + + return 0; + } + + void ParameterSet::Write(impl::binary::BinaryWriterImpl& writer) const + { + writer.WriteInt32(CalculateRowLen()); + + WriteRow(writer, 0); + } + + void ParameterSet::Write(impl::binary::BinaryWriterImpl& writer, SqlUlen begin, SqlUlen end, bool last) const + { + int32_t rowLen = CalculateRowLen(); + + writer.WriteInt32(rowLen); + + SqlUlen intervalEnd = std::min(paramSetSize, end); + + assert(begin < intervalEnd); + + int32_t intervalLen = static_cast(intervalEnd - begin); + + writer.WriteInt32(intervalLen); + writer.WriteBool(last); + + if (rowLen) + { + for (SqlUlen i = begin; i < intervalEnd; ++i) + WriteRow(writer, i); + } + } + + void ParameterSet::WriteRow(impl::binary::BinaryWriterImpl& writer, SqlUlen idx) const + { + uint16_t prev = 0; + + int appOffset = paramBindOffset ? *paramBindOffset : 0; + + for (ParameterBindingMap::const_iterator it = parameters.begin(); it != parameters.end(); ++it) + { + uint16_t paramIdx = it->first; + const Parameter& param = it->second; + + while ((paramIdx - prev) > 1) + { + writer.WriteNull(); + ++prev; + } + + param.Write(writer, appOffset, idx); + + prev = paramIdx; + } + } + + int32_t ParameterSet::CalculateRowLen() const + { + if (!parameters.empty()) + return static_cast(parameters.rbegin()->first); + + return 0; + } + + int32_t ParameterSet::GetParamSetSize() const + { + return static_cast(paramSetSize); + } + } + } +} diff --git a/modules/platforms/cpp/odbc/src/config/connection_info.cpp b/modules/platforms/cpp/odbc/src/config/connection_info.cpp index 341ab7fea2863..6fe03f24dcf40 100644 --- a/modules/platforms/cpp/odbc/src/config/connection_info.cpp +++ b/modules/platforms/cpp/odbc/src/config/connection_info.cpp @@ -29,11 +29,11 @@ #ifndef SQL_ASYNC_NOTIFICATION_NOT_CAPABLE #define SQL_ASYNC_NOTIFICATION_NOT_CAPABLE 0x00000000L -#endif +#endif #ifndef SQL_ASYNC_NOTIFICATION_CAPABLE #define SQL_ASYNC_NOTIFICATION_CAPABLE 0x00000001L -#endif +#endif namespace ignite { @@ -95,7 +95,9 @@ namespace ignite DBG_STR_CASE(SQL_SQL92_VALUE_EXPRESSIONS); DBG_STR_CASE(SQL_STATIC_CURSOR_ATTRIBUTES1); DBG_STR_CASE(SQL_STATIC_CURSOR_ATTRIBUTES2); - default: + DBG_STR_CASE(SQL_PARAM_ARRAY_ROW_COUNTS); + DBG_STR_CASE(SQL_PARAM_ARRAY_SELECTS); + default: break; } return "<< UNKNOWN TYPE >>"; @@ -117,7 +119,7 @@ namespace ignite strParams[SQL_DBMS_VER] = "03.00"; #ifdef SQL_DRIVER_VER - // Driver version. At a minimum, the version is of the form + // Driver version. At a minimum, the version is of the form // ##.##.####, where the first two digits are the major version, // the next two digits are the minor version, and the last four // digits are the release version. @@ -125,7 +127,7 @@ namespace ignite #endif // SQL_DRIVER_VER #ifdef SQL_COLUMN_ALIAS - // A character string: "Y" if the data source supports column + // A character string: "Y" if the data source supports column // aliases; otherwise, "N". strParams[SQL_COLUMN_ALIAS] = "Y"; #endif // SQL_COLUMN_ALIAS @@ -168,7 +170,7 @@ namespace ignite #endif // SQL_TABLE_TERM #ifdef SQL_SCHEMA_TERM - // A character string with the data source vendor's name for + // A character string with the data source vendor's name for // a schema; for example, "owner", "Authorization ID", or "Schema". strParams[SQL_SCHEMA_TERM] = "schema"; #endif // SQL_SCHEMA_TERM @@ -194,9 +196,9 @@ namespace ignite #ifdef SQL_ASYNC_NOTIFICATION // Indicates if the driver supports asynchronous notification. - // SQL_ASYNC_NOTIFICATION_CAPABLE = Asynchronous execution + // SQL_ASYNC_NOTIFICATION_CAPABLE = Asynchronous execution // notification is supported by the driver. - // SQL_ASYNC_NOTIFICATION_NOT_CAPABLE Asynchronous execution + // SQL_ASYNC_NOTIFICATION_NOT_CAPABLE Asynchronous execution // notification is not supported by the driver. intParams[SQL_ASYNC_NOTIFICATION] = SQL_ASYNC_NOTIFICATION_NOT_CAPABLE; #endif // SQL_ASYNC_NOTIFICATION @@ -207,7 +209,7 @@ namespace ignite #endif // SQL_GETDATA_EXTENSIONS #ifdef SQL_ODBC_INTERFACE_CONFORMANCE - // Indicates the level of the ODBC 3.x interface that the driver + // Indicates the level of the ODBC 3.x interface that the driver // complies with. intParams[SQL_ODBC_INTERFACE_CONFORMANCE] = SQL_OIC_CORE; #endif // SQL_ODBC_INTERFACE_CONFORMANCE @@ -229,7 +231,7 @@ namespace ignite #endif // SQL_SCHEMA_USAGE #ifdef SQL_MAX_IDENTIFIER_LEN - // Indicates the maximum size in characters that the data source + // Indicates the maximum size in characters that the data source // supports for user-defined names. intParams[SQL_MAX_IDENTIFIER_LEN] = 128; #endif // SQL_MAX_IDENTIFIER_LEN @@ -243,7 +245,7 @@ namespace ignite #ifdef SQL_NUMERIC_FUNCTIONS // Bitmask enumerating the scalar numeric functions supported by // the driver and associated data source. - intParams[SQL_NUMERIC_FUNCTIONS] = SQL_FN_NUM_ABS | SQL_FN_NUM_ACOS | SQL_FN_NUM_ASIN | + intParams[SQL_NUMERIC_FUNCTIONS] = SQL_FN_NUM_ABS | SQL_FN_NUM_ACOS | SQL_FN_NUM_ASIN | SQL_FN_NUM_ATAN | SQL_FN_NUM_ATAN2 | SQL_FN_NUM_CEILING | SQL_FN_NUM_COS | SQL_FN_NUM_COT | SQL_FN_NUM_EXP | SQL_FN_NUM_FLOOR | SQL_FN_NUM_LOG | SQL_FN_NUM_MOD | SQL_FN_NUM_SIGN | SQL_FN_NUM_SIN | SQL_FN_NUM_SQRT | SQL_FN_NUM_TAN | SQL_FN_NUM_PI | SQL_FN_NUM_RAND | @@ -273,7 +275,7 @@ namespace ignite #endif // SQL_TIMEDATE_FUNCTIONS #ifdef SQL_TIMEDATE_ADD_INTERVALS - // Bitmask enumerating timestamp intervals supported by the driver + // Bitmask enumerating timestamp intervals supported by the driver // and associated data source for the TIMESTAMPADD scalar function. intParams[SQL_TIMEDATE_ADD_INTERVALS] = 0; #endif // SQL_TIMEDATE_ADD_INTERVALS @@ -303,7 +305,7 @@ namespace ignite #endif // SQL_CONVERT_FUNCTIONS #ifdef SQL_OJ_CAPABILITIES - // Bitmask enumerating the types of outer joins supported by the + // Bitmask enumerating the types of outer joins supported by the // driver and data source. intParams[SQL_OJ_CAPABILITIES] = SQL_OJ_LEFT | SQL_OJ_NOT_ORDERED | SQL_OJ_ALL_COMPARISON_OPS; #endif // SQL_OJ_CAPABILITIES @@ -333,7 +335,7 @@ namespace ignite #ifdef SQL_SQL92_VALUE_EXPRESSIONS // Bitmask enumerating the value expressions supported, // as defined in SQL-92. - intParams[SQL_SQL92_VALUE_EXPRESSIONS] = SQL_SVE_CASE | + intParams[SQL_SQL92_VALUE_EXPRESSIONS] = SQL_SVE_CASE | SQL_SVE_CAST | SQL_SVE_COALESCE | SQL_SVE_NULLIF; #endif // SQL_SQL92_VALUE_EXPRESSIONS @@ -369,6 +371,40 @@ namespace ignite intParams[SQL_STATIC_CURSOR_ATTRIBUTES2] = 0; #endif //SQL_STATIC_CURSOR_ATTRIBUTES2 +#ifdef SQL_PARAM_ARRAY_ROW_COUNTS + // Enumerating the driver's properties regarding the availability of row counts in a parameterized + // execution. Has the following values: + // + // SQL_PARC_BATCH = Individual row counts are available for each set of parameters. This is conceptually + // equivalent to the driver generating a batch of SQL statements, one for each parameter set in the + // array. Extended error information can be retrieved by using the SQL_PARAM_STATUS_PTR descriptor + // field. + // + // SQL_PARC_NO_BATCH = There is only one row count available, which is the cumulative row count + // resulting from the execution of the statement for the entire array of parameters. This is + // conceptually equivalent to treating the statement together with the complete parameter array as + // one atomic unit. Errors are handled the same as if one statement were executed. + intParams[SQL_PARAM_ARRAY_ROW_COUNTS] = SQL_PARC_NO_BATCH; +#endif //SQL_PARAM_ARRAY_ROW_COUNTS + +#ifdef SQL_PARAM_ARRAY_SELECTS + // Enumerating the driver's properties regarding the availability of result sets in a parameterized + // execution. Has the following values: + // + // SQL_PAS_BATCH = There is one result set available per set of parameters. This is conceptually + // equivalent to the driver generating a batch of SQL statements, one for each parameter set in + // the array. + // + // SQL_PAS_NO_BATCH = There is only one result set available, which represents the cumulative result set + // resulting from the execution of the statement for the complete array of parameters. This is + // conceptually equivalent to treating the statement together with the complete parameter array as + // one atomic unit. + // + // SQL_PAS_NO_SELECT = A driver does not allow a result - set generating statement to be executed with + // an array of parameters. + intParams[SQL_PARAM_ARRAY_SELECTS] = SQL_PAS_NO_SELECT; +#endif //SQL_PARAM_ARRAY_SELECTS + //======================= Short Params ======================== #ifdef SQL_MAX_CONCURRENT_ACTIVITIES // The maximum number of active statements that the driver can @@ -412,10 +448,10 @@ namespace ignite StringInfoMap::const_iterator itStr = strParams.find(type); - if (itStr != strParams.cend()) + if (itStr != strParams.cend()) { unsigned short strlen = static_cast( - utility::CopyStringToBuffer(itStr->second, + utility::CopyStringToBuffer(itStr->second, reinterpret_cast(buf), buflen)); if (reslen) diff --git a/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp b/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp index 215d77f5c249f..6e873c296f1b8 100644 --- a/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp +++ b/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp @@ -75,6 +75,12 @@ namespace /** SQL state HYT01 constant. */ const std::string STATE_HYT01 = "HYT01"; + + /** SQL state HY003 constant. */ + const std::string STATE_HY003 = "HY003"; + + /** SQL state HY105 constant. */ + const std::string STATE_HY105 = "HY105"; } namespace ignite @@ -246,6 +252,12 @@ namespace ignite case SQL_STATE_HYT01_CONNECTIOIN_TIMEOUT: return STATE_HYT01; + case SQL_STATE_HY003_INVALID_APPLICATION_BUFFER_TYPE: + return STATE_HY003; + + case SQL_STATE_HY105_INVALID_PARAMETER_TYPE: + return STATE_HY105; + default: break; } diff --git a/modules/platforms/cpp/odbc/src/message.cpp b/modules/platforms/cpp/odbc/src/message.cpp new file mode 100644 index 0000000000000..741540526fb19 --- /dev/null +++ b/modules/platforms/cpp/odbc/src/message.cpp @@ -0,0 +1,366 @@ +/* + * 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. + */ + +#include "ignite/odbc/message.h" +#include "ignite/odbc/utility.h" + +namespace ignite +{ + namespace odbc + { + HandshakeRequest::HandshakeRequest(int64_t version, bool distributedJoins, bool enforceJoinOrder): + version(version), + distributedJoins(distributedJoins), + enforceJoinOrder(enforceJoinOrder) + { + // No-op. + } + + HandshakeRequest::~HandshakeRequest() + { + // No-op. + } + + void HandshakeRequest::Write(impl::binary::BinaryWriterImpl& writer) const + { + writer.WriteInt8(REQUEST_TYPE_HANDSHAKE); + + writer.WriteInt64(version); + + writer.WriteBool(distributedJoins); + writer.WriteBool(enforceJoinOrder); + } + + QueryExecuteRequest::QueryExecuteRequest(const std::string& schema, const std::string& sql, const app::ParameterSet& params): + cache(schema), + sql(sql), + params(params) + { + // No-op. + } + + QueryExecuteRequest::~QueryExecuteRequest() + { + // No-op. + } + + void QueryExecuteRequest::Write(impl::binary::BinaryWriterImpl& writer) const + { + writer.WriteInt8(REQUEST_TYPE_EXECUTE_SQL_QUERY); + + if (cache.empty()) + writer.WriteNull(); + else + writer.WriteObject(cache); + + writer.WriteObject(sql); + + params.Write(writer); + } + + QueryExecuteBatchtRequest::QueryExecuteBatchtRequest(const std::string& schema, const std::string& sql, + const app::ParameterSet& params, SqlUlen begin, SqlUlen end, bool last): + schema(schema), + sql(sql), + params(params), + begin(begin), + end(end), + last(last) + { + // No-op. + } + + QueryExecuteBatchtRequest::~QueryExecuteBatchtRequest() + { + // No-op. + } + + void QueryExecuteBatchtRequest::Write(impl::binary::BinaryWriterImpl& writer) const + { + writer.WriteInt8(REQUEST_TYPE_EXECUTE_SQL_QUERY_BATCH); + + if (schema.empty()) + writer.WriteNull(); + else + writer.WriteObject(schema); + + writer.WriteObject(sql); + + params.Write(writer, begin, end, last); + } + + QueryCloseRequest::QueryCloseRequest(int64_t queryId): queryId(queryId) + { + // No-op. + } + + QueryCloseRequest::~QueryCloseRequest() + { + // No-op. + } + + void QueryCloseRequest::Write(impl::binary::BinaryWriterImpl& writer) const + { + writer.WriteInt8(REQUEST_TYPE_CLOSE_SQL_QUERY); + writer.WriteInt64(queryId); + } + + QueryFetchRequest::QueryFetchRequest(int64_t queryId, int32_t pageSize): + queryId(queryId), + pageSize(pageSize) + { + // No-op. + } + + QueryFetchRequest::~QueryFetchRequest() + { + // No-op. + } + + void QueryFetchRequest::Write(impl::binary::BinaryWriterImpl& writer) const + { + writer.WriteInt8(REQUEST_TYPE_FETCH_SQL_QUERY); + writer.WriteInt64(queryId); + writer.WriteInt32(pageSize); + } + + QueryGetColumnsMetaRequest::QueryGetColumnsMetaRequest(const std::string& schema, const std::string& table, const std::string& column): + schema(schema), + table(table), + column(column) + { + // No-op. + } + + QueryGetColumnsMetaRequest::~QueryGetColumnsMetaRequest() + { + // No-op. + } + + void QueryGetColumnsMetaRequest::Write(impl::binary::BinaryWriterImpl& writer) const + { + writer.WriteInt8(REQUEST_TYPE_GET_COLUMNS_METADATA); + + utility::WriteString(writer, schema); + utility::WriteString(writer, table); + utility::WriteString(writer, column); + } + + QueryGetTablesMetaRequest::QueryGetTablesMetaRequest(const std::string& catalog, const std::string& schema, const std::string& table, const std::string& tableTypes): + catalog(catalog), + schema(schema), + table(table), + tableTypes(tableTypes) + { + // No-op. + } + + QueryGetTablesMetaRequest::~QueryGetTablesMetaRequest() + { + // No-op. + } + + void QueryGetTablesMetaRequest::Write(impl::binary::BinaryWriterImpl& writer) const + { + writer.WriteInt8(REQUEST_TYPE_GET_TABLES_METADATA); + + utility::WriteString(writer, catalog); + utility::WriteString(writer, schema); + utility::WriteString(writer, table); + utility::WriteString(writer, tableTypes); + } + + void QueryGetParamsMetaRequest::Write(impl::binary::BinaryWriterImpl& writer) const + { + writer.WriteInt8(REQUEST_TYPE_GET_PARAMS_METADATA); + + utility::WriteString(writer, cacheName); + utility::WriteString(writer, sqlQuery); + } + + Response::Response(): + status(RESPONSE_STATUS_FAILED), + error() + { + // No-op. + } + + Response::~Response() + { + // No-op. + } + + void Response::Read(impl::binary::BinaryReaderImpl& reader) + { + status = reader.ReadInt8(); + + if (status == RESPONSE_STATUS_SUCCESS) + ReadOnSuccess(reader); + else + utility::ReadString(reader, error); + } + + void Response::ReadOnSuccess(impl::binary::BinaryReaderImpl&) + { + // No-op. + } + + HandshakeResponse::HandshakeResponse(): + accepted(false), + protoVerSince(), + currentVer() + { + // No-op. + } + + HandshakeResponse::~HandshakeResponse() + { + // No-op. + } + + void HandshakeResponse::ReadOnSuccess(impl::binary::BinaryReaderImpl& reader) + { + accepted = reader.ReadBool(); + + if (!accepted) + { + utility::ReadString(reader, protoVerSince); + utility::ReadString(reader, currentVer); + } + } + + QueryCloseResponse::QueryCloseResponse(): queryId(0) + { + // No-op. + } + + QueryCloseResponse::~QueryCloseResponse() + { + // No-op. + } + + void QueryCloseResponse::ReadOnSuccess(impl::binary::BinaryReaderImpl& reader) + { + queryId = reader.ReadInt64(); + } + + QueryExecuteResponse::QueryExecuteResponse(): queryId(0), meta() + { + // No-op. + } + + QueryExecuteResponse::~QueryExecuteResponse() + { + // No-op. + } + + void QueryExecuteResponse::ReadOnSuccess(impl::binary::BinaryReaderImpl& reader) + { + queryId = reader.ReadInt64(); + + meta::ReadColumnMetaVector(reader, meta); + } + + QueryExecuteBatchResponse::QueryExecuteBatchResponse(): + affectedRows(0), + errorSetIdx(-1), + errorMessage() + { + // No-op. + } + + QueryExecuteBatchResponse::~QueryExecuteBatchResponse() + { + // No-op. + } + + void QueryExecuteBatchResponse::ReadOnSuccess(impl::binary::BinaryReaderImpl& reader) + { + bool success = reader.ReadBool(); + affectedRows = reader.ReadInt64(); + + if (!success) + { + errorSetIdx = reader.ReadInt64(); + errorMessage = reader.ReadObject(); + } + } + + QueryFetchResponse::QueryFetchResponse(ResultPage& resultPage): queryId(0), resultPage(resultPage) + { + // No-op. + } + + QueryFetchResponse::~QueryFetchResponse() + { + // No-op. + } + + void QueryFetchResponse::ReadOnSuccess(impl::binary::BinaryReaderImpl& reader) + { + queryId = reader.ReadInt64(); + + resultPage.Read(reader); + } + + QueryGetColumnsMetaResponse::QueryGetColumnsMetaResponse() + { + // No-op. + } + + QueryGetColumnsMetaResponse::~QueryGetColumnsMetaResponse() + { + // No-op. + } + + void QueryGetColumnsMetaResponse::ReadOnSuccess(impl::binary::BinaryReaderImpl& reader) + { + meta::ReadColumnMetaVector(reader, meta); + } + + QueryGetTablesMetaResponse::QueryGetTablesMetaResponse() + { + // No-op. + } + + QueryGetTablesMetaResponse::~QueryGetTablesMetaResponse() + { + // No-op. + } + + void QueryGetTablesMetaResponse::ReadOnSuccess(impl::binary::BinaryReaderImpl& reader) + { + meta::ReadTableMetaVector(reader, meta); + } + + QueryGetParamsMetaResponse::QueryGetParamsMetaResponse() + { + // No-op. + } + + QueryGetParamsMetaResponse::~QueryGetParamsMetaResponse() + { + // No-op. + } + + void QueryGetParamsMetaResponse::ReadOnSuccess(impl::binary::BinaryReaderImpl& reader) + { + utility::ReadByteArray(reader, typeIds); + } + } +} + diff --git a/modules/platforms/cpp/odbc/src/odbc.cpp b/modules/platforms/cpp/odbc/src/odbc.cpp index caf31d91860b1..5f4db6597bfea 100644 --- a/modules/platforms/cpp/odbc/src/odbc.cpp +++ b/modules/platforms/cpp/odbc/src/odbc.cpp @@ -630,27 +630,8 @@ namespace ignite if (!statement) return SQL_INVALID_HANDLE; - if (ioType != SQL_PARAM_INPUT) - return SQL_ERROR; - - if (!IsSqlTypeSupported(paramSqlType)) - return SQL_ERROR; - - IgniteSqlType driverType = ToDriverType(bufferType); - - if (driverType == IGNITE_ODBC_C_TYPE_UNSUPPORTED) - return SQL_ERROR; - - if (buffer) - { - ApplicationDataBuffer dataBuffer(driverType, buffer, bufferLen, resLen); - - Parameter param(dataBuffer, paramSqlType, columnSize, decDigits); - - statement->BindParameter(paramIdx, param); - } - else - statement->UnbindParameter(paramIdx); + statement->BindParameter(paramIdx, ioType, bufferType, paramSqlType, + columnSize, decDigits, buffer, bufferLen, resLen); return statement->GetDiagnosticRecords().GetReturnCode(); } diff --git a/modules/platforms/cpp/odbc/src/query/batch_query.cpp b/modules/platforms/cpp/odbc/src/query/batch_query.cpp new file mode 100644 index 0000000000000..11ddd93e19302 --- /dev/null +++ b/modules/platforms/cpp/odbc/src/query/batch_query.cpp @@ -0,0 +1,197 @@ +/* + * 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. + */ + +#include "ignite/odbc/connection.h" +#include "ignite/odbc/message.h" +#include "ignite/odbc/query/batch_query.h" + +namespace ignite +{ + namespace odbc + { + namespace query + { + BatchQuery::BatchQuery(diagnostic::Diagnosable& diag, Connection& connection, + const std::string& sql, const app::ParameterSet& params) : + Query(diag, BATCH), + connection(connection), + sql(sql), + params(params), + resultMeta(), + rowsAffected(0), + setsProcessed(0), + executed(false), + dataRetrieved(false) + { + // No-op. + } + + BatchQuery::~BatchQuery() + { + // No-op. + } + + SqlResult BatchQuery::Execute() + { + if (executed) + { + diag.AddStatusRecord(SQL_STATE_HY010_SEQUENCE_ERROR, "Query cursor is in open state already."); + + return SQL_RESULT_ERROR; + } + + int32_t maxPageSize = connection.GetConfiguration().GetPageSize(); + int32_t rowNum = params.GetParamSetSize(); + SqlResult res; + + int32_t processed = 0; + + do { + int32_t currentPageSize = std::min(maxPageSize, rowNum - processed); + bool lastPage = currentPageSize == rowNum - processed; + + res = MakeRequestExecuteBatch(processed, processed + currentPageSize, lastPage); + + processed += currentPageSize; + } while (res == SQL_RESULT_SUCCESS && processed < rowNum); + + params.SetParamsProcessed(static_cast(setsProcessed)); + + return res; + } + + const meta::ColumnMetaVector& BatchQuery::GetMeta() const + { + return resultMeta; + } + + SqlResult BatchQuery::FetchNextRow(app::ColumnBindingMap& columnBindings) + { + if (!executed) + { + diag.AddStatusRecord(SQL_STATE_HY010_SEQUENCE_ERROR, "Query was not executed."); + + return SQL_RESULT_ERROR; + } + + if (dataRetrieved) + return SQL_RESULT_NO_DATA; + + app::ColumnBindingMap::iterator it = columnBindings.find(1); + + if (it != columnBindings.end()) + it->second.PutInt64(rowsAffected); + + dataRetrieved = true; + + return SQL_RESULT_SUCCESS; + } + + SqlResult BatchQuery::GetColumn(uint16_t columnIdx, app::ApplicationDataBuffer& buffer) + { + if (!executed) + { + diag.AddStatusRecord(SQL_STATE_HY010_SEQUENCE_ERROR, "Query was not executed."); + + return SQL_RESULT_ERROR; + } + + if (dataRetrieved) + return SQL_RESULT_NO_DATA; + + if (columnIdx != 1) + { + std::stringstream builder; + builder << "Column with id " << columnIdx << " is not available in result set."; + + diag.AddStatusRecord(SQL_STATE_HY000_GENERAL_ERROR, builder.str()); + + return SQL_RESULT_ERROR; + } + + buffer.PutInt64(rowsAffected); + + return SQL_RESULT_SUCCESS; + } + + SqlResult BatchQuery::Close() + { + return SQL_RESULT_SUCCESS; + } + + bool BatchQuery::DataAvailable() const + { + return false; + } + + int64_t BatchQuery::AffectedRows() const + { + return rowsAffected; + } + + SqlResult BatchQuery::MakeRequestExecuteBatch(SqlUlen begin, SqlUlen end, bool last) + { + const std::string& schema = connection.GetCache(); + + QueryExecuteBatchtRequest req(schema, sql, params, begin, end, last); + QueryExecuteBatchResponse rsp; + + try + { + connection.SyncMessage(req, rsp); + } + catch (const IgniteError& err) + { + diag.AddStatusRecord(SQL_STATE_HYT01_CONNECTIOIN_TIMEOUT, err.GetText()); + + return SQL_RESULT_ERROR; + } + + if (rsp.GetStatus() != RESPONSE_STATUS_SUCCESS) + { + LOG_MSG("Error: " << rsp.GetError()); + + diag.AddStatusRecord(SQL_STATE_HY000_GENERAL_ERROR, rsp.GetError()); + + return SQL_RESULT_ERROR; + } + + rowsAffected += rsp.GetAffectedRows(); + LOG_MSG("rowsAffected: " << rowsAffected); + + if (!rsp.GetErrorMessage().empty()) + { + LOG_MSG("Error: " << rsp.GetErrorMessage()); + + setsProcessed += rsp.GetErrorSetIdx(); + LOG_MSG("setsProcessed: " << setsProcessed); + + diag.AddStatusRecord(SQL_STATE_HY000_GENERAL_ERROR, rsp.GetErrorMessage(), + static_cast(setsProcessed), 0); + + return SQL_RESULT_SUCCESS_WITH_INFO; + } + + setsProcessed += end - begin; + LOG_MSG("setsProcessed: " << setsProcessed); + + return SQL_RESULT_SUCCESS; + } + } + } +} + diff --git a/modules/platforms/cpp/odbc/src/query/data_query.cpp b/modules/platforms/cpp/odbc/src/query/data_query.cpp index 5b1b758bd4281..f4402d41f7895 100644 --- a/modules/platforms/cpp/odbc/src/query/data_query.cpp +++ b/modules/platforms/cpp/odbc/src/query/data_query.cpp @@ -18,6 +18,7 @@ #include "ignite/odbc/connection.h" #include "ignite/odbc/message.h" #include "ignite/odbc/query/data_query.h" +#include "ignite/odbc/query/batch_query.h" namespace ignite { @@ -25,9 +26,8 @@ namespace ignite { namespace query { - DataQuery::DataQuery(diagnostic::Diagnosable& diag, - Connection& connection, const std::string& sql, - const app::ParameterBindingMap& params) : + DataQuery::DataQuery(diagnostic::Diagnosable& diag, Connection& connection, + const std::string& sql, const app::ParameterSet& params) : Query(diag, DATA), connection(connection), sql(sql), @@ -38,9 +38,9 @@ namespace ignite DataQuery::~DataQuery() { - Close(); + InternalClose(); } - + SqlResult DataQuery::Execute() { if (cursor.get()) @@ -139,6 +139,11 @@ namespace ignite } SqlResult DataQuery::Close() + { + return InternalClose(); + } + + SqlResult DataQuery::InternalClose() { if (!cursor.get()) return SQL_RESULT_SUCCESS; diff --git a/modules/platforms/cpp/odbc/src/statement.cpp b/modules/platforms/cpp/odbc/src/statement.cpp index 9aca8c9164209..6154f9126568e 100644 --- a/modules/platforms/cpp/odbc/src/statement.cpp +++ b/modules/platforms/cpp/odbc/src/statement.cpp @@ -16,6 +16,7 @@ */ #include "ignite/odbc/system/odbc_constants.h" +#include "ignite/odbc/query/batch_query.h" #include "ignite/odbc/query/data_query.h" #include "ignite/odbc/query/column_metadata_query.h" #include "ignite/odbc/query/table_metadata_query.h" @@ -38,9 +39,8 @@ namespace ignite currentQuery(), rowsFetched(0), rowStatuses(0), - paramBindOffset(0), columnBindOffset(0), - currentParamIdx(0) + parameters() { // No-op. } @@ -55,8 +55,6 @@ namespace ignite IGNITE_ODBC_API_CALL_ALWAYS_SUCCESS; columnBindings[columnIdx] = buffer; - - columnBindings[columnIdx].SetPtrToOffsetPtr(&columnBindOffset); } void Statement::UnbindColumn(uint16_t columnIdx) @@ -108,25 +106,72 @@ namespace ignite return SQL_RESULT_SUCCESS; } - void Statement::BindParameter(uint16_t paramIdx, const app::Parameter& param) + void Statement::BindParameter(uint16_t paramIdx, int16_t ioType, int16_t bufferType, int16_t paramSqlType, + SqlUlen columnSize, int16_t decDigits, void* buffer, SqlLen bufferLen, SqlLen* resLen) { - IGNITE_ODBC_API_CALL(InternalBindParameter(paramIdx, param)); + IGNITE_ODBC_API_CALL(InternalBindParameter(paramIdx, ioType, bufferType, paramSqlType, + columnSize, decDigits, buffer, bufferLen, resLen)); } - - SqlResult Statement::InternalBindParameter(uint16_t paramIdx, const app::Parameter& param) + SqlResult Statement::InternalBindParameter(uint16_t paramIdx, int16_t ioType, int16_t bufferType, + int16_t paramSqlType, SqlUlen columnSize, int16_t decDigits, void* buffer, SqlLen bufferLen, SqlLen* resLen) { + using namespace type_traits; + using app::ApplicationDataBuffer; + using app::Parameter; + if (paramIdx == 0) { - AddStatusRecord(SQL_STATE_24000_INVALID_CURSOR_STATE, - "The value specified for the argument ParameterNumber was less than 1."); + std::stringstream builder; + builder << "The value specified for the argument ParameterNumber was less than 1. [ParameterNumber=" << paramIdx << ']'; + + AddStatusRecord(SQL_STATE_24000_INVALID_CURSOR_STATE, builder.str()); + + return SQL_RESULT_ERROR; + } + + if (ioType != SQL_PARAM_INPUT) + { + std::stringstream builder; + builder << "The value specified for the argument InputOutputType was not SQL_PARAM_INPUT. [ioType=" << ioType << ']'; + + AddStatusRecord(SQL_STATE_HY105_INVALID_PARAMETER_TYPE, builder.str()); + + return SQL_RESULT_ERROR; + } + + if (!IsSqlTypeSupported(paramSqlType)) + { + std::stringstream builder; + builder << "Data type is not supported. [typeId=" << paramSqlType << ']'; + + AddStatusRecord(SQL_STATE_HYC00_OPTIONAL_FEATURE_NOT_IMPLEMENTED, builder.str()); + + return SQL_RESULT_ERROR; + } + + IgniteSqlType driverType = ToDriverType(bufferType); + + if (driverType == IGNITE_ODBC_C_TYPE_UNSUPPORTED) + { + std::stringstream builder; + builder << "The argument TargetType was not a valid data type. [TargetType=" << bufferType << ']'; + + AddStatusRecord(SQL_STATE_HY003_INVALID_APPLICATION_BUFFER_TYPE, builder.str()); return SQL_RESULT_ERROR; } - paramBindings[paramIdx] = param; + if (buffer) + { + ApplicationDataBuffer dataBuffer(driverType, buffer, bufferLen, resLen); + + Parameter param(dataBuffer, paramSqlType, columnSize, decDigits); - paramBindings[paramIdx].GetBuffer().SetPtrToOffsetPtr(¶mBindOffset); + parameters.BindParameter(paramIdx, param); + } + else + parameters.UnbindParameter(paramIdx); return SQL_RESULT_SUCCESS; } @@ -135,14 +180,14 @@ namespace ignite { IGNITE_ODBC_API_CALL_ALWAYS_SUCCESS; - paramBindings.erase(paramIdx); + parameters.UnbindParameter(paramIdx); } void Statement::UnbindAllParameters() { IGNITE_ODBC_API_CALL_ALWAYS_SUCCESS; - paramBindings.clear(); + parameters.UnbindAll(); } void Statement::SetAttribute(int attr, void* value, SQLINTEGER valueLen) @@ -156,7 +201,7 @@ namespace ignite { case SQL_ATTR_ROW_ARRAY_SIZE: { - SQLULEN val = reinterpret_cast(value); + SqlUlen val = reinterpret_cast(value); LOG_MSG("SQL_ATTR_ROW_ARRAY_SIZE: %d\n", val); @@ -199,6 +244,20 @@ namespace ignite break; } + case SQL_ATTR_PARAMSET_SIZE: + { + parameters.SetParamSetSize(reinterpret_cast(value)); + + break; + } + + case SQL_ATTR_PARAMS_PROCESSED_PTR: + { + parameters.SetParamsProcessedPtr(reinterpret_cast(value)); + + break; + } + default: { AddStatusRecord(SQL_STATE_HYC00_OPTIONAL_FEATURE_NOT_IMPLEMENTED, @@ -236,6 +295,9 @@ namespace ignite *val = static_cast(this); + if (valueLen) + *valueLen = SQL_IS_POINTER; + break; } @@ -245,14 +307,20 @@ namespace ignite *val = static_cast(1); + if (valueLen) + *valueLen = SQL_IS_INTEGER; + break; } case SQL_ATTR_ROWS_FETCHED_PTR: { - SQLULEN** val = reinterpret_cast(buf); + SqlUlen** val = reinterpret_cast(buf); + + *val = reinterpret_cast(GetRowsFetchedPtr()); - *val = reinterpret_cast(GetRowsFetchedPtr()); + if (valueLen) + *valueLen = SQL_IS_POINTER; break; } @@ -263,6 +331,9 @@ namespace ignite *val = reinterpret_cast(GetRowStatusesPtr()); + if (valueLen) + *valueLen = SQL_IS_POINTER; + break; } @@ -270,16 +341,46 @@ namespace ignite { SQLULEN** val = reinterpret_cast(buf); - *val = reinterpret_cast(GetParamBindOffsetPtr()); + *val = reinterpret_cast(parameters.GetParamBindOffsetPtr()); + + if (valueLen) + *valueLen = SQL_IS_POINTER; break; } case SQL_ATTR_ROW_BIND_OFFSET_PTR: { - SQLULEN** val = reinterpret_cast(buf); + SqlUlen** val = reinterpret_cast(buf); + + *val = reinterpret_cast(GetColumnBindOffsetPtr()); + + if (valueLen) + *valueLen = SQL_IS_POINTER; + + break; + } + + case SQL_ATTR_PARAMSET_SIZE: + { + SqlUlen* val = reinterpret_cast(buf); + + *val = static_cast(parameters.GetParamSetSize()); + + if (valueLen) + *valueLen = SQL_IS_UINTEGER; + + break; + } + + case SQL_ATTR_PARAMS_PROCESSED_PTR: + { + SqlUlen** val = reinterpret_cast(buf); + + *val = parameters.GetParamsProcessedPtr(); - *val = reinterpret_cast(GetColumnBindOffsetPtr()); + if (valueLen) + *valueLen = SQL_IS_POINTER; break; } @@ -317,7 +418,7 @@ namespace ignite return SQL_RESULT_SUCCESS; } - if (paramTypes.empty()) + if (!parameters.IsMetadataSet()) { SqlResult res = UpdateParamsMeta(); @@ -325,7 +426,7 @@ namespace ignite return res; } - paramNum = static_cast(paramTypes.size()); + paramNum = parameters.GetExpectedParamNum(); return SQL_RESULT_SUCCESS; } @@ -334,12 +435,7 @@ namespace ignite { IGNITE_ODBC_API_CALL_ALWAYS_SUCCESS; - paramBindOffset = ptr; - } - - int* Statement::GetParamBindOffsetPtr() - { - return paramBindOffset; + parameters.SetParamBindOffsetPtr(ptr); } void Statement::GetColumnData(uint16_t columnIdx, app::ApplicationDataBuffer& buffer) @@ -373,10 +469,10 @@ namespace ignite if (currentQuery.get()) currentQuery->Close(); - currentQuery.reset(new query::DataQuery(*this, connection, query, paramBindings)); - // Resetting parameters types as we are changing the query. - paramTypes.clear(); + parameters.Prepare(); + + currentQuery.reset(new query::DataQuery(*this, connection, query, parameters)); return SQL_RESULT_SUCCESS; } @@ -410,20 +506,31 @@ namespace ignite return SQL_RESULT_ERROR; } - bool paramDataReady = true; - - app::ParameterBindingMap::iterator it; - for (it = paramBindings.begin(); it != paramBindings.end(); ++it) + if (parameters.GetParamSetSize() > 1 && currentQuery->GetType() == query::Query::DATA) { - app::Parameter& param = it->second; + query::DataQuery& qry = static_cast(*currentQuery); - param.ResetStoredData(); + currentQuery.reset(new query::BatchQuery(*this, connection, qry.GetSql(), parameters)); + } + else if (parameters.GetParamSetSize() == 1 && currentQuery->GetType() == query::Query::BATCH) + { + query::BatchQuery& qry = static_cast(*currentQuery); - paramDataReady &= param.IsDataReady(); + currentQuery.reset(new query::DataQuery(*this, connection, qry.GetSql(), parameters)); } - if (!paramDataReady) + if (parameters.IsDataAtExecNeeded()) + { + if (currentQuery->GetType() == query::Query::BATCH) + { + AddStatusRecord(SQL_STATE_HYC00_OPTIONAL_FEATURE_NOT_IMPLEMENTED, + "Data-at-execution is not supported together with batching."); + + return SQL_RESULT_ERROR; + } + return SQL_RESULT_NEED_DATA; + } return currentQuery->Execute(); } @@ -549,8 +656,10 @@ namespace ignite { if (!type_traits::IsSqlTypeSupported(sqlType)) { - AddStatusRecord(SQL_STATE_HYC00_OPTIONAL_FEATURE_NOT_IMPLEMENTED, - "Data type is not supported."); + std::stringstream builder; + builder << "Data type is not supported. [typeId=" << sqlType << ']'; + + AddStatusRecord(SQL_STATE_HYC00_OPTIONAL_FEATURE_NOT_IMPLEMENTED, builder.str()); return SQL_RESULT_ERROR; } @@ -572,7 +681,7 @@ namespace ignite { if (!currentQuery.get()) return SQL_RESULT_SUCCESS; - + SqlResult result = currentQuery->Close(); if (result == SQL_RESULT_SUCCESS) @@ -599,6 +708,12 @@ namespace ignite return SQL_RESULT_ERROR; } + if (columnBindOffset) + { + for (app::ColumnBindingMap::iterator it = columnBindings.begin(); it != columnBindings.end(); ++it) + it->second.SetByteOffset(*columnBindOffset); + } + SqlResult res = currentQuery->FetchNextRow(columnBindings); if (res == SQL_RESULT_SUCCESS) @@ -767,35 +882,24 @@ namespace ignite return SQL_RESULT_ERROR; } - app::ParameterBindingMap::iterator it; + app::Parameter *selected = parameters.GetSelectedParameter(); - if (currentParamIdx) + if (selected && !selected->IsDataReady()) { - it = paramBindings.find(currentParamIdx); + AddStatusRecord(SQL_STATE_22026_DATA_LENGTH_MISMATCH, + "Less data was sent for a parameter than was specified with " + "the StrLen_or_IndPtr argument in SQLBindParameter."); - if (it != paramBindings.end() && !it->second.IsDataReady()) - { - AddStatusRecord(SQL_STATE_22026_DATA_LENGTH_MISMATCH, - "Less data was sent for a parameter than was specified with " - "the StrLen_or_IndPtr argument in SQLBindParameter."); - - return SQL_RESULT_ERROR; - } + return SQL_RESULT_ERROR; } - for (it = paramBindings.begin(); it != paramBindings.end(); ++it) - { - uint16_t paramIdx = it->first; - app::Parameter& param = it->second; - - if (!param.IsDataReady()) - { - *paramPtr = param.GetBuffer().GetData(); + selected = parameters.SelectNextParameter(); - currentParamIdx = paramIdx; + if (selected) + { + *paramPtr = selected->GetBuffer().GetData(); - return SQL_RESULT_NEED_DATA; - } + return SQL_RESULT_NEED_DATA; } SqlResult res = currentQuery->Execute(); @@ -822,7 +926,7 @@ namespace ignite return SQL_RESULT_ERROR; } - if (currentParamIdx == 0) + if (!parameters.IsParameterSelected()) { AddStatusRecord(SQL_STATE_HY010_SEQUENCE_ERROR, "Parameter is not selected with the SQLParamData."); @@ -830,9 +934,9 @@ namespace ignite return SQL_RESULT_ERROR; } - app::ParameterBindingMap::iterator it = paramBindings.find(currentParamIdx); + app::Parameter* param = parameters.GetSelectedParameter(); - if (it == paramBindings.end()) + if (!param) { AddStatusRecord(SQL_STATE_HY000_GENERAL_ERROR, "Selected parameter has been unbound."); @@ -840,9 +944,7 @@ namespace ignite return SQL_RESULT_ERROR; } - app::Parameter& param = it->second; - - param.PutData(data, len); + param->PutData(data, len); return SQL_RESULT_SUCCESS; } @@ -872,10 +974,7 @@ namespace ignite return SQL_RESULT_ERROR; } - int8_t type = 0; - - if (paramNum > 0 && static_cast(paramNum) <= paramTypes.size()) - type = paramTypes[paramNum - 1]; + int8_t type = parameters.GetParamType(paramNum, 0); LOG_MSG("Type: %d\n", type); @@ -886,10 +985,7 @@ namespace ignite if (res != SQL_RESULT_SUCCESS) return res; - if (paramNum < 1 || static_cast(paramNum) > paramTypes.size()) - type = impl::binary::IGNITE_HDR_NULL; - else - type = paramTypes[paramNum - 1]; + type = parameters.GetParamType(paramNum, impl::binary::IGNITE_HDR_NULL); } if (dataType) @@ -942,10 +1038,12 @@ namespace ignite return SQL_RESULT_ERROR; } - paramTypes = rsp.GetTypeIds(); + parameters.UpdateParamsTypes(rsp.GetTypeIds()); - for (size_t i = 0; i < paramTypes.size(); ++i) - LOG_MSG("[%zu] Parameter type: %u\n", i, paramTypes[i]); + for (size_t i = 0; i < rsp.GetTypeIds().size(); ++i) + { + LOG_MSG("[" << i << "] Parameter type: " << rsp.GetTypeIds()[i]); + } return SQL_RESULT_SUCCESS; } From 7fbaecc67f1b204162bda4595d6c118ddd45f963 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Fri, 16 Jun 2017 20:01:49 +0300 Subject: [PATCH 125/357] IGNITE-5527: Prevent starvation in stripe pool on unstable topology. --- .../processors/cache/GridDeferredAckMessageSender.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridDeferredAckMessageSender.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridDeferredAckMessageSender.java index 7145dc2cd7fee..967404a1ccf3f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridDeferredAckMessageSender.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridDeferredAckMessageSender.java @@ -173,7 +173,8 @@ private DeferredAckMessageBuffer(UUID nodeId) { * @return {@code True} if request was handled, {@code false} if this buffer is filled and cannot be used. */ public boolean add(GridCacheVersion ver) { - readLock().lock(); + if(!readLock().tryLock()) + return false; // Here, writeLock is help by another thread and guard is already true. boolean snd = false; From f81964f59b0ea5b8dfdc8eb2acc34d2a5b8fee07 Mon Sep 17 00:00:00 2001 From: sboikov Date: Tue, 10 Jan 2017 16:59:17 +0300 Subject: [PATCH 126/357] 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 38f0ea80f3d95be16b38b621b3bcc2910c463997 Mon Sep 17 00:00:00 2001 From: sboikov Date: Tue, 10 Jan 2017 16:59:17 +0300 Subject: [PATCH 127/357] 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 d6d81de9d19a1..bf5652fd9ce60 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; @@ -4292,6 +4291,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)) { @@ -4350,6 +4353,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 5dd74ff635de50ff9561ccdb51bdeb620f60c3db Mon Sep 17 00:00:00 2001 From: sboikov Date: Tue, 10 Jan 2017 16:59:17 +0300 Subject: [PATCH 128/357] 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 c802b478ef47271f5b8864e0b0ae29925107e75f Mon Sep 17 00:00:00 2001 From: agura Date: Wed, 21 Jun 2017 18:52:17 +0300 Subject: [PATCH 129/357] Compilation is fixed --- .../cache/store/cassandra/datasource/DataSource.java | 3 --- .../service/GridServiceProcessorMultiNodeSelfTest.java | 7 +++++++ 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/DataSource.java b/modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/DataSource.java index d07a183ade4eb..abf22a6d45832 100644 --- a/modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/DataSource.java +++ b/modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/DataSource.java @@ -56,9 +56,6 @@ public class DataSource implements Externalizable { /** */ private static final long serialVersionUID = 0L; - /** Default expiration timeout for Cassandra driver session. */ - public static final long DFLT_SESSION_EXPIRATION_TIMEOUT = 300000; // 5 minutes. - /** * Null object, used as a replacement for those Cassandra connection options which * don't support serialization (RetryPolicy, LoadBalancingPolicy and etc). 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 5728bffa102ad..2958b7cb248c8 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 @@ -163,6 +163,8 @@ public void testDeployOnEachNodeButClientUpdateTopology() throws Exception { DummyService.exeLatch(name, latch); + int clients = 2; + startExtraNodes(servers, clients); try { @@ -170,6 +172,11 @@ public void testDeployOnEachNodeButClientUpdateTopology() throws Exception { waitForDeployment(name, 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, nodeCount() + servers, DummyService.started(name) - DummyService.cancelled(name)); + checkCount(name, g.services().serviceDescriptors(), nodeCount() + servers); } finally { From 5fb5c7e3b54ae4efb7a6a1832ba647677d93e0cd Mon Sep 17 00:00:00 2001 From: Evgenii Zhuravlev Date: Thu, 22 Jun 2017 09:43:03 +0300 Subject: [PATCH 130/357] IGNITE-5399 Manual cache rebalancing feature is broken --- .../java/org/apache/ignite/IgniteCache.java | 3 +- .../GridCachePartitionExchangeManager.java | 6 +- .../processors/cache/GridCachePreloader.java | 4 +- .../cache/GridCachePreloaderAdapter.java | 4 +- .../preloader/GridDhtPartitionDemander.java | 17 +- .../GridDhtPartitionsExchangeFuture.java | 7 +- .../dht/preloader/GridDhtPreloader.java | 2 +- .../CacheManualRebalancingTest.java | 178 ++++++++++++++++++ .../testsuites/IgniteCacheTestSuite5.java | 4 +- 9 files changed, 201 insertions(+), 24 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/CacheManualRebalancingTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java index d7bccf58941b1..ef2c4b6a00166 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java @@ -872,7 +872,8 @@ public Map> invokeAll(Set keys, * See {@link CacheConfiguration#getRebalanceDelay()} for more information on how to configure * rebalance re-partition delay. *

    - * @return Future that will be completed when rebalancing is finished. + * @return Future that will be completed when rebalancing is finished. Future.get() returns true + * when rebalance was successfully finished. */ public IgniteFuture rebalance(); 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 c175c43ae2f2a..07805f3f1b50b 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 @@ -80,6 +80,7 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.timeout.GridTimeoutObject; import org.apache.ignite.internal.util.GridListSet; +import org.apache.ignite.internal.util.future.GridCompoundFuture; import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.lang.IgnitePair; import org.apache.ignite.internal.util.tostring.GridToStringExclude; @@ -724,7 +725,7 @@ public void forceDummyExchange(boolean reassign, * @param exchFut Exchange future. */ public IgniteInternalFuture forceRebalance(GridDhtPartitionsExchangeFuture exchFut) { - GridFutureAdapter fut = new GridFutureAdapter<>(); + GridCompoundFuture fut = new GridCompoundFuture<>(CU.boolReducer()); exchWorker.addFuture( new GridDhtPartitionsExchangeFuture(cctx, exchFut.discoveryEvent(), exchFut.exchangeId(), fut)); @@ -1830,6 +1831,9 @@ void addFuture(GridDhtPartitionsExchangeFuture exchFut) { } } + if (exchFut.forcedRebalanceFuture() != null) + exchFut.forcedRebalanceFuture().markInitialized(); + if (assignsCancelled) { // Pending exchange. U.log(log, "Skipping rebalancing (obsolete exchange ID) " + "[top=" + exchFut.topologyVersion() + ", evt=" + exchFut.discoveryEvent().name() + diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java index 0c2869101aa79..110a53b3d2154 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java @@ -28,7 +28,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessageV2; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloaderAssignments; -import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.future.GridCompoundFuture; import org.apache.ignite.lang.IgnitePredicate; import org.jetbrains.annotations.Nullable; @@ -92,7 +92,7 @@ public Runnable addAssignments(GridDhtPreloaderAssignments assignments, boolean forcePreload, int cnt, Runnable next, - @Nullable GridFutureAdapter forcedRebFut); + @Nullable GridCompoundFuture forcedRebFut); /** * @param p Preload predicate. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java index 8ae67215dabc9..fe4859b2dc4b3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java @@ -30,8 +30,8 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessageV2; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloaderAssignments; +import org.apache.ignite.internal.util.future.GridCompoundFuture; import org.apache.ignite.internal.util.future.GridFinishedFuture; -import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.lang.IgnitePredicate; import org.jetbrains.annotations.Nullable; @@ -167,7 +167,7 @@ public GridCachePreloaderAdapter(GridCacheContext cctx) { /** {@inheritDoc} */ @Override public Runnable addAssignments(GridDhtPreloaderAssignments assignments, boolean forcePreload, - int cnt, Runnable next, @Nullable GridFutureAdapter forcedRebFut) { + int cnt, Runnable next, @Nullable GridCompoundFuture forcedRebFut) { return null; } 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 9ece00cd01049..5c6eb2b9a53d9 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 @@ -54,6 +54,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology; import org.apache.ignite.internal.processors.timeout.GridTimeoutObject; import org.apache.ignite.internal.processors.timeout.GridTimeoutObjectAdapter; +import org.apache.ignite.internal.util.future.GridCompoundFuture; import org.apache.ignite.internal.util.GridLeanSet; import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.future.GridFutureAdapter; @@ -301,7 +302,7 @@ Runnable addAssignments(final GridDhtPreloaderAssignments assigns, boolean force, int cnt, final Runnable next, - @Nullable final GridFutureAdapter forcedRebFut) { + @Nullable final GridCompoundFuture forcedRebFut) { if (log.isDebugEnabled()) log.debug("Adding partition assignments: " + assigns); @@ -324,18 +325,8 @@ Runnable addAssignments(final GridDhtPreloaderAssignments assigns, }); } - if (forcedRebFut != null) { - fut.listen(new CI1>() { - @Override public void apply(IgniteInternalFuture future) { - try { - forcedRebFut.onDone(future.get()); - } - catch (Exception e) { - forcedRebFut.onDone(e); - } - } - }); - } + if (forcedRebFut != null) + forcedRebFut.add(fut); rebalanceFut = fut; 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 2245d17e83523..d3e3701936786 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 @@ -62,6 +62,7 @@ import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.timeout.GridTimeoutObjectAdapter; +import org.apache.ignite.internal.util.future.GridCompoundFuture; import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.tostring.GridToStringInclude; @@ -198,7 +199,7 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter forcedRebFut; + private GridCompoundFuture forcedRebFut; /** * Dummy future created to trigger reassignments if partition @@ -236,7 +237,7 @@ public GridDhtPartitionsExchangeFuture( * @param forcedRebFut Forced Rebalance future. */ public GridDhtPartitionsExchangeFuture(GridCacheSharedContext cctx, DiscoveryEvent discoEvt, - GridDhtPartitionExchangeId exchId, GridFutureAdapter forcedRebFut) { + GridDhtPartitionExchangeId exchId, GridCompoundFuture forcedRebFut) { dummy = false; forcePreload = true; @@ -417,7 +418,7 @@ public GridDhtPartitionExchangeId exchangeId() { /** * @return Forced Rebalance future. */ - @Nullable public GridFutureAdapter forcedRebalanceFuture() { + @Nullable public GridCompoundFuture forcedRebalanceFuture() { return forcedRebFut; } 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 4aff4d5a71f42..2efaed859b714 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 @@ -413,7 +413,7 @@ public void handleDemandMessage(int idx, UUID id, GridDhtPartitionDemandMessage /** {@inheritDoc} */ @Override public Runnable addAssignments(GridDhtPreloaderAssignments assignments, - boolean forcePreload, int cnt, Runnable next, @Nullable GridFutureAdapter forcedRebFut) { + boolean forcePreload, int cnt, Runnable next, @Nullable GridCompoundFuture forcedRebFut) { return demander.addAssignments(assignments, forcePreload, cnt, next, forcedRebFut); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/CacheManualRebalancingTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/CacheManualRebalancingTest.java new file mode 100644 index 0000000000000..363e07b890711 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/CacheManualRebalancingTest.java @@ -0,0 +1,178 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.distributed.rebalancing; + +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteCompute; +import org.apache.ignite.IgniteDataStreamer; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.CachePeekMode; +import org.apache.ignite.compute.ComputeTaskFuture; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.util.lang.GridAbsPredicate; +import org.apache.ignite.lang.IgniteFuture; +import org.apache.ignite.lang.IgniteRunnable; +import org.apache.ignite.resources.IgniteInstanceResource; +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.testframework.junits.common.GridCommonAbstractTest; + +import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC; +import static org.apache.ignite.cache.CacheRebalanceMode.ASYNC; +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; + +/** */ +public class CacheManualRebalancingTest extends GridCommonAbstractTest { + /** */ + private static final String MYCACHE = "mycache"; + + /** */ + public static final TcpDiscoveryVmIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + + /** */ + public static final int NODES_CNT = 2; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(final String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER); + + cfg.setCacheConfiguration(cacheConfiguration()); + + return cfg; + } + + /** + * @return Cache configuration. + */ + private static CacheConfiguration cacheConfiguration() { + return new CacheConfiguration(MYCACHE) + .setAtomicityMode(ATOMIC) + .setCacheMode(CacheMode.PARTITIONED) + .setWriteSynchronizationMode(FULL_SYNC) + .setRebalanceMode(ASYNC) + .setRebalanceDelay(-1) + .setBackups(1) + .setCopyOnRead(true) + .setReadFromBackup(true); + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + startGrids(NODES_CNT); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + } + + /** {@inheritDoc} */ + @Override protected long getTestTimeout() { + return 400_000; + } + + /** + * @throws Exception If failed. + */ + public void testRebalance() throws Exception { + // Fill cache with large dataset to make rebalancing slow. + try (IgniteDataStreamer streamer = grid(0).dataStreamer(MYCACHE)) { + for (int i = 0; i < 100_000; i++) + streamer.addData(i, i); + } + + // Start new node. + final IgniteEx newNode = startGrid(NODES_CNT); + + int newNodeCacheSize; + + // Start manual rebalancing. + IgniteCompute compute = newNode.compute().withAsync(); + + compute.broadcast(new MyCallable()); + + final ComputeTaskFuture rebalanceTaskFuture = compute.future(); + + boolean rebalanceFinished = GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + return rebalanceTaskFuture.isDone(); + } + }, 10_000); + + assertTrue(rebalanceFinished); + + assertTrue(newNode.context().cache().cache(MYCACHE).context().preloader().rebalanceFuture().isDone()); + + newNodeCacheSize = newNode.cache(MYCACHE).localSize(CachePeekMode.ALL); + + System.out.println("New node cache local size: " + newNodeCacheSize); + + assertTrue(newNodeCacheSize > 0); + + } + + /** */ + public static class MyCallable implements IgniteRunnable { + /** */ + @IgniteInstanceResource + Ignite localNode; + + /** {@inheritDoc} */ + @Override public void run() { + IgniteLogger log = localNode.log(); + + log.info("Start local rebalancing caches"); + + for (String cacheName : localNode.cacheNames()) { + IgniteCache cache = localNode.cache(cacheName); + + assertNotNull(cache); + + boolean finished; + + log.info("Start rebalancing cache: " + cacheName + ", size: " + cache.localSize()); + + do { + IgniteFuture rebalance = cache.rebalance(); + + log.info("Wait rebalancing cache: " + cacheName + " - " + rebalance); + + finished = (Boolean)rebalance.get(); + + log.info("Rebalancing cache: " + cacheName + " - " + rebalance); + + if (finished) { + log.info("Finished rebalancing cache: " + cacheName + ", size: " + + cache.localSize(CachePeekMode.PRIMARY) + cache.localSize(CachePeekMode.BACKUP)); + } else + log.info("Rescheduled rebalancing cache: " + cacheName + ", size: " + cache.localSize()); + } + while (!finished); + } + + log.info("Finished local rebalancing caches"); + } + } +} 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 58a26e76ddd2f..464ded85c8587 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 @@ -31,8 +31,9 @@ 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.replicated.IgniteCacheSyncRebalanceModeSelfTest; import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheTxIteratorSelfTest; +import org.apache.ignite.internal.processors.cache.distributed.rebalancing.CacheManualRebalancingTest; +import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheSyncRebalanceModeSelfTest; import org.apache.ignite.internal.processors.cache.store.IgniteCacheWriteBehindNoUpdateSelfTest; /** @@ -66,6 +67,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(GridCacheOffHeapCleanupTest.class); suite.addTestSuite(CacheRebalancingSelfTest.class); + suite.addTestSuite(CacheManualRebalancingTest.class); return suite; } From 01d41b72ecc3e81dfc8966cc0e395c247037241c Mon Sep 17 00:00:00 2001 From: Evgenii Zhuravlev Date: Wed, 21 Jun 2017 13:48:15 +0300 Subject: [PATCH 131/357] GG-12256 H2Indexes are not deleted if key class implements Externalizable --- .../processors/query/GridQueryProcessor.java | 26 ++- .../cache/SqlQueryAfterCacheClearedTest.java | 163 ++++++++++++++++++ 2 files changed, 187 insertions(+), 2 deletions(-) create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/SqlQueryAfterCacheClearedTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java index 0bde688ded248..c79a65a6f1347 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java @@ -268,6 +268,17 @@ private void initializeCache(GridCacheContext cctx) throws IgniteCheckedEx else desc.keyClass(Object.class); } + else if (binaryEnabled) { + if (!valMustDeserialize && !SQL_TYPES.contains(valCls)) + desc.valueClass(Object.class); + else + desc.valueClass(valCls); + + if (!keyMustDeserialize && !SQL_TYPES.contains(keyCls)) + desc.keyClass(Object.class); + else + desc.keyClass(keyCls); + } else { if (keyCls == null) throw new IgniteCheckedException("Failed to find key class in the node classpath " + @@ -298,7 +309,7 @@ private void initializeCache(GridCacheContext cctx) throws IgniteCheckedEx TypeId typeId; TypeId altTypeId = null; - if (valCls == null || (binaryEnabled && !keyOrValMustDeserialize)) { + if (valCls == null || (binaryEnabled && (!valMustDeserialize))) { processBinaryMeta(qryEntity, desc); typeId = new TypeId(ccfg.getName(), ctx.cacheObjects().typeId(qryEntity.getValueType())); @@ -385,6 +396,17 @@ private void initializeCache(GridCacheContext cctx) throws IgniteCheckedEx else desc.keyClass(Object.class); } + else if (binaryEnabled) { + if (!valMustDeserialize && !SQL_TYPES.contains(valCls)) + desc.valueClass(Object.class); + else + desc.valueClass(valCls); + + if (!keyMustDeserialize && !SQL_TYPES.contains(keyCls)) + desc.keyClass(Object.class); + else + desc.keyClass(keyCls); + } else { desc.valueClass(valCls); desc.keyClass(keyCls); @@ -407,7 +429,7 @@ private void initializeCache(GridCacheContext cctx) throws IgniteCheckedEx TypeId typeId; TypeId altTypeId = null; - if (valCls == null || (binaryEnabled && !keyOrValMustDeserialize)) { + if (valCls == null || (binaryEnabled && !valMustDeserialize)) { processBinaryMeta(meta, desc); typeId = new TypeId(ccfg.getName(), ctx.cacheObjects().typeId(meta.getValueType())); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/SqlQueryAfterCacheClearedTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/SqlQueryAfterCacheClearedTest.java new file mode 100644 index 0000000000000..fa2d9ca341030 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/SqlQueryAfterCacheClearedTest.java @@ -0,0 +1,163 @@ +/* + * 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 org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.QueryEntity; +import org.apache.ignite.cache.affinity.AffinityKey; +import org.apache.ignite.cache.query.SqlQuery; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.jetbrains.annotations.NotNull; +import org.junit.Test; + +import java.util.LinkedHashMap; + +import static java.util.Collections.singletonList; +import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_TIERED; +import static org.apache.ignite.cache.CacheMode.PARTITIONED; + +public class SqlQueryAfterCacheClearedTest extends GridCommonAbstractTest { + + + /** */ + public static final TcpDiscoveryVmIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + + private final static String CACHE_NAME = "propertyCache"; + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + startGrids(1); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(final String gridName) throws Exception { + return new IgniteConfiguration() + .setGridName(gridName) + .setPeerClassLoadingEnabled(false) + .setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(IP_FINDER)) + .setCacheConfiguration(cacheCfg()); + } + + /** */ + private static CacheConfiguration cacheCfg() { + final CacheConfiguration ccfg = new CacheConfiguration(CACHE_NAME) + .setCacheMode(PARTITIONED) + .setMemoryMode(OFFHEAP_TIERED) + .setQueryEntities(singletonList(createQueryEntityConfig())); + return ccfg; + } + + @Test + public void testQueryCacheWasCleared() throws InterruptedException { + IgniteCache cache = grid(0).cache(CACHE_NAME); + + Property property1 = new Property(1, 2); + Property property2 = new Property(2, 2); + + cache.put(property1.getKey(), property1); + cache.put(property2.getKey(), property2); + + assertEquals(cache.size(),2); + assertEquals(cache.query(selectAllQuery()).getAll().size(), 2); + + cache.clear(); + + assertEquals(0, cache.size()); + assertEquals(0, cache.query(selectAllQuery()).getAll().size()); + } + + @Test + public void testQueryEntriesWereRemoved() { + IgniteCache cache = grid(0).cache(CACHE_NAME); + + Property property1 = new Property(1, 2); + Property property2 = new Property(2, 2); + + cache.put(property1.getKey(), property1); + cache.put(property2.getKey(), property2); + + assertEquals(cache.size(),2); + assertEquals(cache.query(selectAllQuery()).getAll().size(), 2); + + cache.remove(new PropertyAffinityKey(1, 2)); + cache.remove(new PropertyAffinityKey(2, 2)); + + assertEquals(0, cache.size()); + assertEquals(0, cache.query(selectAllQuery()).getAll().size()); + } + + @NotNull + private SqlQuery selectAllQuery() { + return new SqlQuery<>(Property.class, "from Property"); + } + + private static QueryEntity createQueryEntityConfig() { + QueryEntity queryEntity = new QueryEntity(); + queryEntity.setKeyType(PropertyAffinityKey.class.getName()); + queryEntity.setValueType(Property.class.getName()); + queryEntity.setFields(getMapOfFields()); + return queryEntity; + } + + @NotNull + private static LinkedHashMap getMapOfFields() { + LinkedHashMap mapOfFields = new LinkedHashMap<>(); + mapOfFields.put("id", Integer.class.getName()); + mapOfFields.put("region", Integer.class.getName()); + mapOfFields.put("key", PropertyAffinityKey.class.getName()); + return mapOfFields; + } + + static class Property { + private final int id; + private final int region; + private final String someData = "Some attributes"; + + Property(int id, int region) { + this.id = id; + this.region = region; + } + + public PropertyAffinityKey getKey() { + return new PropertyAffinityKey(id, region); + } + + public int getId() { + return id; + } + } + + static class PropertyAffinityKey extends AffinityKey { + public PropertyAffinityKey(final int thirdPartyPropertyId, final int region) { + super(thirdPartyPropertyId, region); + } + + public PropertyAffinityKey() { + // Required by Ignite + } + } +} From 5ac9afc719138e37a7d97d9d9db05243eee9a942 Mon Sep 17 00:00:00 2001 From: Evgenii Zhuravlev Date: Thu, 22 Jun 2017 12:36:14 +0300 Subject: [PATCH 132/357] IGNITE-5399 add test to testsuite --- .../ignite/testsuites/IgniteCacheQuerySelfTestSuite4.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite4.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite4.java index c4fcdacb603c8..2bcfd1dbec332 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite4.java +++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite4.java @@ -18,6 +18,7 @@ package org.apache.ignite.testsuites; import junit.framework.TestSuite; +import org.apache.ignite.internal.processors.cache.SqlQueryAfterCacheClearedTest; import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryAsyncFailoverAtomicPrimaryWriteOrderSelfTest; import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryAsyncFailoverTxReplicatedSelfTest; import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryAsyncFailoverTxSelfTest; @@ -50,6 +51,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(CacheContinuousQueryAsyncFailoverAtomicPrimaryWriteOrderSelfTest.class); suite.addTestSuite(CacheContinuousQueryAsyncFailoverTxReplicatedSelfTest.class); suite.addTestSuite(CacheContinuousQueryAsyncFailoverTxSelfTest.class); + suite.addTestSuite(SqlQueryAfterCacheClearedTest.class); return suite; } From a935d40a80e2f928a84a145aba540a45b156687f Mon Sep 17 00:00:00 2001 From: Evgenii Zhuravlev Date: Thu, 22 Jun 2017 15:10:32 +0300 Subject: [PATCH 133/357] GG-12256 Minor fixes --- .../processors/query/GridQueryProcessor.java | 28 +------ .../cache/SqlQueryAfterCacheClearedTest.java | 75 +++++++++++-------- 2 files changed, 46 insertions(+), 57 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java index c79a65a6f1347..7511a991472c1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java @@ -256,19 +256,7 @@ private void initializeCache(GridCacheContext cctx) throws IgniteCheckedEx desc.tableName(qryEntity.getTableName()); - if (binaryEnabled && !keyOrValMustDeserialize) { - // Safe to check null. - if (SQL_TYPES.contains(valCls)) - desc.valueClass(valCls); - else - desc.valueClass(Object.class); - - if (SQL_TYPES.contains(keyCls)) - desc.keyClass(keyCls); - else - desc.keyClass(Object.class); - } - else if (binaryEnabled) { + if (binaryEnabled) { if (!valMustDeserialize && !SQL_TYPES.contains(valCls)) desc.valueClass(Object.class); else @@ -384,19 +372,7 @@ else if (binaryEnabled) { desc.name(simpleValType); - if (binaryEnabled && !keyOrValMustDeserialize) { - // Safe to check null. - if (SQL_TYPES.contains(valCls)) - desc.valueClass(valCls); - else - desc.valueClass(Object.class); - - if (SQL_TYPES.contains(keyCls)) - desc.keyClass(keyCls); - else - desc.keyClass(Object.class); - } - else if (binaryEnabled) { + if (binaryEnabled) { if (!valMustDeserialize && !SQL_TYPES.contains(valCls)) desc.valueClass(Object.class); else diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/SqlQueryAfterCacheClearedTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/SqlQueryAfterCacheClearedTest.java index fa2d9ca341030..669161fb98ac5 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/SqlQueryAfterCacheClearedTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/SqlQueryAfterCacheClearedTest.java @@ -27,7 +27,6 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.jetbrains.annotations.NotNull; -import org.junit.Test; import java.util.LinkedHashMap; @@ -35,13 +34,15 @@ import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_TIERED; import static org.apache.ignite.cache.CacheMode.PARTITIONED; +/** + * + */ public class SqlQueryAfterCacheClearedTest extends GridCommonAbstractTest { - - /** */ public static final TcpDiscoveryVmIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); - private final static String CACHE_NAME = "propertyCache"; + /** */ + private static final String CACHE_NAME = "propertyCache"; /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { @@ -64,22 +65,21 @@ public class SqlQueryAfterCacheClearedTest extends GridCommonAbstractTest { /** */ private static CacheConfiguration cacheCfg() { - final CacheConfiguration ccfg = new CacheConfiguration(CACHE_NAME) + return new CacheConfiguration(CACHE_NAME) .setCacheMode(PARTITIONED) .setMemoryMode(OFFHEAP_TIERED) .setQueryEntities(singletonList(createQueryEntityConfig())); - return ccfg; } - @Test + /** */ public void testQueryCacheWasCleared() throws InterruptedException { IgniteCache cache = grid(0).cache(CACHE_NAME); - Property property1 = new Property(1, 2); - Property property2 = new Property(2, 2); + Property prop1 = new Property(1, 2); + Property prop2 = new Property(2, 2); - cache.put(property1.getKey(), property1); - cache.put(property2.getKey(), property2); + cache.put(prop1.getKey(), prop1); + cache.put(prop2.getKey(), prop2); assertEquals(cache.size(),2); assertEquals(cache.query(selectAllQuery()).getAll().size(), 2); @@ -90,15 +90,15 @@ public void testQueryCacheWasCleared() throws InterruptedException { assertEquals(0, cache.query(selectAllQuery()).getAll().size()); } - @Test + /** */ public void testQueryEntriesWereRemoved() { IgniteCache cache = grid(0).cache(CACHE_NAME); - Property property1 = new Property(1, 2); - Property property2 = new Property(2, 2); + Property prop1 = new Property(1, 2); + Property prop2 = new Property(2, 2); - cache.put(property1.getKey(), property1); - cache.put(property2.getKey(), property2); + cache.put(prop1.getKey(), prop1); + cache.put(prop2.getKey(), prop2); assertEquals(cache.size(),2); assertEquals(cache.query(selectAllQuery()).getAll().size(), 2); @@ -110,21 +110,22 @@ public void testQueryEntriesWereRemoved() { assertEquals(0, cache.query(selectAllQuery()).getAll().size()); } - @NotNull - private SqlQuery selectAllQuery() { + /** */ + @NotNull private SqlQuery selectAllQuery() { return new SqlQuery<>(Property.class, "from Property"); } + /** */ private static QueryEntity createQueryEntityConfig() { - QueryEntity queryEntity = new QueryEntity(); - queryEntity.setKeyType(PropertyAffinityKey.class.getName()); - queryEntity.setValueType(Property.class.getName()); - queryEntity.setFields(getMapOfFields()); - return queryEntity; + QueryEntity qryEntity = new QueryEntity(); + qryEntity.setKeyType(PropertyAffinityKey.class.getName()); + qryEntity.setValueType(Property.class.getName()); + qryEntity.setFields(getMapOfFields()); + return qryEntity; } - @NotNull - private static LinkedHashMap getMapOfFields() { + /** */ + @NotNull private static LinkedHashMap getMapOfFields() { LinkedHashMap mapOfFields = new LinkedHashMap<>(); mapOfFields.put("id", Integer.class.getName()); mapOfFields.put("region", Integer.class.getName()); @@ -132,32 +133,44 @@ private static LinkedHashMap getMapOfFields() { return mapOfFields; } - static class Property { + /** + * + */ + private static class Property { + /** Id. */ private final int id; + + /** Region. */ private final int region; - private final String someData = "Some attributes"; + /** */ Property(int id, int region) { this.id = id; this.region = region; } + /** */ public PropertyAffinityKey getKey() { return new PropertyAffinityKey(id, region); } + /** */ public int getId() { return id; } } - static class PropertyAffinityKey extends AffinityKey { - public PropertyAffinityKey(final int thirdPartyPropertyId, final int region) { - super(thirdPartyPropertyId, region); + /** + * + */ + private static class PropertyAffinityKey extends AffinityKey { + /** */ + PropertyAffinityKey(final int thirdPartyPropId, final int region) { + super(thirdPartyPropId, region); } + /** */ public PropertyAffinityKey() { - // Required by Ignite } } } From 7e2468770a4eb47a4f61204d8c2000b6ab67c967 Mon Sep 17 00:00:00 2001 From: nikolay_tikhonov Date: Thu, 22 Jun 2017 16:13:01 +0300 Subject: [PATCH 134/357] IGNITE-GG-12197 Fixed "Ignore events for discarded update in CLOCK mode". Signed-off-by: nikolay_tikhonov --- .../processors/cache/GridCacheMapEntry.java | 10 +- .../CacheContinuousQueryHandler.java | 128 +++++++----- .../CacheContinuousQueryManager.java | 5 +- .../cache/version/GridCacheVersion.java | 9 + ...tinuousQueryClockModeConflictSelfTest.java | 196 ++++++++++++++++++ ...ntinuousQueryFailoverAbstractSelfTest.java | 14 +- ...CacheContinuousQueryOrderingEventTest.java | 131 ++++++++---- .../IgniteCacheQuerySelfTestSuite3.java | 2 + 8 files changed, 392 insertions(+), 103 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryClockModeConflictSelfTest.java 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..e8c449994a253 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 @@ -2280,16 +2280,12 @@ else if (ttl != CU.TTL_ZERO) if (updateCntr != null) updateCntr0 = updateCntr; - cctx.continuousQueries().onEntryUpdated( + cctx.continuousQueries().skipUpdateEvent( + lsnrs, key, - evtVal, - prevVal, - isInternal() || !context().userCache(), partition(), - primary, - false, updateCntr0, - null, + primary, topVer); } 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..db8575c5ff89e 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 @@ -137,6 +137,9 @@ public class CacheContinuousQueryHandler implements GridContinuousHandler /** */ private boolean locCache; + /** Local query. */ + private transient boolean localQuery; + /** */ private transient boolean keepBinary; @@ -242,6 +245,13 @@ public void localCache(boolean locCache) { this.locCache = locCache; } + /** + * @param loc Local query. + */ + public void localQuery(boolean loc) { + this.localQuery = loc; + } + /** * @param taskHash Task hash. */ @@ -334,13 +344,13 @@ public void keepBinary(boolean keepBinary) { } } - entryBufs = new ConcurrentHashMap<>(); - - backupQueue = new ConcurrentLinkedDeque8<>(); - - ackBuf = new AcknowledgeBuffer(); - - rcvs = new ConcurrentHashMap<>(); + // Not need to support Fault Tolerance for local queries or local cache. + if (!isQueryOnlyLocal()) { + entryBufs = new ConcurrentHashMap<>(); + backupQueue = new ConcurrentLinkedDeque8<>(); + ackBuf = new AcknowledgeBuffer(); + rcvs = new ConcurrentHashMap<>(); + } this.nodeId = nodeId; @@ -403,7 +413,7 @@ public void keepBinary(boolean keepBinary) { primary, evt, recordIgniteEvt, - fut); + isQueryOnlyLocal() ? null : fut); ctx.asyncCallbackPool().execute(clsr, evt.partitionId()); } @@ -415,7 +425,7 @@ public void keepBinary(boolean keepBinary) { + ", notify=" + notify + ']'); if (primary || skipPrimaryCheck) { - if (fut == null) + if (fut == null || isQueryOnlyLocal()) onEntryUpdate(evt, notify, loc, recordIgniteEvt); else { fut.addContinuousQueryClosure(new CI1() { @@ -437,57 +447,66 @@ public void keepBinary(boolean keepBinary) { } @Override public void cleanupBackupQueue(Map updateCntrs) { - Collection backupQueue0 = backupQueue; + if (!isQueryOnlyLocal()) { + Collection backupQueue0 = backupQueue; - if (backupQueue0 != null) { - Iterator it = backupQueue0.iterator(); + if (backupQueue0 != null) { + Iterator it = backupQueue0.iterator(); - while (it.hasNext()) { - CacheContinuousQueryEntry backupEntry = it.next(); + while (it.hasNext()) { + CacheContinuousQueryEntry backupEntry = it.next(); - Long updateCntr = updateCntrs.get(backupEntry.partition()); + Long updateCntr = updateCntrs.get(backupEntry.partition()); - if (updateCntr != null && backupEntry.updateCounter() <= updateCntr) - it.remove(); + if (updateCntr != null && backupEntry.updateCounter() <= updateCntr) + it.remove(); + } } } } @Override public void flushBackupQueue(GridKernalContext ctx, AffinityTopologyVersion topVer) { - Collection backupQueue0 = backupQueue; + if (!isQueryOnlyLocal()) { + Collection backupQueue0 = backupQueue; - if (backupQueue0 == null) - return; + if (backupQueue0 == null) + return; - try { - ClusterNode nodeId0 = ctx.discovery().node(nodeId); + try { + ClusterNode nodeId0 = ctx.discovery().node(nodeId); - if (nodeId0 != null) { - GridCacheContext cctx = cacheContext(ctx); + if (nodeId0 != null) { + GridCacheContext cctx = cacheContext(ctx); - for (CacheContinuousQueryEntry e : backupQueue0) { - if (!e.isFiltered()) - prepareEntry(cctx, nodeId, e); + for (CacheContinuousQueryEntry e : backupQueue0) { + if (!e.isFiltered()) + prepareEntry(cctx, nodeId, e); + + e.topologyVersion(topVer); + } - e.topologyVersion(topVer); + ctx.continuous().addBackupNotification(nodeId, routineId, backupQueue0, topic); } + else + // Node which start CQ leave topology. Not needed to put data to backup queue. + backupQueue = null; - ctx.continuous().addBackupNotification(nodeId, routineId, backupQueue0, topic); + backupQueue0.clear(); + } + catch (IgniteCheckedException e) { + U.error(ctx.log(CU.CONTINUOUS_QRY_LOG_CATEGORY), + "Failed to send backup event notification to node: " + nodeId, e); } - else - // Node which start CQ leave topology. Not needed to put data to backup queue. - backupQueue = null; - - backupQueue0.clear(); - } - catch (IgniteCheckedException e) { - U.error(ctx.log(CU.CONTINUOUS_QRY_LOG_CATEGORY), - "Failed to send backup event notification to node: " + nodeId, e); } + else + assert backupQueue == null; // For local CQ backup queue should be null. } @Override public void acknowledgeBackupOnTimeout(GridKernalContext ctx) { - sendBackupAcknowledge(ackBuf.acknowledgeOnTimeout(), routineId, ctx); + if (!localQuery) + sendBackupAcknowledge(ackBuf.acknowledgeOnTimeout(), routineId, ctx); + else + assert ackBuf == null; // For local CQ ack buffer should be null. } @Override public void skipUpdateEvent(CacheContinuousQueryEvent evt, @@ -502,14 +521,18 @@ public void keepBinary(boolean keepBinary) { } @Override public void onPartitionEvicted(int part) { - Collection backupQueue0 = backupQueue; + if (!isQueryOnlyLocal()) { + Collection backupQueue0 = backupQueue; - if (backupQueue0 != null) { - for (Iterator it = backupQueue0.iterator(); it.hasNext(); ) { - if (it.next().partition() == part) - it.remove(); + if (backupQueue0 != null) { + for (Iterator it = backupQueue0.iterator(); it.hasNext(); ) { + if (it.next().partition() == part) + it.remove(); + } } } + else + assert backupQueue == null; // For local CQ backup queue should be null. } @Override public boolean oldValueRequired() { @@ -788,7 +811,11 @@ private void onEntryUpdate(CacheContinuousQueryEvent evt, boolean notify, boolea final CacheContinuousQueryEntry entry = evt.entry(); if (loc) { - if (!locCache) { + if (isQueryOnlyLocal()) { + if (!entry.isFiltered()) + locLsnr.onUpdated(F.>asList(evt)); + } + else { Collection> evts = handleEvent(ctx, entry); if (!evts.isEmpty()) @@ -797,10 +824,6 @@ private void onEntryUpdate(CacheContinuousQueryEvent evt, boolean notify, boolea if (!internal && !skipPrimaryCheck) sendBackupAcknowledge(ackBuf.onAcknowledged(entry), routineId, ctx); } - else { - if (!entry.isFiltered()) - locLsnr.onUpdated(F.>asList(evt)); - } } else { if (!entry.isFiltered()) @@ -848,6 +871,13 @@ private void onEntryUpdate(CacheContinuousQueryEvent evt, boolean notify, boolea } } + /** + * @return {@code True} if query deployed only localy (local query or local cache), otherwise {@code false}. + */ + private boolean isQueryOnlyLocal() { + return locCache || localQuery; + } + /** * @return Task name. */ 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..9a4ca90269727 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 @@ -641,6 +641,7 @@ private UUID executeQuery0(CacheEntryUpdatedListener locLsnr, hnd.internal(internal); hnd.keepBinary(keepBinary); hnd.localCache(cctx.isLocal()); + hnd.localQuery(loc); IgnitePredicate pred = (loc || cctx.config().getCacheMode() == CacheMode.LOCAL) ? F.nodeForNodeId(cctx.localNodeId()) : cctx.config().getNodeFilter(); @@ -649,14 +650,14 @@ private UUID executeQuery0(CacheEntryUpdatedListener locLsnr, UUID id = cctx.kernalContext().continuous().startRoutine( hnd, - internal && loc, + loc, bufSize, timeInterval, autoUnsubscribe, pred).get(); try { - if (hnd.isQuery() && cctx.userCache() && !onStart) + if (hnd.isQuery() && cctx.userCache() && !onStart && !loc) hnd.waitTopologyFuture(cctx.kernalContext()); } catch (IgniteCheckedException e) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersion.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersion.java index 95aab7407116c..7796dd5f9cdbc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersion.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersion.java @@ -190,6 +190,15 @@ public IgniteUuid asGridUuid() { return new IgniteUuid(new UUID(((long)topVer << 32) | nodeOrderDrId, globalTime), order); } + /** + * For testing purpose!!! + * + * @param globalTime Global time. + */ + public void globalTime(long globalTime) { + this.globalTime = globalTime; + } + /** {@inheritDoc} */ @Override public void onAckReceived() { // No-op. diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryClockModeConflictSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryClockModeConflictSelfTest.java new file mode 100644 index 0000000000000..9de738346090c --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryClockModeConflictSelfTest.java @@ -0,0 +1,196 @@ +/* + * 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.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import javax.cache.event.CacheEntryEvent; +import javax.cache.event.CacheEntryListenerException; +import javax.cache.event.CacheEntryUpdatedListener; +import javax.cache.event.EventType; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteException; +import org.apache.ignite.cache.CacheAtomicWriteOrderMode; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheWriteSynchronizationMode; +import org.apache.ignite.cache.query.ContinuousQuery; +import org.apache.ignite.cache.query.QueryCursor; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.managers.communication.GridIoMessage; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicFullUpdateRequest; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +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.communication.tcp.TcpCommunicationSpi; +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.common.GridCommonAbstractTest; + +/** + * Test coverages case when update discarded + */ +public class CacheContinuousQueryClockModeConflictSelfTest extends GridCommonAbstractTest { + /** */ + private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); + + /** */ + public static final int KEYS = 0; + + /** */ + public static final int ITERATION_CNT = 50; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder); + cfg.setCommunicationSpi(new TestCommunicationSpi()); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + // Node should start exactly in the same order. + startGrid(0); + startGrid(1); + startGrid(2); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + } + + /** + * @throws Exception If failed. + */ + public void testIgniteLocalContinuousQuery() throws Exception { + for (int i = 0; i < ITERATION_CNT; i++) { + String cacheName = "test-cache-" + i; + + IgniteCache cache2 = ignite(2).createCache(cacheConfiguration(cacheName)); + + try { + IgniteCache cache0 = ignite(0).cache(cacheName); + IgniteCache cache1 = ignite(1).cache(cacheName); + + final BlockingQueue> evts0 = new LinkedBlockingQueue<>(); + QueryCursor query = registerQuery(cache0, evts0); + + final BlockingQueue> evts1 = new LinkedBlockingQueue<>(); + QueryCursor query1 = registerQuery(cache1, evts1); + + final BlockingQueue> evts2 = new LinkedBlockingQueue<>(); + QueryCursor query2 = registerQuery(cache2, evts2); + + Integer key = keyForNode(affinity(cache1), new AtomicInteger(0), + grid(0).cluster().localNode()); + + // This update should CacheVersion with orderId == 3. + cache2.put(key, 42); + + // The update will be have orderId == 2 and will be discarded. + cache1.put(key, 41); + + assertEquals("Entry was updated.", 42, cache1.get(key)); + + cache0.remove(key); + + assertNull("Entry was not removed.", cache1.get(key)); + + CacheEntryEvent e = evts0.poll(200, TimeUnit.MILLISECONDS); + + assertNotNull("Missed events.", e); + assertEquals("Invalid event.", EventType.CREATED, e.getEventType()); + assertNull("Invalid event.", e.getOldValue()); + assertEquals("Invalid event.", Integer.valueOf(42), e.getValue()); + + e = evts0.poll(200, TimeUnit.MILLISECONDS); + + assertNotNull("Missed events.", e); + assertEquals("Invalid event.", EventType.REMOVED, e.getEventType()); + assertEquals("Invalid event.", Integer.valueOf(42), e.getOldValue()); + + assertTrue("Received extra events.", evts1.isEmpty()); + assertTrue("Received extra events.", evts2.isEmpty()); + + query.close(); + query1.close(); + query2.close(); + } + finally { + grid(0).destroyCache(cacheName); + } + } + } + + private QueryCursor registerQuery(IgniteCache cache, final BlockingQueue> evts) { + ContinuousQuery qry = new ContinuousQuery(); + + qry.setLocalListener(new CacheEntryUpdatedListener() { + @Override public void onUpdated(Iterable iterable) throws CacheEntryListenerException { + for (Object o : iterable) + evts.add((CacheEntryEvent)o); + } + }); + + qry.setLocal(true); + + return cache.query(qry); + } + + /** + * @param name Cache name. + * @return Cache configuration. + */ + private CacheConfiguration cacheConfiguration(String name) { + return new CacheConfiguration(name) + .setAtomicityMode(CacheAtomicityMode.ATOMIC) + .setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC) + .setAtomicWriteOrderMode(CacheAtomicWriteOrderMode.CLOCK) + .setBackups(0); + } + + /** + * + */ + private static class TestCommunicationSpi extends TcpCommunicationSpi { + /** {@inheritDoc} */ + @Override public void sendMessage(ClusterNode node, Message msg, IgniteInClosure ackC) + throws IgniteSpiException { + Object msg0 = ((GridIoMessage)msg).message(); + + if (msg0 instanceof GridNearAtomicFullUpdateRequest) { + GridCacheVersion ver = ((GridNearAtomicFullUpdateRequest)msg0).updateVersion(); + + ver.globalTime(142); + } + + super.sendMessage(node, msg, ackC); + } + } +} 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..2df38d767299b 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 @@ -258,15 +258,15 @@ public void testFirstFilteredEvent() throws Exception { qryClnCache.put(key, -1); qryClnCache.put(keys.get(0), 100); - } - GridTestUtils.waitForCondition(new GridAbsPredicate() { - @Override public boolean apply() { - return lsnr.evts.size() == 1; - } - }, 5000); + GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + return lsnr.evts.size() == 1; + } + }, 5000); - assertEquals(lsnr.evts.size(), 1); + assertEquals(1, lsnr.evts.size()); + } } /** 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..2460a8045cd1f 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 @@ -145,7 +145,37 @@ public void testAtomicOnheapTwoBackup() throws Exception { CacheConfiguration ccfg = cacheConfiguration(PARTITIONED, 2, ATOMIC, ONHEAP_TIERED, PRIMARY_SYNC); - doOrderingTest(ccfg, false); + doOrderingTest(ccfg, false, false); + } + + /** + * @throws Exception If failed. + */ + public void testAtomicOnheapTwoBackupLocalQuery() throws Exception { + CacheConfiguration ccfg = cacheConfiguration(PARTITIONED, 2, ATOMIC, + ONHEAP_TIERED, PRIMARY_SYNC); + + doOrderingTest(ccfg, false, true); + } + + /** + * @throws Exception If failed. + */ + public void testAtomicOnheapTwoBackupLocalQueryAsync() throws Exception { + CacheConfiguration ccfg = cacheConfiguration(PARTITIONED, 2, ATOMIC, + ONHEAP_TIERED, PRIMARY_SYNC); + + doOrderingTest(ccfg, true, true); + } + + /** + * @throws Exception If failed. + */ + public void testAtomicOnheapTwoBackupLocalQueryAsyncFullSync() throws Exception { + CacheConfiguration ccfg = cacheConfiguration(PARTITIONED, 2, ATOMIC, + ONHEAP_TIERED, FULL_SYNC); + + doOrderingTest(ccfg, false, true); } /** @@ -155,7 +185,7 @@ public void testAtomicOffheapTwoBackup() throws Exception { CacheConfiguration ccfg = cacheConfiguration(PARTITIONED, 2, ATOMIC, OFFHEAP_TIERED, PRIMARY_SYNC); - doOrderingTest(ccfg, false); + doOrderingTest(ccfg, false, false); } /** @@ -165,7 +195,7 @@ public void testAtomicOffheapValuesTwoBackup() throws Exception { CacheConfiguration ccfg = cacheConfiguration(PARTITIONED, 2, ATOMIC, OFFHEAP_VALUES, PRIMARY_SYNC); - doOrderingTest(ccfg, false); + doOrderingTest(ccfg, false, false); } /** @@ -175,7 +205,7 @@ public void testAtomicReplicatedOffheap() throws Exception { CacheConfiguration ccfg = cacheConfiguration(REPLICATED, 0, ATOMIC, OFFHEAP_TIERED, PRIMARY_SYNC); - doOrderingTest(ccfg, false); + doOrderingTest(ccfg, false, false); } /** @@ -185,7 +215,7 @@ public void testTxOnheapTwoBackup() throws Exception { CacheConfiguration ccfg = cacheConfiguration(PARTITIONED, 2, TRANSACTIONAL, ONHEAP_TIERED, FULL_SYNC); - doOrderingTest(ccfg, false); + doOrderingTest(ccfg, false, false); } /** @@ -195,7 +225,7 @@ public void testTxOnheapWithoutBackup() throws Exception { CacheConfiguration ccfg = cacheConfiguration(PARTITIONED, 0, TRANSACTIONAL, ONHEAP_TIERED, PRIMARY_SYNC); - doOrderingTest(ccfg, false); + doOrderingTest(ccfg, false, false); } /** @@ -205,7 +235,7 @@ public void testTxOnheapWithoutBackupFullSync() throws Exception { CacheConfiguration ccfg = cacheConfiguration(PARTITIONED, 0, TRANSACTIONAL, ONHEAP_TIERED, FULL_SYNC); - doOrderingTest(ccfg, false); + doOrderingTest(ccfg, false, false); } // ASYNC @@ -217,7 +247,7 @@ public void testAtomicOnheapTwoBackupAsync() throws Exception { CacheConfiguration ccfg = cacheConfiguration(PARTITIONED, 2, ATOMIC, ONHEAP_TIERED, PRIMARY_SYNC); - doOrderingTest(ccfg, true); + doOrderingTest(ccfg, true, false); } /** @@ -227,7 +257,7 @@ public void testAtomicOnheapTwoBackupAsyncFullSync() throws Exception { CacheConfiguration ccfg = cacheConfiguration(PARTITIONED, 2, ATOMIC, ONHEAP_TIERED, FULL_SYNC); - doOrderingTest(ccfg, true); + doOrderingTest(ccfg, true, false); } /** @@ -237,7 +267,7 @@ public void testAtomicOffheapTwoBackupAsync() throws Exception { CacheConfiguration ccfg = cacheConfiguration(PARTITIONED, 2, ATOMIC, OFFHEAP_TIERED, PRIMARY_SYNC); - doOrderingTest(ccfg, true); + doOrderingTest(ccfg, true, false); } /** @@ -247,7 +277,7 @@ public void testAtomicOffheapTwoBackupAsyncFullSync() throws Exception { CacheConfiguration ccfg = cacheConfiguration(PARTITIONED, 2, ATOMIC, OFFHEAP_TIERED, FULL_SYNC); - doOrderingTest(ccfg, true); + doOrderingTest(ccfg, true, false); } /** @@ -257,7 +287,7 @@ public void testAtomicOffheapValuesTwoBackupAsync() throws Exception { CacheConfiguration ccfg = cacheConfiguration(PARTITIONED, 2, ATOMIC, OFFHEAP_VALUES, PRIMARY_SYNC); - doOrderingTest(ccfg, true); + doOrderingTest(ccfg, true, false); } /** @@ -267,7 +297,7 @@ public void testAtomicOffheapValuesTwoBackupAsyncFullSync() throws Exception { CacheConfiguration ccfg = cacheConfiguration(PARTITIONED, 2, ATOMIC, OFFHEAP_VALUES, FULL_SYNC); - doOrderingTest(ccfg, true); + doOrderingTest(ccfg, true, false); } /** @@ -277,7 +307,7 @@ public void testAtomicReplicatedAsync() throws Exception { CacheConfiguration ccfg = cacheConfiguration(REPLICATED, 0, ATOMIC, ONHEAP_TIERED, PRIMARY_SYNC); - doOrderingTest(ccfg, true); + doOrderingTest(ccfg, true, false); } /** @@ -287,7 +317,7 @@ public void testAtomicReplicatedAsyncFullSync() throws Exception { CacheConfiguration ccfg = cacheConfiguration(REPLICATED, 0, ATOMIC, ONHEAP_TIERED, FULL_SYNC); - doOrderingTest(ccfg, true); + doOrderingTest(ccfg, true, false); } /** @@ -297,7 +327,7 @@ public void testAtomicReplicatedOffheapAsync() throws Exception { CacheConfiguration ccfg = cacheConfiguration(REPLICATED, 0, ATOMIC, OFFHEAP_TIERED, PRIMARY_SYNC); - doOrderingTest(ccfg, true); + doOrderingTest(ccfg, true, false); } /** @@ -307,7 +337,7 @@ public void testAtomicOnheapWithoutBackupAsync() throws Exception { CacheConfiguration ccfg = cacheConfiguration(PARTITIONED, 0, ATOMIC, ONHEAP_TIERED, PRIMARY_SYNC); - doOrderingTest(ccfg, true); + doOrderingTest(ccfg, true, false); } /** @@ -317,7 +347,7 @@ public void testTxOnheapTwoBackupAsync() throws Exception { CacheConfiguration ccfg = cacheConfiguration(PARTITIONED, 2, TRANSACTIONAL, ONHEAP_TIERED, PRIMARY_SYNC); - doOrderingTest(ccfg, true); + doOrderingTest(ccfg, true, false); } /** @@ -327,7 +357,7 @@ public void testTxOnheapAsync() throws Exception { CacheConfiguration ccfg = cacheConfiguration(PARTITIONED, 0, TRANSACTIONAL, ONHEAP_TIERED, PRIMARY_SYNC); - doOrderingTest(ccfg, true); + doOrderingTest(ccfg, true, false); } /** @@ -337,17 +367,17 @@ public void testTxOnheapAsyncFullSync() throws Exception { CacheConfiguration ccfg = cacheConfiguration(PARTITIONED, 0, TRANSACTIONAL, ONHEAP_TIERED, FULL_SYNC); - doOrderingTest(ccfg, true); + doOrderingTest(ccfg, true, false); } /** * @param ccfg Cache configuration. * @param async Async filter. + * @param localQry Start local query. + * * @throws Exception If failed. */ - protected void doOrderingTest( - final CacheConfiguration ccfg, - final boolean async) + protected void doOrderingTest(final CacheConfiguration ccfg, final boolean async, final boolean localQry) throws Exception { ignite(0).createCache(ccfg); @@ -372,15 +402,17 @@ protected void doOrderingTest( qry.setLocalListener(new TestCacheAsyncEventListener(queue, qryCntr)); qry.setRemoteFilterFactory(FactoryBuilder.factoryOf( - new CacheTestRemoteFilterAsync(ccfg.getName()))); + new CacheTestRemoteFilterAsync(ccfg.getName(), localQry))); } else { qry.setLocalListener(new TestCacheEventListener(queue, qryCntr)); qry.setRemoteFilterFactory(FactoryBuilder.factoryOf( - new CacheTestRemoteFilter(ccfg.getName()))); + new CacheTestRemoteFilter(ccfg.getName(), localQry))); } + qry.setLocal(localQry); + rcvdEvts.add(queue); IgniteCache cache = grid(idx).cache(ccfg.getName()); @@ -448,14 +480,17 @@ protected void doOrderingTest( f.get(15, TimeUnit.SECONDS); - GridTestUtils.waitForCondition(new PA() { + boolean res = GridTestUtils.waitForCondition(new PA() { @Override public boolean apply() { - return qryCntr.get() >= ITERATION_CNT * threadCnt * LISTENER_CNT * NODES; + return qryCntr.get() >= ITERATION_CNT * threadCnt * LISTENER_CNT * + (localQry ? 1 : NODES); } }, 1000L); + assertTrue("Failed to wait all events.", res); + for (BlockingQueue> queue : rcvdEvts) - checkEvents(queue, ITERATION_CNT * threadCnt); + checkEvents(queue, ITERATION_CNT * threadCnt, localQry); assertFalse("Ordering invocations of filter broken.", fail); } @@ -469,9 +504,12 @@ protected void doOrderingTest( /** * @param queue Event queue. + * @param localQry Local query flag. * @throws Exception If failed. */ - private void checkEvents(BlockingQueue> queue, int expCnt) + private void checkEvents(BlockingQueue> queue, + int expCnt, + boolean localQry) throws Exception { CacheEntryEvent evt; int cnt = 0; @@ -483,11 +521,17 @@ private void checkEvents(BlockingQueue e.getValue().val1) + fail = true; + } + else if (!new QueryTestValue(prevVal.val1 + 1).equals(e.getValue())) fail = true; } } 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 6b2fea0a5ab55..3767bafe97199 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.CacheContinuousQueryClockModeConflictSelfTest; 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; @@ -125,6 +126,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(IgniteCacheContinuousQueryBackupQueueTest.class); suite.addTestSuite(IgniteCacheContinuousQueryNoUnsubscribeTest.class); suite.addTestSuite(ContinuousQueryPeerClassLoadingTest.class); + suite.addTestSuite(CacheContinuousQueryClockModeConflictSelfTest.class); return suite; } From 5858efd406bb54352de14a0a7e7f21c2ac7bf899 Mon Sep 17 00:00:00 2001 From: sboikov Date: Fri, 16 Dec 2016 19:23:29 +0300 Subject: [PATCH 135/357] 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 136/357] 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 137/357] 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 138/357] 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 139/357] 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 140/357] 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 141/357] 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 142/357] 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 143/357] 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 144/357] 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 145/357] 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 146/357] 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 147/357] 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 a64339449be8fa602cab3f2868c5f74004a7b747 Mon Sep 17 00:00:00 2001 From: Igor Sapego Date: Fri, 23 Jun 2017 16:57:49 +0300 Subject: [PATCH 148/357] IGNITE-4370: Implemented writing of batch of parameters for ODBC. (cherry picked from commit c10be5780589cc84e7929e234e4411d515166e0b) (cherry picked from commit d268b32cb252a5f06887d2b803d27ddc20ded95f) --- .../processors/odbc/OdbcMessageParser.java | 57 ++- .../processors/odbc/OdbcNioListener.java | 3 +- .../odbc/OdbcQueryExecuteBatchRequest.java | 95 ++++ .../odbc/OdbcQueryExecuteBatchResult.java | 75 ++++ .../odbc/OdbcQueryExecuteRequest.java | 6 +- .../odbc/OdbcQueryExecuteResult.java | 2 +- .../internal/processors/odbc/OdbcRequest.java | 3 + .../processors/odbc/OdbcRequestHandler.java | 106 ++++- .../src/application_data_buffer_test.cpp | 146 ++----- .../cpp/odbc-test/src/column_test.cpp | 21 +- .../cpp/odbc-test/src/queries_test.cpp | 386 ++++++++++++++++- .../platforms/cpp/odbc-test/src/row_test.cpp | 9 +- modules/platforms/cpp/odbc/Makefile.am | 3 + .../platforms/cpp/odbc/include/Makefile.am | 2 + .../ignite/odbc/app/application_data_buffer.h | 38 +- .../odbc/include/ignite/odbc/app/parameter.h | 20 +- .../include/ignite/odbc/app/parameter_set.h | 268 ++++++++++++ .../odbc/include/ignite/odbc/common_types.h | 4 +- .../ignite/odbc/diagnostic/diagnosable.h | 4 +- .../cpp/odbc/include/ignite/odbc/message.h | 408 +++++++----------- .../include/ignite/odbc/query/batch_query.h | 160 +++++++ .../include/ignite/odbc/query/data_query.h | 20 +- .../odbc/include/ignite/odbc/query/query.h | 3 + .../ignite/odbc/query/type_info_query.h | 2 +- .../cpp/odbc/include/ignite/odbc/statement.h | 64 +-- .../cpp/odbc/project/vs/odbc.vcxproj | 5 + .../cpp/odbc/project/vs/odbc.vcxproj.filters | 15 + .../odbc/src/app/application_data_buffer.cpp | 90 +++- .../platforms/cpp/odbc/src/app/parameter.cpp | 11 +- .../cpp/odbc/src/app/parameter_set.cpp | 242 +++++++++++ .../cpp/odbc/src/config/connection_info.cpp | 36 ++ modules/platforms/cpp/odbc/src/message.cpp | 366 ++++++++++++++++ modules/platforms/cpp/odbc/src/odbc.cpp | 16 +- .../cpp/odbc/src/query/batch_query.cpp | 198 +++++++++ .../cpp/odbc/src/query/data_query.cpp | 13 +- modules/platforms/cpp/odbc/src/statement.cpp | 267 ++++++++---- 36 files changed, 2599 insertions(+), 565 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteBatchRequest.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteBatchResult.java create mode 100644 modules/platforms/cpp/odbc/include/ignite/odbc/app/parameter_set.h create mode 100644 modules/platforms/cpp/odbc/include/ignite/odbc/query/batch_query.h create mode 100644 modules/platforms/cpp/odbc/src/app/parameter_set.cpp create mode 100644 modules/platforms/cpp/odbc/src/message.cpp create mode 100644 modules/platforms/cpp/odbc/src/query/batch_query.cpp diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcMessageParser.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcMessageParser.java index 7b863d6b2af21..0cb89d71f626b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcMessageParser.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcMessageParser.java @@ -29,6 +29,7 @@ import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl; import java.util.Collection; +import org.jetbrains.annotations.NotNull; /** * ODBC message parser. @@ -99,23 +100,37 @@ public OdbcRequest decode(byte[] msg) { } OdbcRequest res; - + switch (cmd) { case OdbcRequest.EXECUTE_SQL_QUERY: { String cache = reader.readString(); String sql = reader.readString(); - int argsNum = reader.readInt(); - - Object[] params = new Object[argsNum]; + int paramNum = reader.readInt(); - for (int i = 0; i < argsNum; ++i) - params[i] = reader.readObjectDetached(); + Object[] params = readParameterRow(reader, paramNum); res = new OdbcQueryExecuteRequest(cache, sql, params); break; } + case OdbcRequest.QRY_EXEC_BATCH: { + String schema = reader.readString(); + String sql = reader.readString(); + int paramRowLen = reader.readInt(); + int rowNum = reader.readInt(); + boolean last = reader.readBoolean(); + + Object[][] params = new Object[rowNum][]; + + for (int i = 0; i < rowNum; ++i) + params[i] = readParameterRow(reader, paramRowLen); + + res = new OdbcQueryExecuteBatchRequest(schema, sql, last, params); + + break; + } + case OdbcRequest.FETCH_SQL_QUERY: { long queryId = reader.readLong(); int pageSize = reader.readInt(); @@ -170,6 +185,21 @@ public OdbcRequest decode(byte[] msg) { return res; } + /** + * Read row of parameters using reader. + * @param reader reader + * @param paramNum Number of parameters in a row + * @return Parameters array. + */ + @NotNull private static Object[] readParameterRow(BinaryReaderExImpl reader, int paramNum) { + Object[] params = new Object[paramNum]; + + for (int i = 0; i < paramNum; ++i) + params[i] = reader.readObjectDetached(); + + return params; + } + /** * Encode OdbcResponse to byte array. * @@ -218,9 +248,9 @@ else if (res0 instanceof OdbcQueryExecuteResult) { OdbcQueryExecuteResult res = (OdbcQueryExecuteResult) res0; if (log.isDebugEnabled()) - log.debug("Resulting query ID: " + res.getQueryId()); + log.debug("Resulting query ID: " + res.queryId()); - writer.writeLong(res.getQueryId()); + writer.writeLong(res.queryId()); Collection metas = res.getColumnsMetadata(); @@ -231,6 +261,17 @@ else if (res0 instanceof OdbcQueryExecuteResult) { for (OdbcColumnMeta meta : metas) meta.write(writer); } + else if (res0 instanceof OdbcQueryExecuteBatchResult) { + OdbcQueryExecuteBatchResult res = (OdbcQueryExecuteBatchResult) res0; + + writer.writeBoolean(res.errorMessage() == null); + writer.writeLong(res.rowsAffected()); + + if (res.errorMessage() != null) { + writer.writeLong(res.errorSetIdx()); + writer.writeString(res.errorMessage()); + } + } else if (res0 instanceof OdbcQueryFetchResult) { OdbcQueryFetchResult res = (OdbcQueryFetchResult) res0; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcNioListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcNioListener.java index 378e5f25ebebc..a68cf888af48b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcNioListener.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcNioListener.java @@ -23,6 +23,7 @@ import org.apache.ignite.internal.util.nio.GridNioServerListenerAdapter; import org.apache.ignite.internal.util.nio.GridNioSession; import org.apache.ignite.internal.util.nio.GridNioSessionMetaKey; +import org.apache.ignite.internal.util.typedef.internal.U; import org.jetbrains.annotations.Nullable; import java.util.concurrent.atomic.AtomicLong; @@ -100,7 +101,7 @@ public OdbcNioListener(GridKernalContext ctx, GridSpinBusyLock busyLock, int max req = parser.decode(msg); } catch (Exception e) { - log.error("Failed to parse message [id=" + reqId + ", err=" + e + ']'); + U.error(log, "Failed to parse message [id=" + reqId + ']', e); ses.close(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteBatchRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteBatchRequest.java new file mode 100644 index 0000000000000..0ace947126349 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteBatchRequest.java @@ -0,0 +1,95 @@ +package org.apache.ignite.internal.processors.odbc; + +/* + * 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. + */ + +import org.apache.ignite.internal.util.tostring.GridToStringExclude; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.jetbrains.annotations.Nullable; + +/** + * ODBC query execute request with the batch of parameters. + */ +public class OdbcQueryExecuteBatchRequest extends OdbcRequest { + /** Schema. */ + @GridToStringInclude(sensitive = true) + private final String schema; + + /** Sql query. */ + @GridToStringInclude(sensitive = true) + private final String sqlQry; + + /** Last param page flag. */ + private final boolean last; + + /** Sql query arguments. */ + @GridToStringExclude + private final Object[][] args; + + /** + * @param schema Schema. + * @param sqlQry SQL query. + * @param last Last page flag. + * @param args Arguments list. + */ + public OdbcQueryExecuteBatchRequest(@Nullable String schema, String sqlQry, boolean last, Object[][] args) { + super(QRY_EXEC_BATCH); + + assert sqlQry != null : "SQL query should not be null"; + assert args != null : "Parameters should not be null"; + + this.schema = schema; + this.sqlQry = sqlQry; + this.last = last; + this.args = args; + } + + /** + * @return Sql query. + */ + public String sqlQuery() { + return sqlQry; + } + + /** + * @return Sql query arguments. + */ + public Object[][] arguments() { + return args; + } + + /** + * @return Schema. + */ + @Nullable + public String schema() { + return schema; + } + + /** + * @return Last page flag. + */ + public boolean last() { + return last; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(OdbcQueryExecuteBatchRequest.class, this, "args", args, true); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteBatchResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteBatchResult.java new file mode 100644 index 0000000000000..6fc3873dea267 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteBatchResult.java @@ -0,0 +1,75 @@ +/* + * 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.odbc; + +import org.jetbrains.annotations.Nullable; + +/** + * ODBC query execute with batch of parameters result. + */ +public class OdbcQueryExecuteBatchResult { + /** Rows affected. */ + private final long rowsAffected; + + /** Index of the set which caused an error. */ + private final long errorSetIdx; + + /** Error message. */ + private final String errorMessage; + + /** + * @param rowsAffected Number of rows affected by the query. + */ + public OdbcQueryExecuteBatchResult(long rowsAffected) { + this.rowsAffected = rowsAffected; + this.errorSetIdx = -1; + this.errorMessage = null; + } + + /** + * @param rowsAffected Number of rows affected by the query. + * @param errorSetIdx Sets processed. + * @param errorMessage Error message. + */ + public OdbcQueryExecuteBatchResult(long rowsAffected, long errorSetIdx, String errorMessage) { + this.rowsAffected = rowsAffected; + this.errorSetIdx = errorSetIdx; + this.errorMessage = errorMessage; + } + + /** + * @return Number of rows affected by the query. + */ + public long rowsAffected() { + return rowsAffected; + } + + /** + * @return Index of the set which caused an error or -1 if no error occurred. + */ + public long errorSetIdx() { + return errorSetIdx; + } + + /** + * @return Error message. + */ + @Nullable public String errorMessage() { + return errorMessage; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteRequest.java index c0d1c601c92b7..029135cd52e80 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteRequest.java @@ -42,10 +42,12 @@ public class OdbcQueryExecuteRequest extends OdbcRequest { * @param sqlQry SQL query. * @param args Arguments list. */ - public OdbcQueryExecuteRequest(String cacheName, String sqlQry, Object[] args) { + public OdbcQueryExecuteRequest(@Nullable String cacheName, String sqlQry, Object[] args) { super(EXECUTE_SQL_QUERY); - this.cacheName = cacheName.isEmpty() ? null : cacheName; + assert sqlQry != null : "SQL query should not be null"; + + this.cacheName = cacheName; this.sqlQry = sqlQry; this.args = args; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteResult.java index a4038123d2d4d..efa432b615614 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteResult.java @@ -41,7 +41,7 @@ public OdbcQueryExecuteResult(long queryId, Collection columnsMe /** * @return Query ID. */ - public long getQueryId() { + public long queryId() { return queryId; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequest.java index f056fedf079bd..3c0c12e09cd28 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequest.java @@ -42,6 +42,9 @@ public class OdbcRequest { /** Get parameters meta. */ public static final int GET_PARAMS_META = 7; + /** Execute sql query with the batch of parameters. */ + public static final int QRY_EXEC_BATCH = 8; + /** Command. */ private final int cmd; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java index f922d9acc09be..ccced5eacac77 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java @@ -18,6 +18,8 @@ package org.apache.ignite.internal.processors.odbc; import org.apache.ignite.IgniteCache; + +import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.cache.query.QueryCursor; import org.apache.ignite.cache.query.SqlFieldsQuery; @@ -36,7 +38,11 @@ import java.sql.ParameterMetaData; import java.sql.PreparedStatement; import java.sql.Types; -import java.util.*; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicLong; @@ -104,6 +110,9 @@ public OdbcResponse handle(long reqId, OdbcRequest req) { case HANDSHAKE: return performHandshake(reqId, (OdbcHandshakeRequest)req); + case QRY_EXEC_BATCH: + return executeBatchQuery((OdbcQueryExecuteBatchRequest)req); + case EXECUTE_SQL_QUERY: return executeQuery(reqId, (OdbcQueryExecuteRequest)req); @@ -229,6 +238,101 @@ private OdbcResponse executeQuery(long reqId, OdbcQueryExecuteRequest req) { } } + /** + * {@link OdbcQueryExecuteBatchRequest} command handler. + * + * @param req Execute query request. + * @return Response. + */ + private OdbcResponse executeBatchQuery(OdbcQueryExecuteBatchRequest req) { + long rowsAffected = 0; + int currentSet = 0; + + try { + String sql = OdbcEscapeUtils.parse(req.sqlQuery()); + + if (log.isDebugEnabled()) + log.debug("ODBC query parsed [original=" + req.sqlQuery() + ", parsed=" + sql + ']'); + + SqlFieldsQuery qry = new SqlFieldsQuery(sql); + + qry.setDistributedJoins(distributedJoins); + qry.setEnforceJoinOrder(enforceJoinOrder); + + IgniteCache cache0 = ctx.grid().cache(req.schema()); + + if (cache0 == null) + return new OdbcResponse(OdbcResponse.STATUS_FAILED, + "Cache doesn't exist (did you configure it?): " + req.schema()); + + IgniteCache cache = cache0.withKeepBinary(); + + if (cache == null) + return new OdbcResponse(OdbcResponse.STATUS_FAILED, + "Can not get cache with keep binary: " + req.schema()); + + Object[][] paramSet = req.arguments(); + + if (paramSet.length <= 0) + throw new IgniteException("Batch execute request with non-positive batch length. [len=" + + paramSet.length + ']'); + + // Getting meta and do the checks for the first execution. + qry.setArgs(paramSet[0]); + + QueryCursorImpl> qryCur = (QueryCursorImpl>)cache.query(qry); + + if (qryCur.isQuery()) + throw new IgniteException("Batching of parameters only supported for DML statements. [query=" + + req.sqlQuery() + ']'); + + rowsAffected += getRowsAffected(qryCur); + + for (currentSet = 1; currentSet < paramSet.length; ++currentSet) + { + qry.setArgs(paramSet[currentSet]); + + QueryCursor> cur = cache.query(qry); + + rowsAffected += getRowsAffected(cur); + } + + OdbcQueryExecuteBatchResult res = new OdbcQueryExecuteBatchResult(rowsAffected); + + return new OdbcResponse(res); + } + catch (Exception e) { + U.error(log, "Failed to execute SQL query [req=" + req + ']', e); + + OdbcQueryExecuteBatchResult res = new OdbcQueryExecuteBatchResult(rowsAffected, currentSet, + e.getMessage()); + + return new OdbcResponse(res); + } + } + + /** + * Get affected rows for DML statement. + * @param qryCur Cursor. + * @return Number of table rows affected. + */ + private static long getRowsAffected(QueryCursor> qryCur) { + Iterator> iter = qryCur.iterator(); + + if (iter.hasNext()) { + List res = iter.next(); + + if (res.size() > 0) { + Long affected = (Long) res.get(0); + + if (affected != null) + return affected; + } + } + + return 0; + } + /** * {@link OdbcQueryCloseRequest} command handler. * diff --git a/modules/platforms/cpp/odbc-test/src/application_data_buffer_test.cpp b/modules/platforms/cpp/odbc-test/src/application_data_buffer_test.cpp index 82521be53d6e7..0149d6a801bfe 100644 --- a/modules/platforms/cpp/odbc-test/src/application_data_buffer_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/application_data_buffer_test.cpp @@ -44,9 +44,8 @@ BOOST_AUTO_TEST_CASE(TestPutIntToString) { char buffer[1024]; SqlLen reslen = 0; - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, buffer, sizeof(buffer), &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, buffer, sizeof(buffer), &reslen); appBuf.PutInt8(12); BOOST_CHECK(!strcmp(buffer, "12")); @@ -77,9 +76,8 @@ BOOST_AUTO_TEST_CASE(TestPutFloatToString) { char buffer[1024]; SqlLen reslen = 0; - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, buffer, sizeof(buffer), &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, buffer, sizeof(buffer), &reslen); appBuf.PutFloat(12.42f); BOOST_CHECK(!strcmp(buffer, "12.42")); @@ -102,9 +100,8 @@ BOOST_AUTO_TEST_CASE(TestPutGuidToString) { char buffer[1024]; SqlLen reslen = 0; - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, buffer, sizeof(buffer), &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, buffer, sizeof(buffer), &reslen); ignite::Guid guid(0x1da1ef8f39ff4d62ULL, 0x8b72e8e9f3371801ULL); @@ -119,7 +116,7 @@ BOOST_AUTO_TEST_CASE(TestGetGuidFromString) char buffer[] = "1da1ef8f-39ff-4d62-8b72-e8e9f3371801"; SqlLen reslen = sizeof(buffer) - 1; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, buffer, sizeof(buffer) - 1, &reslen, 0); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, buffer, sizeof(buffer) - 1, &reslen); ignite::Guid guid = appBuf.GetGuid(); @@ -130,9 +127,8 @@ BOOST_AUTO_TEST_CASE(TestPutBinaryToString) { char buffer[1024]; SqlLen reslen = 0; - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, buffer, sizeof(buffer), &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, buffer, sizeof(buffer), &reslen); uint8_t binary[] = { 0x21, 0x84, 0xF4, 0xDC, 0x01, 0x00, 0xFF, 0xF0 }; @@ -146,9 +142,8 @@ BOOST_AUTO_TEST_CASE(TestPutStringToString) { char buffer[1024]; SqlLen reslen = 0; - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, buffer, sizeof(buffer), &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, buffer, sizeof(buffer), &reslen); std::string testString("Test string"); @@ -162,9 +157,8 @@ BOOST_AUTO_TEST_CASE(TestPutStringToWstring) { wchar_t buffer[1024]; SqlLen reslen = 0; - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_WCHAR, buffer, sizeof(buffer), &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_WCHAR, buffer, sizeof(buffer), &reslen); std::string testString("Test string"); @@ -176,9 +170,8 @@ BOOST_AUTO_TEST_CASE(TestPutStringToLong) { long numBuf; SqlLen reslen = 0; - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_SIGNED_LONG, &numBuf, sizeof(numBuf), &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_SIGNED_LONG, &numBuf, sizeof(numBuf), &reslen); appBuf.PutString("424242424"); BOOST_CHECK(numBuf == 424242424L); @@ -191,9 +184,8 @@ BOOST_AUTO_TEST_CASE(TestPutStringToTiny) { int8_t numBuf; SqlLen reslen = 0; - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_SIGNED_TINYINT, &numBuf, sizeof(numBuf), &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_SIGNED_TINYINT, &numBuf, sizeof(numBuf), &reslen); appBuf.PutString("12"); BOOST_CHECK(numBuf == 12); @@ -206,9 +198,8 @@ BOOST_AUTO_TEST_CASE(TestPutStringToFloat) { float numBuf; SqlLen reslen = 0; - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_FLOAT, &numBuf, sizeof(numBuf), &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_FLOAT, &numBuf, sizeof(numBuf), &reslen); appBuf.PutString("12.21"); BOOST_CHECK_CLOSE_FRACTION(numBuf, 12.21, FLOAT_PRECISION); @@ -221,9 +212,8 @@ BOOST_AUTO_TEST_CASE(TestPutIntToFloat) { float numBuf; SqlLen reslen = 0; - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_FLOAT, &numBuf, sizeof(numBuf), &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_FLOAT, &numBuf, sizeof(numBuf), &reslen); appBuf.PutInt8(5); BOOST_CHECK_CLOSE_FRACTION(numBuf, 5.0, FLOAT_PRECISION); @@ -248,9 +238,8 @@ BOOST_AUTO_TEST_CASE(TestPutFloatToShort) { short numBuf; SqlLen reslen = 0; - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_SIGNED_SHORT, &numBuf, sizeof(numBuf), &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_SIGNED_SHORT, &numBuf, sizeof(numBuf), &reslen); appBuf.PutDouble(5.42); BOOST_CHECK(numBuf == 5); @@ -270,7 +259,7 @@ BOOST_AUTO_TEST_CASE(TestPutDecimalToDouble) double numBuf; SqlLen reslen = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_DOUBLE, &numBuf, sizeof(numBuf), &reslen, 0); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_DOUBLE, &numBuf, sizeof(numBuf), &reslen); common::Decimal decimal; @@ -299,7 +288,7 @@ BOOST_AUTO_TEST_CASE(TestPutDecimalToLong) long numBuf; SqlLen reslen = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_SIGNED_LONG, &numBuf, sizeof(numBuf), &reslen, 0); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_SIGNED_LONG, &numBuf, sizeof(numBuf), &reslen); common::Decimal decimal; @@ -326,7 +315,7 @@ BOOST_AUTO_TEST_CASE(TestPutDecimalToString) char strBuf[64]; SqlLen reslen = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &strBuf, sizeof(strBuf), &reslen, 0); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &strBuf, sizeof(strBuf), &reslen); common::Decimal decimal; @@ -353,7 +342,7 @@ BOOST_AUTO_TEST_CASE(TestPutDecimalToNumeric) SQL_NUMERIC_STRUCT buf; SqlLen reslen = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_NUMERIC, &buf, sizeof(buf), &reslen, 0); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_NUMERIC, &buf, sizeof(buf), &reslen); common::Decimal decimal; @@ -404,7 +393,7 @@ BOOST_AUTO_TEST_CASE(TestPutDateToString) char strBuf[64] = { 0 }; SqlLen reslen = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &strBuf, sizeof(strBuf), &reslen, 0); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &strBuf, sizeof(strBuf), &reslen); Date date = common::MakeDateGmt(1999, 2, 22); @@ -418,7 +407,7 @@ BOOST_AUTO_TEST_CASE(TestPutTimestampToString) char strBuf[64] = { 0 }; SqlLen reslen = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &strBuf, sizeof(strBuf), &reslen, 0); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &strBuf, sizeof(strBuf), &reslen); Timestamp date = common::MakeTimestampGmt(2018, 11, 1, 17, 45, 59); @@ -432,10 +421,7 @@ BOOST_AUTO_TEST_CASE(TestPutDateToDate) SQL_DATE_STRUCT buf = { 0 }; SqlLen reslen = sizeof(buf); - int offset = 0; - int* offsetPtr = &offset; - - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_TDATE, &buf, sizeof(buf), &reslen, &offsetPtr); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_TDATE, &buf, sizeof(buf), &reslen); Date date = common::MakeDateGmt(1984, 5, 27); @@ -451,10 +437,7 @@ BOOST_AUTO_TEST_CASE(TestPutTimestampToDate) SQL_DATE_STRUCT buf = { 0 }; SqlLen reslen = sizeof(buf); - int offset = 0; - int* offsetPtr = &offset; - - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_TDATE, &buf, sizeof(buf), &reslen, &offsetPtr); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_TDATE, &buf, sizeof(buf), &reslen); Timestamp ts = common::MakeTimestampGmt(2004, 8, 14, 6, 34, 51, 573948623); @@ -470,10 +453,7 @@ BOOST_AUTO_TEST_CASE(TestPutTimestampToTimestamp) SQL_TIMESTAMP_STRUCT buf = { 0 }; SqlLen reslen = sizeof(buf); - int offset = 0; - int* offsetPtr = &offset; - - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_TTIMESTAMP, &buf, sizeof(buf), &reslen, &offsetPtr); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_TTIMESTAMP, &buf, sizeof(buf), &reslen); Timestamp ts = common::MakeTimestampGmt(2004, 8, 14, 6, 34, 51, 573948623); @@ -494,10 +474,7 @@ BOOST_AUTO_TEST_CASE(TestPutDateToTimestamp) SqlLen reslen = sizeof(buf); - int offset = 0; - int* offsetPtr = &offset; - - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_TTIMESTAMP, &buf, sizeof(buf), &reslen, &offsetPtr); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_TTIMESTAMP, &buf, sizeof(buf), &reslen); Date date = common::MakeDateGmt(1984, 5, 27); @@ -516,9 +493,8 @@ BOOST_AUTO_TEST_CASE(TestGetStringFromLong) { long numBuf = 42; SqlLen reslen = sizeof(numBuf); - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_SIGNED_LONG, &numBuf, reslen, &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_SIGNED_LONG, &numBuf, reslen, &reslen); std::string res = appBuf.GetString(32); @@ -535,9 +511,8 @@ BOOST_AUTO_TEST_CASE(TestGetStringFromDouble) { double numBuf = 43.36; SqlLen reslen = sizeof(numBuf); - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_DOUBLE, &numBuf, reslen, &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_DOUBLE, &numBuf, reslen, &reslen); std::string res = appBuf.GetString(32); @@ -554,9 +529,8 @@ BOOST_AUTO_TEST_CASE(TestGetStringFromString) { char buf[] = "Some data 32d2d5hs"; SqlLen reslen = sizeof(buf); - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &buf, reslen, &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &buf, reslen, &reslen); std::string res = appBuf.GetString(reslen); @@ -567,9 +541,8 @@ BOOST_AUTO_TEST_CASE(TestGetFloatFromUshort) { unsigned short numBuf = 7162; SqlLen reslen = sizeof(numBuf); - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_UNSIGNED_SHORT, &numBuf, reslen, &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_UNSIGNED_SHORT, &numBuf, reslen, &reslen); float resFloat = appBuf.GetFloat(); @@ -584,9 +557,8 @@ BOOST_AUTO_TEST_CASE(TestGetFloatFromString) { char buf[] = "28.562"; SqlLen reslen = sizeof(buf); - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &buf, reslen, &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &buf, reslen, &reslen); float resFloat = appBuf.GetFloat(); @@ -601,9 +573,8 @@ BOOST_AUTO_TEST_CASE(TestGetFloatFromFloat) { float buf = 207.49f; SqlLen reslen = sizeof(buf); - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_FLOAT, &buf, reslen, &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_FLOAT, &buf, reslen, &reslen); float resFloat = appBuf.GetFloat(); @@ -618,9 +589,8 @@ BOOST_AUTO_TEST_CASE(TestGetFloatFromDouble) { double buf = 893.162; SqlLen reslen = sizeof(buf); - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_DOUBLE, &buf, reslen, &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_DOUBLE, &buf, reslen, &reslen); float resFloat = appBuf.GetFloat(); @@ -635,9 +605,8 @@ BOOST_AUTO_TEST_CASE(TestGetIntFromString) { char buf[] = "39"; SqlLen reslen = sizeof(buf); - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &buf, reslen, &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &buf, reslen, &reslen); int64_t resInt64 = appBuf.GetInt64(); @@ -660,9 +629,8 @@ BOOST_AUTO_TEST_CASE(TestGetIntFromFloat) { float buf = -107.49f; SqlLen reslen = sizeof(buf); - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_FLOAT, &buf, reslen, &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_FLOAT, &buf, reslen, &reslen); int64_t resInt64 = appBuf.GetInt64(); @@ -685,9 +653,8 @@ BOOST_AUTO_TEST_CASE(TestGetIntFromDouble) { double buf = 42.97f; SqlLen reslen = sizeof(buf); - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_DOUBLE, &buf, reslen, &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_DOUBLE, &buf, reslen, &reslen); int64_t resInt64 = appBuf.GetInt64(); @@ -710,9 +677,8 @@ BOOST_AUTO_TEST_CASE(TestGetIntFromBigint) { uint64_t buf = 19; SqlLen reslen = sizeof(buf); - int* offset = 0; - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_UNSIGNED_BIGINT, &buf, reslen, &reslen, &offset); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_UNSIGNED_BIGINT, &buf, reslen, &reslen); int64_t resInt64 = appBuf.GetInt64(); @@ -744,22 +710,19 @@ BOOST_AUTO_TEST_CASE(TestGetIntWithOffset) { 42, sizeof(uint64_t) } }; - int offset = 0; - int* offsetPtr = &offset; - - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_UNSIGNED_BIGINT, &buf[0].val, sizeof(buf[0].val), &buf[0].reslen, &offsetPtr); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_UNSIGNED_BIGINT, &buf[0].val, sizeof(buf[0].val), &buf[0].reslen); int64_t val = appBuf.GetInt64(); BOOST_CHECK(val == 12); - offset += sizeof(TestStruct); + appBuf.SetByteOffset(sizeof(TestStruct)); val = appBuf.GetInt64(); BOOST_CHECK(val == 42); - offsetPtr = 0; + appBuf.SetByteOffset(0); val = appBuf.GetInt64(); @@ -779,10 +742,7 @@ BOOST_AUTO_TEST_CASE(TestSetStringWithOffset) { "", 0 } }; - int offset = 0; - int* offsetPtr = &offset; - - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &buf[0].val, sizeof(buf[0].val), &buf[0].reslen, &offsetPtr); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &buf[0].val, sizeof(buf[0].val), &buf[0].reslen); appBuf.PutString("Hello Ignite!"); @@ -792,7 +752,7 @@ BOOST_AUTO_TEST_CASE(TestSetStringWithOffset) BOOST_CHECK(res == "Hello Ignite!"); BOOST_CHECK(res.size() == strlen("Hello Ignite!")); - offset += sizeof(TestStruct); + appBuf.SetByteOffset(sizeof(TestStruct)); appBuf.PutString("Hello with offset!"); @@ -814,10 +774,7 @@ BOOST_AUTO_TEST_CASE(TestGetDateFromString) char buf[] = "1999-02-22"; SqlLen reslen = sizeof(buf); - int offset = 0; - int* offsetPtr = &offset; - - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &buf[0], sizeof(buf), &reslen, &offsetPtr); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &buf[0], sizeof(buf), &reslen); Date date = appBuf.GetDate(); @@ -840,10 +797,7 @@ BOOST_AUTO_TEST_CASE(TestGetTimestampFromString) char buf[] = "2018-11-01 17:45:59"; SqlLen reslen = sizeof(buf); - int offset = 0; - int* offsetPtr = &offset; - - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &buf[0], sizeof(buf), &reslen, &offsetPtr); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &buf[0], sizeof(buf), &reslen); Timestamp date = appBuf.GetTimestamp(); @@ -871,10 +825,7 @@ BOOST_AUTO_TEST_CASE(TestGetDateFromDate) SqlLen reslen = sizeof(buf); - int offset = 0; - int* offsetPtr = &offset; - - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_TDATE, &buf, sizeof(buf), &reslen, &offsetPtr); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_TDATE, &buf, sizeof(buf), &reslen); Date date = appBuf.GetDate(); @@ -902,10 +853,7 @@ BOOST_AUTO_TEST_CASE(TestGetTimestampFromDate) SqlLen reslen = sizeof(buf); - int offset = 0; - int* offsetPtr = &offset; - - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_TDATE, &buf, sizeof(buf), &reslen, &offsetPtr); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_TDATE, &buf, sizeof(buf), &reslen); Timestamp ts = appBuf.GetTimestamp(); @@ -937,10 +885,7 @@ BOOST_AUTO_TEST_CASE(TestGetTimestampFromTimestamp) SqlLen reslen = sizeof(buf); - int offset = 0; - int* offsetPtr = &offset; - - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_TTIMESTAMP, &buf, sizeof(buf), &reslen, &offsetPtr); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_TTIMESTAMP, &buf, sizeof(buf), &reslen); Timestamp ts = appBuf.GetTimestamp(); @@ -973,10 +918,7 @@ BOOST_AUTO_TEST_CASE(TestGetDateFromTimestamp) SqlLen reslen = sizeof(buf); - int offset = 0; - int* offsetPtr = &offset; - - ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_TTIMESTAMP, &buf, sizeof(buf), &reslen, &offsetPtr); + ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_TTIMESTAMP, &buf, sizeof(buf), &reslen); Date date = appBuf.GetDate(); diff --git a/modules/platforms/cpp/odbc-test/src/column_test.cpp b/modules/platforms/cpp/odbc-test/src/column_test.cpp index 66d0214d2cf60..a87ec094d84c2 100644 --- a/modules/platforms/cpp/odbc-test/src/column_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/column_test.cpp @@ -66,9 +66,8 @@ BOOST_AUTO_TEST_CASE(TestColumnShort) short shortBuf = 0; SqlLen reslen = 0; - int* offset = 0; - ApplicationDataBuffer appBuf(type_traits::IGNITE_ODBC_C_TYPE_SIGNED_SHORT, &shortBuf, sizeof(shortBuf), &reslen, &offset); + ApplicationDataBuffer appBuf(type_traits::IGNITE_ODBC_C_TYPE_SIGNED_SHORT, &shortBuf, sizeof(shortBuf), &reslen); BOOST_REQUIRE(column.ReadToBuffer(reader, appBuf) == SQL_RESULT_SUCCESS); @@ -114,9 +113,8 @@ BOOST_AUTO_TEST_CASE(TestColumnString) char strBuf[1024] = {}; SqlLen reslen = 0; - int* offset = 0; - ApplicationDataBuffer appBuf(type_traits::IGNITE_ODBC_C_TYPE_CHAR, &strBuf, sizeof(strBuf), &reslen, &offset); + ApplicationDataBuffer appBuf(type_traits::IGNITE_ODBC_C_TYPE_CHAR, &strBuf, sizeof(strBuf), &reslen); BOOST_REQUIRE(column.ReadToBuffer(reader, appBuf) == SQL_RESULT_SUCCESS); @@ -164,9 +162,8 @@ BOOST_AUTO_TEST_CASE(TestColumnStringSeveral) std::string strBuf(data.size() / 3 + 2, 0); SqlLen reslen = 0; - int* offset = 0; - ApplicationDataBuffer appBuf(type_traits::IGNITE_ODBC_C_TYPE_CHAR, &strBuf[0], strBuf.size(), &reslen, &offset); + ApplicationDataBuffer appBuf(type_traits::IGNITE_ODBC_C_TYPE_CHAR, &strBuf[0], strBuf.size(), &reslen); BOOST_REQUIRE(column.ReadToBuffer(reader, appBuf) == SQL_RESULT_SUCCESS); @@ -246,9 +243,8 @@ BOOST_AUTO_TEST_CASE(TestColumnMultiString) char strBuf[1024] = {}; SqlLen reslen = 0; - int* offset = 0; - ApplicationDataBuffer appBuf(type_traits::IGNITE_ODBC_C_TYPE_CHAR, &strBuf, sizeof(strBuf), &reslen, &offset); + ApplicationDataBuffer appBuf(type_traits::IGNITE_ODBC_C_TYPE_CHAR, &strBuf, sizeof(strBuf), &reslen); BOOST_REQUIRE(column1.ReadToBuffer(reader, appBuf) == SQL_RESULT_SUCCESS); @@ -312,9 +308,8 @@ BOOST_AUTO_TEST_CASE(TestColumnByteArray) std::vector buf(data.size()); SqlLen reslen = 0; - int* offset = 0; - ApplicationDataBuffer appBuf(type_traits::IGNITE_ODBC_C_TYPE_BINARY, &buf[0], buf.size(), &reslen, &offset); + ApplicationDataBuffer appBuf(type_traits::IGNITE_ODBC_C_TYPE_BINARY, &buf[0], buf.size(), &reslen); BOOST_REQUIRE(column.ReadToBuffer(reader, appBuf) == SQL_RESULT_SUCCESS); @@ -362,9 +357,8 @@ BOOST_AUTO_TEST_CASE(TestColumnByteArrayHalfBuffer) std::vector buf(data.size()/2); SqlLen reslen = 0; - int* offset = 0; - ApplicationDataBuffer appBuf(type_traits::IGNITE_ODBC_C_TYPE_BINARY, &buf[0], buf.size(), &reslen, &offset); + ApplicationDataBuffer appBuf(type_traits::IGNITE_ODBC_C_TYPE_BINARY, &buf[0], buf.size(), &reslen); BOOST_REQUIRE(column.ReadToBuffer(reader, appBuf) == SQL_RESULT_SUCCESS); @@ -435,9 +429,8 @@ BOOST_AUTO_TEST_CASE(TestColumnByteArrayTwoColumns) int8_t buf[1024] = {}; SqlLen reslen = 0; - int* offset = 0; - ApplicationDataBuffer appBuf(type_traits::IGNITE_ODBC_C_TYPE_BINARY, &buf, sizeof(buf), &reslen, &offset); + ApplicationDataBuffer appBuf(type_traits::IGNITE_ODBC_C_TYPE_BINARY, &buf, sizeof(buf), &reslen); BOOST_REQUIRE(column1.ReadToBuffer(reader, appBuf) == SQL_RESULT_SUCCESS); diff --git a/modules/platforms/cpp/odbc-test/src/queries_test.cpp b/modules/platforms/cpp/odbc-test/src/queries_test.cpp index 422648e627246..8e528b4c0b910 100644 --- a/modules/platforms/cpp/odbc-test/src/queries_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/queries_test.cpp @@ -33,6 +33,7 @@ #include #include "ignite/ignite.h" +#include "ignite/common/fixed_size_array.h" #include "ignite/ignition.h" #include "ignite/impl/binary/binary_utils.h" @@ -276,10 +277,10 @@ struct QueriesTestSuiteFixture } /** - * Insert requested number of TestType vlaues with all defaults except + * Insert requested number of TestType values with all defaults except * for the strFields, which are generated using getTestString(). * - * @param num Number of records to insert. + * @param recordsNum Number of records to insert. * @param merge Set to true to use merge instead. */ void InsertTestStrings(int recordsNum, bool merge = false) @@ -337,6 +338,317 @@ struct QueriesTestSuiteFixture BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); } + /** + * Insert requested number of TestType values in a batch. + * + * @param from Index to start from. + * @param to Index to stop. + * @param expectedToAffect Expected number of affected records. + * @param merge Set to true to use merge instead of insert. + * @return Records inserted. + */ + int InsertTestBatch(int from, int to, int expectedToAffect, bool merge = false) + { + SQLCHAR insertReq[] = "INSERT " + "INTO TestType(_key, i8Field, i16Field, i32Field, strField, floatField, doubleField, boolField, dateField, " + "timestampField) VALUES(?, ?, ?, ?, ?, ?, ?, ?, ?, ?)"; + + SQLCHAR mergeReq[] = "MERGE " + "INTO TestType(_key, i8Field, i16Field, i32Field, strField, floatField, doubleField, boolField, dateField, " + "timestampField) VALUES(?, ?, ?, ?, ?, ?, ?, ?, ?, ?)"; + + SQLRETURN ret; + + int recordsNum = to - from; + + ret = SQLPrepare(stmt, merge ? mergeReq : insertReq, SQL_NTS); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + FixedSizeArray keys(recordsNum); + FixedSizeArray i8Fields(recordsNum); + FixedSizeArray i16Fields(recordsNum); + FixedSizeArray i32Fields(recordsNum); + FixedSizeArray strFields(recordsNum * 1024); + FixedSizeArray floatFields(recordsNum); + FixedSizeArray doubleFields(recordsNum); + FixedSizeArray boolFields(recordsNum); + FixedSizeArray dateFields(recordsNum); + FixedSizeArray timeFields(recordsNum); + FixedSizeArray timestampFields(recordsNum); + + FixedSizeArray strFieldsLen(recordsNum); + + BOOST_CHECKPOINT("Filling param data"); + + for (int i = 0; i < recordsNum; ++i) + { + int seed = from + i; + + keys[i] = seed; + i8Fields[i] = seed * 8; + i16Fields[i] = seed * 16; + i32Fields[i] = seed * 32; + + std::string val = getTestString(seed); + strncpy(strFields.GetData() + 1024 * i, val.c_str(), 1023); + strFieldsLen[i] = val.size(); + + floatFields[i] = seed * 0.5f; + doubleFields[i] = seed * 0.25f; + boolFields[i] = seed % 2 == 0; + + dateFields[i].year = 2017 + seed / 365; + dateFields[i].month = ((seed / 28) % 12) + 1; + dateFields[i].day = (seed % 28) + 1; + + timeFields[i].hour = (seed / 3600) % 24; + timeFields[i].minute = (seed / 60) % 60; + timeFields[i].second = seed % 60; + + timestampFields[i].year = dateFields[i].year; + timestampFields[i].month = dateFields[i].month; + timestampFields[i].day = dateFields[i].day; + timestampFields[i].hour = timeFields[i].hour; + timestampFields[i].minute = timeFields[i].minute; + timestampFields[i].second = timeFields[i].second; + timestampFields[i].fraction = std::abs(seed * 914873) % 1000000000; + } + + SQLULEN setsProcessed = 0; + + BOOST_CHECKPOINT("Setting processed pointer"); + ret = SQLSetStmtAttr(stmt, SQL_ATTR_PARAMS_PROCESSED_PTR, &setsProcessed, SQL_IS_POINTER); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECKPOINT("Binding keys"); + ret = SQLBindParameter(stmt, 1, SQL_PARAM_INPUT, SQL_C_SBIGINT, SQL_BIGINT, 0, 0, keys.GetData(), 0, 0); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECKPOINT("Binding i8Fields"); + ret = SQLBindParameter(stmt, 2, SQL_PARAM_INPUT, SQL_C_STINYINT, SQL_TINYINT, 0, 0, i8Fields.GetData(), 0, 0); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECKPOINT("Binding i16Fields"); + ret = SQLBindParameter(stmt, 3, SQL_PARAM_INPUT, SQL_C_SSHORT, SQL_SMALLINT, 0, 0, i16Fields.GetData(), 0, 0); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECKPOINT("Binding i32Fields"); + ret = SQLBindParameter(stmt, 4, SQL_PARAM_INPUT, SQL_C_SLONG, SQL_INTEGER, 0, 0, i32Fields.GetData(), 0, 0); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECKPOINT("Binding strFields"); + ret = SQLBindParameter(stmt, 5, SQL_PARAM_INPUT, SQL_C_CHAR, SQL_VARCHAR, 1024, 0, strFields.GetData(), 1024, strFieldsLen.GetData()); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECKPOINT("Binding floatFields"); + ret = SQLBindParameter(stmt, 6, SQL_PARAM_INPUT, SQL_C_FLOAT, SQL_FLOAT, 0, 0, floatFields.GetData(), 0, 0); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECKPOINT("Binding doubleFields"); + ret = SQLBindParameter(stmt, 7, SQL_PARAM_INPUT, SQL_C_DOUBLE, SQL_DOUBLE, 0, 0, doubleFields.GetData(), 0, 0); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECKPOINT("Binding boolFields"); + ret = SQLBindParameter(stmt, 8, SQL_PARAM_INPUT, SQL_C_BIT, SQL_BIT, 0, 0, boolFields.GetData(), 0, 0); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECKPOINT("Binding dateFields"); + ret = SQLBindParameter(stmt, 9, SQL_PARAM_INPUT, SQL_C_DATE, SQL_DATE, 0, 0, dateFields.GetData(), 0, 0); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECKPOINT("Binding timestampFields"); + ret = SQLBindParameter(stmt, 10, SQL_PARAM_INPUT, SQL_C_TIMESTAMP, SQL_TIMESTAMP, 0, 0, timestampFields.GetData(), 0, 0); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECKPOINT("Setting paramset size"); + ret = SQLSetStmtAttr(stmt, SQL_ATTR_PARAMSET_SIZE, reinterpret_cast(recordsNum), 0); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECKPOINT("Executing query"); + ret = SQLExecute(stmt); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + SQLLEN affected = 0; + ret = SQLRowCount(stmt, &affected); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECK_EQUAL(affected, expectedToAffect); + + BOOST_CHECKPOINT("Getting next result set"); + ret = SQLMoreResults(stmt); + + if (ret != SQL_NO_DATA) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECKPOINT("Resetting parameters."); + ret = SQLFreeStmt(stmt, SQL_RESET_PARAMS); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECKPOINT("Setting paramset size"); + ret = SQLSetStmtAttr(stmt, SQL_ATTR_PARAMSET_SIZE, reinterpret_cast(1), 0); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + return static_cast(setsProcessed); + } + + void InsertBatchSelect(int recordsNum) + { + Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache"); + + // Inserting values. + int inserted = InsertTestBatch(0, recordsNum, recordsNum); + + BOOST_REQUIRE_EQUAL(inserted, recordsNum); + + int64_t key = 0; + char strField[1024] = { 0 }; + SQLLEN strFieldLen = 0; + + // Binding columns. + SQLRETURN ret = SQLBindCol(stmt, 1, SQL_C_SLONG, &key, 0, 0); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + // Binding columns. + ret = SQLBindCol(stmt, 2, SQL_C_CHAR, &strField, sizeof(strField), &strFieldLen); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + // Just selecting everything to make sure everything is OK + SQLCHAR selectReq[] = "SELECT _key, strField FROM TestType ORDER BY _key"; + + ret = SQLExecDirect(stmt, selectReq, sizeof(selectReq)); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + int selectedRecordsNum = 0; + + ret = SQL_SUCCESS; + + while (ret == SQL_SUCCESS) + { + ret = SQLFetch(stmt); + + if (ret == SQL_NO_DATA) + break; + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + std::string expectedStr = getTestString(selectedRecordsNum); + int64_t expectedKey = selectedRecordsNum; + + BOOST_CHECK_EQUAL(key, expectedKey); + + BOOST_CHECK_EQUAL(std::string(strField, strFieldLen), expectedStr); + + ++selectedRecordsNum; + } + + BOOST_CHECK_EQUAL(recordsNum, selectedRecordsNum); + } + + void InsertNonFullBatchSelect(int recordsNum, int splitAt) + { + Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache"); + + // Inserting values. + int inserted = InsertTestBatch(splitAt, recordsNum, recordsNum - splitAt); + + BOOST_REQUIRE_EQUAL(inserted, recordsNum - splitAt); + + inserted = InsertTestBatch(0, recordsNum, splitAt); + + BOOST_REQUIRE_EQUAL(inserted, splitAt); + + int64_t key = 0; + char strField[1024] = { 0 }; + SQLLEN strFieldLen = 0; + + // Binding columns. + SQLRETURN ret = SQLBindCol(stmt, 1, SQL_C_SLONG, &key, 0, 0); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + // Binding columns. + ret = SQLBindCol(stmt, 2, SQL_C_CHAR, &strField, sizeof(strField), &strFieldLen); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + // Just selecting everything to make sure everything is OK + SQLCHAR selectReq[] = "SELECT _key, strField FROM TestType ORDER BY _key"; + + ret = SQLExecDirect(stmt, selectReq, sizeof(selectReq)); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + int selectedRecordsNum = 0; + + ret = SQL_SUCCESS; + + while (ret == SQL_SUCCESS) + { + ret = SQLFetch(stmt); + + if (ret == SQL_NO_DATA) + break; + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + std::string expectedStr = getTestString(selectedRecordsNum); + int64_t expectedKey = selectedRecordsNum; + + BOOST_CHECK_EQUAL(key, expectedKey); + + BOOST_CHECK_EQUAL(std::string(strField, strFieldLen), expectedStr); + + ++selectedRecordsNum; + } + + BOOST_CHECK_EQUAL(recordsNum, selectedRecordsNum); + } + /** Node started during the test. */ Ignite grid; @@ -1313,6 +1625,76 @@ BOOST_AUTO_TEST_CASE(TestInsertMergeSelect) BOOST_CHECK_EQUAL(recordsNum, selectedRecordsNum); } +BOOST_AUTO_TEST_CASE(TestInsertBatchSelect2) +{ + InsertBatchSelect(2); +} + +BOOST_AUTO_TEST_CASE(TestInsertBatchSelect100) +{ + InsertBatchSelect(100); +} + +BOOST_AUTO_TEST_CASE(TestInsertBatchSelect1000) +{ + InsertBatchSelect(1000); +} + +BOOST_AUTO_TEST_CASE(TestInsertBatchSelect1023) +{ + InsertBatchSelect(1024); +} + +BOOST_AUTO_TEST_CASE(TestInsertBatchSelect1024) +{ + InsertBatchSelect(1024); +} + +BOOST_AUTO_TEST_CASE(TestInsertBatchSelect1025) +{ + InsertBatchSelect(1025); +} + +BOOST_AUTO_TEST_CASE(TestInsertBatchSelect2000) +{ + InsertBatchSelect(2000); +} + +BOOST_AUTO_TEST_CASE(TestInsertBatchSelect2047) +{ + InsertBatchSelect(2048); +} + +BOOST_AUTO_TEST_CASE(TestInsertBatchSelect2048) +{ + InsertBatchSelect(2048); +} + +BOOST_AUTO_TEST_CASE(TestInsertBatchSelect2049) +{ + InsertBatchSelect(2049); +} + +BOOST_AUTO_TEST_CASE(TestNotFullInsertBatchSelect900) +{ + InsertNonFullBatchSelect(900, 42); +} + +BOOST_AUTO_TEST_CASE(TestNotFullInsertBatchSelect1500) +{ + InsertNonFullBatchSelect(1500, 100); +} + +BOOST_AUTO_TEST_CASE(TestNotFullInsertBatchSelect4500) +{ + InsertNonFullBatchSelect(4500, 1500); +} + +BOOST_AUTO_TEST_CASE(TestNotFullInsertBatchSelect4096) +{ + InsertNonFullBatchSelect(4096, 1024); +} + template void CheckMeta(char columns[n][k], SQLLEN columnsLen[n]) { diff --git a/modules/platforms/cpp/odbc-test/src/row_test.cpp b/modules/platforms/cpp/odbc-test/src/row_test.cpp index f38e9c5b58f1d..1ae34c12feadf 100644 --- a/modules/platforms/cpp/odbc-test/src/row_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/row_test.cpp @@ -82,12 +82,11 @@ void CheckRowData(Row& row, size_t rowIdx) char strBuf[1024]; SQLGUID guidBuf; char bitBuf; - int* offset = 0; - ApplicationDataBuffer appLongBuf(type_traits::IGNITE_ODBC_C_TYPE_SIGNED_LONG, &longBuf, sizeof(longBuf), &reslen, &offset); - ApplicationDataBuffer appStrBuf(type_traits::IGNITE_ODBC_C_TYPE_CHAR, &strBuf, sizeof(strBuf), &reslen, &offset); - ApplicationDataBuffer appGuidBuf(type_traits::IGNITE_ODBC_C_TYPE_GUID, &guidBuf, sizeof(guidBuf), &reslen, &offset); - ApplicationDataBuffer appBitBuf(type_traits::IGNITE_ODBC_C_TYPE_BIT, &bitBuf, sizeof(bitBuf), &reslen, &offset); + ApplicationDataBuffer appLongBuf(type_traits::IGNITE_ODBC_C_TYPE_SIGNED_LONG, &longBuf, sizeof(longBuf), &reslen); + ApplicationDataBuffer appStrBuf(type_traits::IGNITE_ODBC_C_TYPE_CHAR, &strBuf, sizeof(strBuf), &reslen); + ApplicationDataBuffer appGuidBuf(type_traits::IGNITE_ODBC_C_TYPE_GUID, &guidBuf, sizeof(guidBuf), &reslen); + ApplicationDataBuffer appBitBuf(type_traits::IGNITE_ODBC_C_TYPE_BIT, &bitBuf, sizeof(bitBuf), &reslen); // Checking size. BOOST_REQUIRE(row.GetSize() == 4); diff --git a/modules/platforms/cpp/odbc/Makefile.am b/modules/platforms/cpp/odbc/Makefile.am index de6f75acf7bd6..8d7693c7a5d1a 100644 --- a/modules/platforms/cpp/odbc/Makefile.am +++ b/modules/platforms/cpp/odbc/Makefile.am @@ -52,6 +52,7 @@ libignite_odbc_la_SOURCES = \ os/linux/src/system/socket_client.cpp \ src/app/application_data_buffer.cpp \ src/app/parameter.cpp \ + src/app/parameter_set.cpp \ src/common_types.cpp \ src/config/configuration.cpp \ src/config/connection_info.cpp \ @@ -68,6 +69,7 @@ libignite_odbc_la_SOURCES = \ src/dsn_config.cpp \ src/query/column_metadata_query.cpp \ src/query/data_query.cpp \ + src/query/batch_query.cpp \ src/query/foreign_keys_query.cpp \ src/query/primary_keys_query.cpp \ src/query/table_metadata_query.cpp \ @@ -76,6 +78,7 @@ libignite_odbc_la_SOURCES = \ src/protocol_version.cpp \ src/result_page.cpp \ src/row.cpp \ + src/message.cpp \ src/column.cpp \ src/statement.cpp \ src/type_traits.cpp \ diff --git a/modules/platforms/cpp/odbc/include/Makefile.am b/modules/platforms/cpp/odbc/include/Makefile.am index 36d8e2535ed09..92c85c56715e1 100644 --- a/modules/platforms/cpp/odbc/include/Makefile.am +++ b/modules/platforms/cpp/odbc/include/Makefile.am @@ -22,6 +22,7 @@ noinst_HEADERS = \ ignite/odbc/query/table_metadata_query.h \ ignite/odbc/query/special_columns_query.h \ ignite/odbc/query/type_info_query.h \ + ignite/odbc/query/batch_query.h \ ignite/odbc/query/data_query.h \ ignite/odbc/query/foreign_keys_query.h \ ignite/odbc/query/column_metadata_query.h \ @@ -35,6 +36,7 @@ noinst_HEADERS = \ ignite/odbc/parser.h \ ignite/odbc/app/application_data_buffer.h \ ignite/odbc/app/parameter.h \ + ignite/odbc/app/parameter_set.h \ ignite/odbc/row.h \ ignite/odbc/utility.h \ ignite/odbc/environment.h \ diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/app/application_data_buffer.h b/modules/platforms/cpp/odbc/include/ignite/odbc/app/application_data_buffer.h index 18ac36aead536..b7989c55982fa 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/app/application_data_buffer.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/app/application_data_buffer.h @@ -54,10 +54,9 @@ namespace ignite * @param buffer Data buffer pointer. * @param buflen Data buffer length. * @param reslen Resulting data length. - * @param offset Pointer to buffer and reslen offset pointer. */ ApplicationDataBuffer(type_traits::IgniteSqlType type, void* buffer, - SqlLen buflen, SqlLen* reslen, int** offset = 0); + SqlLen buflen, SqlLen* reslen); /** * Copy constructor. @@ -80,13 +79,23 @@ namespace ignite ApplicationDataBuffer& operator=(const ApplicationDataBuffer& other); /** - * Set pointer to offset pointer. + * Set offset in bytes for all bound pointers. * - * @param offset Pointer to offset pointer. + * @param offset Offset. */ - void SetPtrToOffsetPtr(int** offset) + void SetByteOffset(int offset) { - this->offset = offset; + this->byteOffset = offset; + } + + /** + * Set offset in elements for all bound pointers. + * + * @param + */ + void SetElementOffset(SqlUlen idx) + { + this->elementOffset = idx; } /** @@ -313,6 +322,13 @@ namespace ignite */ SqlLen GetDataAtExecSize() const; + /** + * Get single element size. + * + * @return Size of the single element. + */ + SqlLen GetElementSize() const; + /** * Get size of the input buffer. * @@ -392,10 +408,11 @@ namespace ignite * Apply buffer offset to pointer. * Adds offset to pointer if offset pointer is not null. * @param ptr Pointer. + * @param elemSize Element size. * @return Pointer with applied offset. */ template - T* ApplyOffset(T* ptr) const; + T* ApplyOffset(T* ptr, size_t elemSize) const; /** Underlying data type. */ type_traits::IgniteSqlType type; @@ -409,8 +426,11 @@ namespace ignite /** Result length. */ SqlLen* reslen; - /** Pointer to implementation pointer to application offset */ - int** offset; + /** Current byte offset */ + int byteOffset; + + /** Current element offset. */ + SqlUlen elementOffset; }; /** Column binging map type alias. */ diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/app/parameter.h b/modules/platforms/cpp/odbc/include/ignite/odbc/app/parameter.h index 1cf85b514aa59..cdaaead40ee3d 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/app/parameter.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/app/parameter.h @@ -22,12 +22,10 @@ #include -#include #include #include #include "ignite/odbc/app/application_data_buffer.h" -#include "ignite/odbc/type_traits.h" namespace ignite { @@ -70,7 +68,7 @@ namespace ignite ~Parameter(); /** - * Copy assigment operator. + * Assignment operator. * * @param other Other instance. * @return This. @@ -78,10 +76,12 @@ namespace ignite Parameter& operator=(const Parameter& other); /** - * Write request using provided writer. + * Write parameter using provided writer. * @param writer Writer. + * @param offset Offset for the buffer. + * @param idx Index for the array-of-parameters case. */ - void Write(impl::binary::BinaryWriterImpl& writer) const; + void Write(impl::binary::BinaryWriterImpl& writer, int offset = 0, SqlUlen idx = 0) const; /** * Get data buffer. @@ -90,6 +90,13 @@ namespace ignite */ ApplicationDataBuffer& GetBuffer(); + /** + * Get data buffer. + * + * @return underlying ApplicationDataBuffer instance. + */ + const ApplicationDataBuffer& GetBuffer() const; + /** * Reset stored at-execution data. */ @@ -128,9 +135,6 @@ namespace ignite /** Stored at-execution data. */ std::vector storedData; }; - - /** Parameter binging map type alias. */ - typedef std::map ParameterBindingMap; } } } diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/app/parameter_set.h b/modules/platforms/cpp/odbc/include/ignite/odbc/app/parameter_set.h new file mode 100644 index 0000000000000..2ab55808c073c --- /dev/null +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/app/parameter_set.h @@ -0,0 +1,268 @@ +/* + * 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. + */ + +#ifndef _IGNITE_ODBC_APP_PARAMETER_SET +#define _IGNITE_ODBC_APP_PARAMETER_SET + +#include + +#include + +#include +#include + +#include "ignite/odbc/app/parameter.h" + +namespace ignite +{ + namespace odbc + { + namespace app + { + /** + * Parameter set. + */ + class ParameterSet + { + /** Parameter binging map type alias. */ + typedef std::map ParameterBindingMap; + + /** Parameter meta vector. */ + typedef std::vector ParameterTypeVector; + public: + /** + * Default constructor. + */ + ParameterSet(); + + /** + * Destructor. + */ + ~ParameterSet() + { + // No-op. + } + + /** + * Set parameters set size. + * + * @param size Size of the parameter set. + */ + void SetParamSetSize(SqlUlen size); + + /** + * Bind parameter. + * + * @param paramIdx Parameter index. + * @param param Parameter. + */ + void BindParameter(uint16_t paramIdx, const Parameter& param); + + /** + * Unbind specified parameter. + * + * @param paramIdx Parameter index. + */ + void UnbindParameter(uint16_t paramIdx); + + /** + * Unbind all parameters. + */ + void UnbindAll(); + + /** + * Get number of binded parameters. + * + * @return Number of binded parameters. + */ + uint16_t GetParametersNumber() const; + + /** + * Set parameter binding offset pointer. + * + * @param ptr Parameter binding offset pointer. + */ + void SetParamBindOffsetPtr(int* ptr); + + /** + * Get parameter binding offset pointer. + * + * @return Parameter binding offset pointer. + */ + int* GetParamBindOffsetPtr(); + + /** + * Prepare parameters set for statement execution. + */ + void Prepare(); + + /** + * Check if the data at-execution is needed. + * + * @return True if the data at execution is needed. + */ + bool IsDataAtExecNeeded() const; + + /** + * Update parameter types metadata. + * + * @param meta Types metadata. + */ + void UpdateParamsTypes(const ParameterTypeVector& meta); + + /** + * Get type id of the parameter. + * + * @param idx Parameter index. + * @param dflt Default value to return if the type can not be found. + * @return Type ID of the parameter or dflt, if the type can not be returned. + */ + int8_t GetParamType(int16_t idx, int8_t dflt); + + /** + * Get expected parameters number. + * Using metadata. If metadata was not updated returns zero. + * + * @return Expected parameters number. + */ + uint16_t GetExpectedParamNum(); + + /** + * Check if the metadata was set for the parameter set. + * + * @return True if the metadata was set for the parameter set. + */ + bool IsMetadataSet() const; + + /** + * Check if the parameter selected for putting data at-execution. + * + * @return True if the parameter selected for putting data at-execution. + */ + bool IsParameterSelected() const; + + /** + * Get parameter by index. + * + * @param idx Index. + * @return Parameter or null, if parameter is not bound. + */ + Parameter* GetParameter(uint16_t idx); + + /** + * Get selected parameter. + * + * @return Parameter or null, if parameter is not bound. + */ + Parameter* GetSelectedParameter(); + + /** + * Internally selects next parameter for putting data at-execution. + * + * @return Parameter if found and null otherwise. + */ + Parameter* SelectNextParameter(); + + /** + * Write only first row of the param set using provided writer. + * @param writer Writer. + */ + void Write(impl::binary::BinaryWriterImpl& writer) const; + + /** + * Write rows of the param set in interval [begin, end) using provided writer. + * @param writer Writer. + * @param begin Beginng of the interval. + * @param end End of the interval. + * @param last Last page flag. + */ + void Write(impl::binary::BinaryWriterImpl& writer, SqlUlen begin, SqlUlen end, bool last) const; + + /** + * Calculate row length. + * + * @return Row length. + */ + int32_t CalculateRowLen() const; + + /** + * Get parameter set size. + * + * @return Number of rows in set. + */ + int32_t GetParamSetSize() const; + + /** + * Set number of parameters processed in batch. + * + * @param processed Processed. + */ + void SetParamsProcessed(SqlUlen processed) const; + + /** + * Number of processed params should be written using provided address. + * + * @param ptr Pointer. + */ + void SetParamsProcessedPtr(SqlUlen* ptr); + + /** + * Get pointer to write number of parameters processed in batch. + * + * @return Pointer to write number of parameters processed in batch. + */ + SqlUlen* GetParamsProcessedPtr(); + + private: + /** + * Write single row of the param set using provided writer. + * @param writer Writer. + * @param idx Row index. + */ + void WriteRow(impl::binary::BinaryWriterImpl& writer, SqlUlen idx) const; + + IGNITE_NO_COPY_ASSIGNMENT(ParameterSet); + + /** Parameters. */ + ParameterBindingMap parameters; + + /** Parameter types. */ + ParameterTypeVector paramTypes; + + /** Offset added to pointers to change binding of parameters. */ + int* paramBindOffset; + + /** Processed parameters. */ + SqlUlen* processedParamRows; + + /** Parameter set size. */ + SqlUlen paramSetSize; + + /** Current position in parametor set. */ + SqlUlen paramSetPos; + + /** Index of the parameter, which is currently being set. */ + uint16_t currentParamIdx; + + /** Parameter types are set. */ + bool typesSet; + }; + } + } +} + +#endif //_IGNITE_ODBC_APP_PARAMETER_SET diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h b/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h index 5d8901d6c2ef0..fc25b56822c66 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h @@ -156,9 +156,7 @@ namespace ignite */ SQL_STATE_HYT01_CONNECTIOIN_TIMEOUT, - /** - * Driver does not support this function. - */ + /** Driver does not support this function. */ SQL_STATE_IM001_FUNCTION_NOT_SUPPORTED }; diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnosable.h b/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnosable.h index 6937fcc6f3196..95e4f548c1f9c 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnosable.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnosable.h @@ -45,14 +45,14 @@ namespace ignite * * @return Diagnostic record. */ - virtual const diagnostic::DiagnosticRecordStorage& GetDiagnosticRecords() const = 0; + virtual const DiagnosticRecordStorage& GetDiagnosticRecords() const = 0; /** * Get diagnostic record. * * @return Diagnostic record. */ - virtual diagnostic::DiagnosticRecordStorage& GetDiagnosticRecords() = 0; + virtual DiagnosticRecordStorage& GetDiagnosticRecords() = 0; /** * Add new status record. diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/message.h b/modules/platforms/cpp/odbc/include/ignite/odbc/message.h index a2bbd99adce3c..cecdac67f9d36 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/message.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/message.h @@ -24,11 +24,11 @@ #include "ignite/impl/binary/binary_writer_impl.h" #include "ignite/impl/binary/binary_reader_impl.h" -#include "ignite/odbc/utility.h" #include "ignite/odbc/result_page.h" +#include "ignite/odbc/protocol_version.h" #include "ignite/odbc/meta/column_meta.h" #include "ignite/odbc/meta/table_meta.h" -#include "ignite/odbc/app/parameter.h" +#include "ignite/odbc/app/parameter_set.h" namespace ignite { @@ -48,7 +48,9 @@ namespace ignite REQUEST_TYPE_GET_TABLES_METADATA = 6, - REQUEST_TYPE_GET_PARAMS_METADATA = 7 + REQUEST_TYPE_GET_PARAMS_METADATA = 7, + + REQUEST_TYPE_EXECUTE_SQL_QUERY_BATCH = 8 }; enum ResponseStatus @@ -71,35 +73,18 @@ namespace ignite * @param distributedJoins Distributed joins flag. * @param enforceJoinOrder Enforce join order flag. */ - HandshakeRequest(int64_t version, bool distributedJoins, bool enforceJoinOrder) : - version(version), - distributedJoins(distributedJoins), - enforceJoinOrder(enforceJoinOrder) - { - // No-op. - } + HandshakeRequest(int64_t version, bool distributedJoins, bool enforceJoinOrder); /** * Destructor. */ - ~HandshakeRequest() - { - // No-op. - } + ~HandshakeRequest(); /** * Write request using provided writer. * @param writer Writer. */ - void Write(ignite::impl::binary::BinaryWriterImpl& writer) const - { - writer.WriteInt8(REQUEST_TYPE_HANDSHAKE); - - writer.WriteInt64(version); - - writer.WriteBool(distributedJoins); - writer.WriteBool(enforceJoinOrder); - } + void Write(impl::binary::BinaryWriterImpl& writer) const; private: /** Protocol version. */ @@ -123,65 +108,80 @@ namespace ignite * * @param cache Cache name. * @param sql SQL query. - * @param argsNum Number of arguments. + * @param params Number of arguments. */ - QueryExecuteRequest(const std::string& cache, const std::string& sql, - const app::ParameterBindingMap& params) : - cache(cache), - sql(sql), - params(params) - { - // No-op. - } + QueryExecuteRequest(const std::string& cache, const std::string& sql, const app::ParameterSet& params); /** * Destructor. */ - ~QueryExecuteRequest() - { - // No-op. - } + ~QueryExecuteRequest(); /** * Write request using provided writer. * @param writer Writer. */ - void Write(ignite::impl::binary::BinaryWriterImpl& writer) const - { - writer.WriteInt8(REQUEST_TYPE_EXECUTE_SQL_QUERY); - utility::WriteString(writer, cache); - utility::WriteString(writer, sql); + void Write(impl::binary::BinaryWriterImpl& writer) const; - writer.WriteInt32(static_cast(params.size())); + private: + /** Cache name. */ + std::string cache; - app::ParameterBindingMap::const_iterator i; - uint16_t prev = 0; + /** SQL query. */ + std::string sql; - for (i = params.begin(); i != params.end(); ++i) { - uint16_t current = i->first; + /** Parameters bindings. */ + const app::ParameterSet& params; + }; - while ((current - prev) > 1) { - writer.WriteNull(); - ++prev; - } + /** + * Query execute batch request. + */ + class QueryExecuteBatchtRequest + { + public: + /** + * Constructor. + * + * @param schema Schema. + * @param sql SQL query. + * @param params Query arguments. + * @param begin Beginng of the interval. + * @param end End of the interval. + */ + QueryExecuteBatchtRequest(const std::string& schema, const std::string& sql, + const app::ParameterSet& params, SqlUlen begin, SqlUlen end, bool last); - i->second.Write(writer); + /** + * Destructor. + */ + ~QueryExecuteBatchtRequest(); - prev = current; - } - } + /** + * Write request using provided writer. + * @param writer Writer. + */ + void Write(impl::binary::BinaryWriterImpl& writer) const; private: - /** Cache name. */ - std::string cache; + /** Schema name. */ + std::string schema; /** SQL query. */ std::string sql; /** Parameters bindings. */ - const app::ParameterBindingMap& params; - }; + const app::ParameterSet& params; + /** Beginng of the interval. */ + SqlUlen begin; + + /** End of the interval. */ + SqlUlen end; + + /** Last page flag. */ + bool last; + }; /** * Query close request. @@ -194,28 +194,18 @@ namespace ignite * * @param queryId Query ID. */ - QueryCloseRequest(int64_t queryId) : queryId(queryId) - { - // No-op. - } + QueryCloseRequest(int64_t queryId); /** * Destructor. */ - ~QueryCloseRequest() - { - // No-op. - } + ~QueryCloseRequest(); /** * Write request using provided writer. * @param writer Writer. */ - void Write(ignite::impl::binary::BinaryWriterImpl& writer) const - { - writer.WriteInt8(REQUEST_TYPE_CLOSE_SQL_QUERY); - writer.WriteInt64(queryId); - } + void Write(impl::binary::BinaryWriterImpl& writer) const; private: /** Query ID. */ @@ -234,31 +224,18 @@ namespace ignite * @param queryId Query ID. * @param pageSize Required page size. */ - QueryFetchRequest(int64_t queryId, int32_t pageSize) : - queryId(queryId), - pageSize(pageSize) - { - // No-op. - } + QueryFetchRequest(int64_t queryId, int32_t pageSize); /** * Destructor. */ - ~QueryFetchRequest() - { - // No-op. - } + ~QueryFetchRequest(); /** * Write request using provided writer. * @param writer Writer. */ - void Write(ignite::impl::binary::BinaryWriterImpl& writer) const - { - writer.WriteInt8(REQUEST_TYPE_FETCH_SQL_QUERY); - writer.WriteInt64(queryId); - writer.WriteInt32(pageSize); - } + void Write(impl::binary::BinaryWriterImpl& writer) const; private: /** Query ID. */ @@ -281,34 +258,18 @@ namespace ignite * @param table Table name. * @param column Column name. */ - QueryGetColumnsMetaRequest(const std::string& schema, const std::string& table, const std::string& column) : - schema(schema), - table(table), - column(column) - { - // No-op. - } + QueryGetColumnsMetaRequest(const std::string& schema, const std::string& table, const std::string& column); /** * Destructor. */ - ~QueryGetColumnsMetaRequest() - { - // No-op. - } + ~QueryGetColumnsMetaRequest(); /** * Write request using provided writer. * @param writer Writer. */ - void Write(ignite::impl::binary::BinaryWriterImpl& writer) const - { - writer.WriteInt8(REQUEST_TYPE_GET_COLUMNS_METADATA); - - utility::WriteString(writer, schema); - utility::WriteString(writer, table); - utility::WriteString(writer, column); - } + void Write(impl::binary::BinaryWriterImpl& writer) const; private: /** Schema search pattern. */ @@ -336,36 +297,18 @@ namespace ignite * @param tableTypes Table types search pattern. */ QueryGetTablesMetaRequest(const std::string& catalog, const std::string& schema, - const std::string& table, const std::string& tableTypes) : - catalog(catalog), - schema(schema), - table(table), - tableTypes(tableTypes) - { - // No-op. - } + const std::string& table, const std::string& tableTypes); /** * Destructor. */ - ~QueryGetTablesMetaRequest() - { - // No-op. - } + ~QueryGetTablesMetaRequest(); /** * Write request using provided writer. * @param writer Writer. */ - void Write(ignite::impl::binary::BinaryWriterImpl& writer) const - { - writer.WriteInt8(REQUEST_TYPE_GET_TABLES_METADATA); - - utility::WriteString(writer, catalog); - utility::WriteString(writer, schema); - utility::WriteString(writer, table); - utility::WriteString(writer, tableTypes); - } + void Write(impl::binary::BinaryWriterImpl& writer) const; private: /** Column search pattern. */ @@ -414,13 +357,7 @@ namespace ignite * Write request using provided writer. * @param writer Writer. */ - void Write(ignite::impl::binary::BinaryWriterImpl& writer) const - { - writer.WriteInt8(REQUEST_TYPE_GET_PARAMS_METADATA); - - utility::WriteString(writer, cacheName); - utility::WriteString(writer, sqlQuery); - } + void Write(impl::binary::BinaryWriterImpl& writer) const; private: /** Cache name. */ @@ -439,33 +376,19 @@ namespace ignite /** * Constructor. */ - Response() : status(RESPONSE_STATUS_FAILED), error() - { - // No-op. - } + Response(); /** * Destructor. */ - virtual ~Response() - { - // No-op. - } + virtual ~Response(); /** * Read response using provided reader. * @param reader Reader. */ - void Read(ignite::impl::binary::BinaryReaderImpl& reader) - { - status = reader.ReadInt8(); + void Read(impl::binary::BinaryReaderImpl& reader); - if (status == RESPONSE_STATUS_SUCCESS) - ReadOnSuccess(reader); - else - utility::ReadString(reader, error);; - } - /** * Get request processing status. * @return Status. @@ -488,10 +411,7 @@ namespace ignite /** * Read data if response status is RESPONSE_STATUS_SUCCESS. */ - virtual void ReadOnSuccess(ignite::impl::binary::BinaryReaderImpl&) - { - // No-op. - } + virtual void ReadOnSuccess(impl::binary::BinaryReaderImpl&); private: /** Request processing status. */ @@ -510,21 +430,12 @@ namespace ignite /** * Constructor. */ - HandshakeResponse() : - accepted(false), - protoVerSince(), - currentVer() - { - // No-op. - } + HandshakeResponse(); /** * Destructor. */ - ~HandshakeResponse() - { - // No-op. - } + ~HandshakeResponse(); /** * Check if the handshake has been accepted. @@ -558,16 +469,7 @@ namespace ignite * Read response using provided reader. * @param reader Reader. */ - virtual void ReadOnSuccess(ignite::impl::binary::BinaryReaderImpl& reader) - { - accepted = reader.ReadBool(); - - if (!accepted) - { - utility::ReadString(reader, protoVerSince); - utility::ReadString(reader, currentVer); - } - } + virtual void ReadOnSuccess(impl::binary::BinaryReaderImpl& reader); /** Handshake accepted. */ bool accepted; @@ -588,18 +490,12 @@ namespace ignite /** * Constructor. */ - QueryCloseResponse() : queryId(0) - { - // No-op. - } + QueryCloseResponse(); /** * Destructor. */ - ~QueryCloseResponse() - { - // No-op. - } + virtual ~QueryCloseResponse(); /** * Get query ID. @@ -615,10 +511,7 @@ namespace ignite * Read response using provided reader. * @param reader Reader. */ - virtual void ReadOnSuccess(ignite::impl::binary::BinaryReaderImpl& reader) - { - queryId = reader.ReadInt64(); - } + virtual void ReadOnSuccess(impl::binary::BinaryReaderImpl& reader); /** Query ID. */ int64_t queryId; @@ -633,18 +526,12 @@ namespace ignite /** * Constructor. */ - QueryExecuteResponse() : queryId(0), meta() - { - // No-op. - } + QueryExecuteResponse(); /** * Destructor. */ - ~QueryExecuteResponse() - { - // No-op. - } + virtual ~QueryExecuteResponse(); /** * Get query ID. @@ -669,12 +556,7 @@ namespace ignite * Read response using provided reader. * @param reader Reader. */ - virtual void ReadOnSuccess(ignite::impl::binary::BinaryReaderImpl& reader) - { - queryId = reader.ReadInt64(); - - meta::ReadColumnMetaVector(reader, meta); - } + virtual void ReadOnSuccess(impl::binary::BinaryReaderImpl& reader); /** Query ID. */ int64_t queryId; @@ -684,28 +566,82 @@ namespace ignite }; /** - * Query fetch response. + * Query execute batch start response. */ - class QueryFetchResponse : public Response + class QueryExecuteBatchResponse : public Response { public: /** * Constructor. - * @param resultPage Result page. */ - QueryFetchResponse(ResultPage& resultPage) : queryId(0), resultPage(resultPage) + QueryExecuteBatchResponse(); + + /** + * Destructor. + */ + virtual ~QueryExecuteBatchResponse(); + + /** + * Affected rows. + * @return Affected rows. + */ + int64_t GetAffectedRows() const { - // No-op. + return affectedRows; } /** - * Destructor. + * Get index of the set which caused an error. + * @return Index of the set which caused an error. */ - ~QueryFetchResponse() + int64_t GetErrorSetIdx() const { - // No-op. + return affectedRows; + } + + /** + * Get error message. + * @return Error message. + */ + const std::string& GetErrorMessage() const + { + return errorMessage; } + private: + /** + * Read response using provided reader. + * @param reader Reader. + */ + virtual void ReadOnSuccess(impl::binary::BinaryReaderImpl& reader); + + /** Affected rows. */ + int64_t affectedRows; + + /** Index of the set which caused an error. */ + int64_t errorSetIdx; + + /** Error message. */ + std::string errorMessage; + }; + + /** + * Query fetch response. + */ + class QueryFetchResponse : public Response + { + public: + /** + * Constructor. + * @param resultPage Result page. + */ + QueryFetchResponse(ResultPage& resultPage); + + /** + * Destructor. + */ + virtual ~QueryFetchResponse(); + /** * Get query ID. * @return Query ID. @@ -720,12 +656,7 @@ namespace ignite * Read response using provided reader. * @param reader Reader. */ - virtual void ReadOnSuccess(ignite::impl::binary::BinaryReaderImpl& reader) - { - queryId = reader.ReadInt64(); - - resultPage.Read(reader); - } + virtual void ReadOnSuccess(impl::binary::BinaryReaderImpl& reader); /** Query ID. */ int64_t queryId; @@ -743,18 +674,12 @@ namespace ignite /** * Constructor. */ - QueryGetColumnsMetaResponse() - { - // No-op. - } + QueryGetColumnsMetaResponse(); /** * Destructor. */ - ~QueryGetColumnsMetaResponse() - { - // No-op. - } + virtual ~QueryGetColumnsMetaResponse(); /** * Get column metadata. @@ -770,10 +695,7 @@ namespace ignite * Read response using provided reader. * @param reader Reader. */ - virtual void ReadOnSuccess(ignite::impl::binary::BinaryReaderImpl& reader) - { - meta::ReadColumnMetaVector(reader, meta); - } + virtual void ReadOnSuccess(impl::binary::BinaryReaderImpl& reader); /** Columns metadata. */ meta::ColumnMetaVector meta; @@ -788,18 +710,12 @@ namespace ignite /** * Constructor. */ - QueryGetTablesMetaResponse() - { - // No-op. - } + QueryGetTablesMetaResponse(); /** * Destructor. */ - ~QueryGetTablesMetaResponse() - { - // No-op. - } + virtual ~QueryGetTablesMetaResponse(); /** * Get column metadata. @@ -815,10 +731,7 @@ namespace ignite * Read response using provided reader. * @param reader Reader. */ - virtual void ReadOnSuccess(ignite::impl::binary::BinaryReaderImpl& reader) - { - meta::ReadTableMetaVector(reader, meta); - } + virtual void ReadOnSuccess(impl::binary::BinaryReaderImpl& reader); /** Columns metadata. */ meta::TableMetaVector meta; @@ -833,18 +746,12 @@ namespace ignite /** * Constructor. */ - QueryGetParamsMetaResponse() - { - // No-op. - } + QueryGetParamsMetaResponse(); /** * Destructor. */ - ~QueryGetParamsMetaResponse() - { - // No-op. - } + virtual ~QueryGetParamsMetaResponse(); /** * Get parameter type IDs. @@ -860,10 +767,7 @@ namespace ignite * Read response using provided reader. * @param reader Reader. */ - virtual void ReadOnSuccess(ignite::impl::binary::BinaryReaderImpl& reader) - { - utility::ReadByteArray(reader, typeIds); - } + virtual void ReadOnSuccess(impl::binary::BinaryReaderImpl& reader); /** Columns metadata. */ std::vector typeIds; diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/query/batch_query.h b/modules/platforms/cpp/odbc/include/ignite/odbc/query/batch_query.h new file mode 100644 index 0000000000000..a691f7351d956 --- /dev/null +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/query/batch_query.h @@ -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. + */ + +#ifndef _IGNITE_ODBC_QUERY_BATCH_QUERY +#define _IGNITE_ODBC_QUERY_BATCH_QUERY + +#include "ignite/odbc/query/query.h" +#include "ignite/odbc/app/parameter_set.h" +#include "ignite/odbc/cursor.h" + +namespace ignite +{ + namespace odbc + { + /** Connection forward-declaration. */ + class Connection; + + namespace query + { + /** + * Query. + */ + class BatchQuery : public Query + { + public: + /** + * Constructor. + * + * @param diag Diagnostics collector. + * @param connection Associated connection. + * @param sql SQL query string. + * @param params SQL params. + */ + BatchQuery(diagnostic::Diagnosable& diag, Connection& connection, + const std::string& sql, const app::ParameterSet& params); + + /** + * Destructor. + */ + virtual ~BatchQuery(); + + /** + * Execute query. + * + * @return True on success. + */ + virtual SqlResult Execute(); + + /** + * Get column metadata. + * + * @return Column metadata. + */ + virtual const meta::ColumnMetaVector& GetMeta() const; + + /** + * Fetch next result row to application buffers. + * + * @param columnBindings Application buffers to put data to. + * @return Operation result. + */ + virtual SqlResult FetchNextRow(app::ColumnBindingMap& columnBindings); + + /** + * Get data of the specified column in the result set. + * + * @param columnIdx Column index. + * @param buffer Buffer to put column data to. + * @return Operation result. + */ + virtual SqlResult GetColumn(uint16_t columnIdx, app::ApplicationDataBuffer& buffer); + + /** + * Close query. + * + * @return Result. + */ + virtual SqlResult Close(); + + /** + * Check if data is available. + * + * @return True if data is available. + */ + virtual bool DataAvailable() const; + + /** + * Get number of rows affected by the statement. + * + * @return Number of rows affected by the statement. + */ + virtual int64_t AffectedRows() const; + + /** + * Get SQL query string. + * + * @return SQL query string. + */ + const std::string& GetSql() const + { + return sql; + } + + private: + IGNITE_NO_COPY_ASSIGNMENT(BatchQuery); + + /** + * Make query execute request and use response to set internal + * state. + * + * @param begin Paramset interval beginning. + * @param end Paramset interval end. + * @param last Last page flag. + * @return Result. + */ + SqlResult MakeRequestExecuteBatch(SqlUlen begin, SqlUlen end, bool last); + + /** Connection associated with the statement. */ + Connection& connection; + + /** SQL Query. */ + std::string sql; + + /** Parameter bindings. */ + const app::ParameterSet& params; + + /** Columns metadata. */ + meta::ColumnMetaVector resultMeta; + + /** Number of rows affected. */ + int64_t rowsAffected; + + /** Number of parameter sets successfully processed. */ + int64_t setsProcessed; + + /** Query executed. */ + bool executed; + + /** Data retrieved. */ + bool dataRetrieved; + }; + } + } +} + +#endif //_IGNITE_ODBC_QUERY_BATCH_QUERY diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/query/data_query.h b/modules/platforms/cpp/odbc/include/ignite/odbc/query/data_query.h index 68bb8776417c8..0424bf8364a6d 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/query/data_query.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/query/data_query.h @@ -19,7 +19,7 @@ #define _IGNITE_ODBC_QUERY_DATA_QUERY #include "ignite/odbc/query/query.h" -#include "ignite/odbc/app/parameter.h" +#include "ignite/odbc/app/parameter_set.h" #include "ignite/odbc/cursor.h" namespace ignite @@ -46,7 +46,7 @@ namespace ignite * @param params SQL params. */ DataQuery(diagnostic::Diagnosable& diag, Connection& connection, - const std::string& sql, const app::ParameterBindingMap& params); + const std::string& sql, const app::ParameterSet& params); /** * Destructor. @@ -122,24 +122,32 @@ namespace ignite * Make query execute request and use response to set internal * state. * - * @return True on success. + * @return Result. */ SqlResult MakeRequestExecute(); /** * Make query close request. * - * @return True on success. + * @return Result. */ SqlResult MakeRequestClose(); /** * Make data fetch request and use response to set internal state. * - * @return True on success. + * @return Result. */ SqlResult MakeRequestFetch(); + /** + * Close query. + * Non-virtual implementation. + * + * @return True on success. + */ + SqlResult InternalClose(); + /** Connection associated with the statement. */ Connection& connection; @@ -147,7 +155,7 @@ namespace ignite std::string sql; /** Parameter bindings. */ - const app::ParameterBindingMap& params; + const app::ParameterSet& params; /** Columns metadata. */ meta::ColumnMetaVector resultMeta; diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/query/query.h b/modules/platforms/cpp/odbc/include/ignite/odbc/query/query.h index 40be1ed4c8bd6..d8103af182a30 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/query/query.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/query/query.h @@ -47,6 +47,9 @@ namespace ignite /** Data query type. */ DATA, + /** Batch query type. */ + BATCH, + /** Foreign keys query type. */ FOREIGN_KEYS, diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/query/type_info_query.h b/modules/platforms/cpp/odbc/include/ignite/odbc/query/type_info_query.h index d337d0311b972..ee37bb012bb33 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/query/type_info_query.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/query/type_info_query.h @@ -95,7 +95,7 @@ namespace ignite * @return Number of rows affected by the statement. */ virtual int64_t AffectedRows() const; - + private: IGNITE_NO_COPY_ASSIGNMENT(TypeInfoQuery); diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h b/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h index 596fc66e817a6..132cf78e45238 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h @@ -23,19 +23,12 @@ #include #include -#include -#include -#include - #include "ignite/odbc/meta/column_meta.h" -#include "ignite/odbc/meta/table_meta.h" #include "ignite/odbc/query/query.h" #include "ignite/odbc/app/application_data_buffer.h" -#include "ignite/odbc/app/parameter.h" +#include "ignite/odbc/app/parameter_set.h" #include "ignite/odbc/diagnostic/diagnosable_adapter.h" #include "ignite/odbc/common_types.h" -#include "ignite/odbc/cursor.h" -#include "ignite/odbc/utility.h" namespace ignite { @@ -102,7 +95,19 @@ namespace ignite * @param resLen A pointer to a buffer for the parameter's length. */ void BindParameter(uint16_t paramIdx, int16_t ioType, int16_t bufferType, int16_t paramSqlType, - SqlUlen columnSize, int16_t decDigits, void* buffer, SqlLen bufferLen, SqlLen* resLen); + SqlUlen columnSize, int16_t decDigits, void* buffer, SqlLen bufferLen, SqlLen* resLen); + + /** + * Unbind specified parameter. + * + * @param paramIdx Parameter index. + */ + void UnbindParameter(uint16_t paramIdx); + + /** + * Unbind all parameters. + */ + void UnbindAllParameters(); /** * Set statement attribute. @@ -126,9 +131,9 @@ namespace ignite /** * Get number of binded parameters. * - * @return Number of binded parameters. + * @param paramNum Number of binded parameters. */ - uint16_t GetParametersNumber(); + void GetParametersNumber(uint16_t& paramNum); /** * Set parameter binding offset pointer. @@ -137,13 +142,6 @@ namespace ignite */ void SetParamBindOffsetPtr(int* ptr); - /** - * Get parameter binding offset pointer. - * - * @return Parameter binding offset pointer. - */ - int* GetParamBindOffsetPtr(); - /** * Get value of the column in the result set. * @@ -399,7 +397,14 @@ namespace ignite * Bind parameter. * * @param paramIdx Parameter index. - * @param param Parameter. + * @param ioType Type of the parameter (input/output). + * @param bufferType The data type of the parameter. + * @param paramSqlType The SQL data type of the parameter. + * @param columnSize The size of the column or expression of the corresponding parameter marker. + * @param decDigits The decimal digits of the column or expression of the corresponding parameter marker. + * @param buffer A pointer to a buffer for the parameter's data. + * @param bufferLen Length of the ParameterValuePtr buffer in bytes. + * @param resLen A pointer to a buffer for the parameter's length. */ void SafeBindParameter(uint16_t paramIdx, const app::Parameter& param); @@ -455,6 +460,15 @@ namespace ignite */ SqlResult InternalGetAttribute(int attr, void* buf, SQLINTEGER bufLen, SQLINTEGER* valueLen); + /** + * Get number of binded parameters. + * Internal call. + * + * @param paramNum Number of binded parameters. + * @return Operation result. + */ + SqlResult InternalGetParametersNumber(uint16_t& paramNum); + /** * Get value of the column in the result set. * @@ -681,12 +695,6 @@ namespace ignite /** Column bindings. */ app::ColumnBindingMap columnBindings; - /** Parameter bindings. */ - app::ParameterBindingMap paramBindings; - - /** Parameter meta. */ - std::vector paramTypes; - /** Underlying query. */ std::auto_ptr currentQuery; @@ -696,14 +704,10 @@ namespace ignite /** Array to store statuses of rows fetched by the last fetch. */ uint16_t* rowStatuses; - /** Offset added to pointers to change binding of parameters. */ - int* paramBindOffset; - /** Offset added to pointers to change binding of column data. */ int* columnBindOffset; - /** Index of the parameter, which is currently being set. */ - uint16_t currentParamIdx; + app::ParameterSet parameters; }; } } diff --git a/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj b/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj index b00e4326e8671..e47f618aa83ad 100644 --- a/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj +++ b/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj @@ -160,6 +160,7 @@ + @@ -172,10 +173,12 @@ + + @@ -197,6 +200,7 @@ + @@ -215,6 +219,7 @@ + diff --git a/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj.filters b/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj.filters index 1828be1028382..e252d5d6c14dd 100644 --- a/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj.filters +++ b/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj.filters @@ -139,6 +139,15 @@ Code + + Code\app + + + Code\query + + + Code + @@ -266,5 +275,11 @@ Code + + Code\app + + + Code\query + \ No newline at end of file diff --git a/modules/platforms/cpp/odbc/src/app/application_data_buffer.cpp b/modules/platforms/cpp/odbc/src/app/application_data_buffer.cpp index 026cd60f79206..bb9e58498f763 100644 --- a/modules/platforms/cpp/odbc/src/app/application_data_buffer.cpp +++ b/modules/platforms/cpp/odbc/src/app/application_data_buffer.cpp @@ -33,35 +33,38 @@ namespace ignite { namespace app { - using ignite::impl::binary::BinaryUtils; + using impl::binary::BinaryUtils; ApplicationDataBuffer::ApplicationDataBuffer() : type(type_traits::IGNITE_ODBC_C_TYPE_UNSUPPORTED), buffer(0), buflen(0), reslen(0), - offset(0) + byteOffset(0), + elementOffset(0) { // No-op. } ApplicationDataBuffer::ApplicationDataBuffer(type_traits::IgniteSqlType type, - void* buffer, SqlLen buflen, SqlLen* reslen, int** offset) : + void* buffer, SqlLen buflen, SqlLen* reslen) : type(type), buffer(buffer), buflen(buflen), reslen(reslen), - offset(offset) + byteOffset(0), + elementOffset(0) { // No-op. } - ApplicationDataBuffer::ApplicationDataBuffer(const ApplicationDataBuffer & other) : + ApplicationDataBuffer::ApplicationDataBuffer(const ApplicationDataBuffer& other) : type(other.type), buffer(other.buffer), buflen(other.buflen), reslen(other.reslen), - offset(other.offset) + byteOffset(other.byteOffset), + elementOffset(other.elementOffset) { // No-op. } @@ -77,7 +80,8 @@ namespace ignite buffer = other.buffer; buflen = other.buflen; reslen = other.reslen; - offset = other.offset; + byteOffset = other.byteOffset; + elementOffset = other.elementOffset; return *this; } @@ -1100,22 +1104,22 @@ namespace ignite const void* ApplicationDataBuffer::GetData() const { - return ApplyOffset(buffer); + return ApplyOffset(buffer, GetElementSize()); } const SqlLen* ApplicationDataBuffer::GetResLen() const { - return ApplyOffset(reslen); + return ApplyOffset(reslen, sizeof(*reslen)); } void* ApplicationDataBuffer::GetData() { - return ApplyOffset(buffer); + return ApplyOffset(buffer, GetElementSize()); } SqlLen* ApplicationDataBuffer::GetResLen() { - return ApplyOffset(reslen); + return ApplyOffset(reslen, sizeof(*reslen)); } template @@ -1461,12 +1465,12 @@ namespace ignite } template - T* ApplicationDataBuffer::ApplyOffset(T* ptr) const + T* ApplicationDataBuffer::ApplyOffset(T* ptr, size_t elemSize) const { - if (!ptr || !offset || !*offset) + if (!ptr) return ptr; - return utility::GetPointerWithOffset(ptr, **offset); + return utility::GetPointerWithOffset(ptr, byteOffset + elemSize * elementOffset); } bool ApplicationDataBuffer::IsDataAtExec() const @@ -1556,6 +1560,64 @@ namespace ignite return 0; } + SqlLen ApplicationDataBuffer::GetElementSize() const + { + using namespace type_traits; + + switch (type) + { + case IGNITE_ODBC_C_TYPE_WCHAR: + case IGNITE_ODBC_C_TYPE_CHAR: + case IGNITE_ODBC_C_TYPE_BINARY: + return buflen; + + case IGNITE_ODBC_C_TYPE_SIGNED_SHORT: + case IGNITE_ODBC_C_TYPE_UNSIGNED_SHORT: + return static_cast(sizeof(short)); + + case IGNITE_ODBC_C_TYPE_SIGNED_LONG: + case IGNITE_ODBC_C_TYPE_UNSIGNED_LONG: + return static_cast(sizeof(long)); + + case IGNITE_ODBC_C_TYPE_FLOAT: + return static_cast(sizeof(float)); + + case IGNITE_ODBC_C_TYPE_DOUBLE: + return static_cast(sizeof(double)); + + case IGNITE_ODBC_C_TYPE_BIT: + case IGNITE_ODBC_C_TYPE_SIGNED_TINYINT: + case IGNITE_ODBC_C_TYPE_UNSIGNED_TINYINT: + return static_cast(sizeof(char)); + + case IGNITE_ODBC_C_TYPE_SIGNED_BIGINT: + case IGNITE_ODBC_C_TYPE_UNSIGNED_BIGINT: + return static_cast(sizeof(SQLBIGINT)); + + case IGNITE_ODBC_C_TYPE_TDATE: + return static_cast(sizeof(SQL_DATE_STRUCT)); + + case IGNITE_ODBC_C_TYPE_TTIME: + return static_cast(sizeof(SQL_TIME_STRUCT)); + + case IGNITE_ODBC_C_TYPE_TTIMESTAMP: + return static_cast(sizeof(SQL_TIMESTAMP_STRUCT)); + + case IGNITE_ODBC_C_TYPE_NUMERIC: + return static_cast(sizeof(SQL_NUMERIC_STRUCT)); + + case IGNITE_ODBC_C_TYPE_GUID: + return static_cast(sizeof(SQLGUID)); + + case IGNITE_ODBC_C_TYPE_DEFAULT: + case IGNITE_ODBC_C_TYPE_UNSUPPORTED: + default: + break; + } + + return 0; + } + SqlLen ApplicationDataBuffer::GetInputSize() const { if (!IsDataAtExec()) diff --git a/modules/platforms/cpp/odbc/src/app/parameter.cpp b/modules/platforms/cpp/odbc/src/app/parameter.cpp index ded2e4bd29072..083b744c04686 100644 --- a/modules/platforms/cpp/odbc/src/app/parameter.cpp +++ b/modules/platforms/cpp/odbc/src/app/parameter.cpp @@ -16,8 +16,6 @@ */ #include -#include -#include #include "ignite/odbc/system/odbc_constants.h" #include "ignite/odbc/app/parameter.h" @@ -78,7 +76,7 @@ namespace ignite return *this; } - void Parameter::Write(ignite::impl::binary::BinaryWriterImpl& writer) const + void Parameter::Write(impl::binary::BinaryWriterImpl& writer, int offset, SqlUlen idx) const { if (buffer.GetInputSize() == SQL_NULL_DATA) { @@ -89,6 +87,8 @@ namespace ignite // Buffer to use to get data. ApplicationDataBuffer buf(buffer); + buf.SetByteOffset(offset); + buf.SetElementOffset(idx); SqlLen storedDataLen = static_cast(storedData.size()); @@ -209,6 +209,11 @@ namespace ignite return buffer; } + const ApplicationDataBuffer& Parameter::GetBuffer() const + { + return buffer; + } + void Parameter::ResetStoredData() { storedData.clear(); diff --git a/modules/platforms/cpp/odbc/src/app/parameter_set.cpp b/modules/platforms/cpp/odbc/src/app/parameter_set.cpp new file mode 100644 index 0000000000000..c110d05dc5b38 --- /dev/null +++ b/modules/platforms/cpp/odbc/src/app/parameter_set.cpp @@ -0,0 +1,242 @@ +/* + * 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. + */ + +#include "ignite/odbc/app/parameter_set.h" + +namespace ignite +{ + namespace odbc + { + namespace app + { + ParameterSet::ParameterSet(): + parameters(), + paramTypes(), + paramBindOffset(0), + processedParamRows(0), + paramSetSize(1), + paramSetPos(0), + currentParamIdx(0), + typesSet(false) + { + // No-op. + } + + void ParameterSet::SetParamSetSize(SqlUlen size) + { + paramSetSize = size; + } + + void ParameterSet::BindParameter(uint16_t paramIdx, const Parameter& param) + { + parameters[paramIdx] = param; + } + + void ParameterSet::UnbindParameter(uint16_t paramIdx) + { + parameters.erase(paramIdx); + } + + void ParameterSet::UnbindAll() + { + parameters.clear(); + } + + uint16_t ParameterSet::GetParametersNumber() const + { + return static_cast(parameters.size()); + } + + void ParameterSet::SetParamBindOffsetPtr(int* ptr) + { + paramBindOffset = ptr; + } + + int* ParameterSet::GetParamBindOffsetPtr() + { + return paramBindOffset; + } + + void ParameterSet::Prepare() + { + paramTypes.clear(); + + typesSet = false; + + paramSetPos = 0; + + for (ParameterBindingMap::iterator it = parameters.begin(); it != parameters.end(); ++it) + it->second.ResetStoredData(); + } + + bool ParameterSet::IsDataAtExecNeeded() const + { + for (ParameterBindingMap::const_iterator it = parameters.begin(); it != parameters.end(); ++it) + { + if (!it->second.IsDataReady()) + return true; + } + + return false; + } + + void ParameterSet::SetParamsProcessedPtr(SqlUlen* ptr) + { + processedParamRows = ptr; + } + + SqlUlen* ParameterSet::GetParamsProcessedPtr() + { + return processedParamRows; + } + + void ParameterSet::SetParamsProcessed(SqlUlen processed) const + { + if (processedParamRows) + *processedParamRows = processed; + } + + void ParameterSet::UpdateParamsTypes(const ParameterTypeVector& meta) + { + paramTypes = meta; + + typesSet = true; + } + + int8_t ParameterSet::GetParamType(int16_t idx, int8_t dflt) + { + if (idx > 0 && static_cast(idx) <= paramTypes.size()) + return paramTypes[idx - 1]; + + return dflt; + } + + uint16_t ParameterSet::GetExpectedParamNum() + { + return static_cast(paramTypes.size()); + } + + bool ParameterSet::IsMetadataSet() const + { + return typesSet; + } + + bool ParameterSet::IsParameterSelected() const + { + return currentParamIdx != 0; + } + + Parameter* ParameterSet::GetParameter(uint16_t idx) + { + ParameterBindingMap::iterator it = parameters.find(currentParamIdx); + + if (it != parameters.end()) + return &it->second; + + return 0; + } + + Parameter* ParameterSet::GetSelectedParameter() + { + return GetParameter(currentParamIdx); + } + + Parameter* ParameterSet::SelectNextParameter() + { + for (ParameterBindingMap::iterator it = parameters.begin(); it != parameters.end(); ++it) + { + uint16_t paramIdx = it->first; + Parameter& param = it->second; + + if (!param.IsDataReady()) + { + currentParamIdx = paramIdx; + + return ¶m; + } + } + + return 0; + } + + void ParameterSet::Write(impl::binary::BinaryWriterImpl& writer) const + { + writer.WriteInt32(CalculateRowLen()); + + WriteRow(writer, 0); + } + + void ParameterSet::Write(impl::binary::BinaryWriterImpl& writer, SqlUlen begin, SqlUlen end, bool last) const + { + int32_t rowLen = CalculateRowLen(); + + writer.WriteInt32(rowLen); + + SqlUlen intervalEnd = std::min(paramSetSize, end); + + assert(begin < intervalEnd); + + int32_t intervalLen = static_cast(intervalEnd - begin); + + writer.WriteInt32(intervalLen); + writer.WriteBool(last); + + if (rowLen) + { + for (SqlUlen i = begin; i < intervalEnd; ++i) + WriteRow(writer, i); + } + } + + void ParameterSet::WriteRow(impl::binary::BinaryWriterImpl& writer, SqlUlen idx) const + { + uint16_t prev = 0; + + int appOffset = paramBindOffset ? *paramBindOffset : 0; + + for (ParameterBindingMap::const_iterator it = parameters.begin(); it != parameters.end(); ++it) + { + uint16_t paramIdx = it->first; + const Parameter& param = it->second; + + while ((paramIdx - prev) > 1) + { + writer.WriteNull(); + ++prev; + } + + param.Write(writer, appOffset, idx); + + prev = paramIdx; + } + } + + int32_t ParameterSet::CalculateRowLen() const + { + if (!parameters.empty()) + return static_cast(parameters.rbegin()->first); + + return 0; + } + + int32_t ParameterSet::GetParamSetSize() const + { + return static_cast(paramSetSize); + } + } + } +} diff --git a/modules/platforms/cpp/odbc/src/config/connection_info.cpp b/modules/platforms/cpp/odbc/src/config/connection_info.cpp index a34d43447e4f2..a74f23cb65e7d 100644 --- a/modules/platforms/cpp/odbc/src/config/connection_info.cpp +++ b/modules/platforms/cpp/odbc/src/config/connection_info.cpp @@ -118,6 +118,8 @@ namespace ignite DBG_STR_CASE(SQL_CONVERT_WLONGVARCHAR); DBG_STR_CASE(SQL_CONVERT_WVARCHAR); DBG_STR_CASE(SQL_CONVERT_GUID); + DBG_STR_CASE(SQL_PARAM_ARRAY_ROW_COUNTS); + DBG_STR_CASE(SQL_PARAM_ARRAY_SELECTS); default: break; } @@ -592,6 +594,40 @@ namespace ignite SQL_CVT_BINARY | SQL_CVT_VARBINARY | SQL_CVT_LONGVARBINARY | SQL_CVT_GUID; #endif //SQL_CONVERT_GUID +#ifdef SQL_PARAM_ARRAY_ROW_COUNTS + // Enumerating the driver's properties regarding the availability of row counts in a parameterized + // execution. Has the following values: + // + // SQL_PARC_BATCH = Individual row counts are available for each set of parameters. This is conceptually + // equivalent to the driver generating a batch of SQL statements, one for each parameter set in the + // array. Extended error information can be retrieved by using the SQL_PARAM_STATUS_PTR descriptor + // field. + // + // SQL_PARC_NO_BATCH = There is only one row count available, which is the cumulative row count + // resulting from the execution of the statement for the entire array of parameters. This is + // conceptually equivalent to treating the statement together with the complete parameter array as + // one atomic unit. Errors are handled the same as if one statement were executed. + intParams[SQL_PARAM_ARRAY_ROW_COUNTS] = SQL_PARC_NO_BATCH; +#endif //SQL_PARAM_ARRAY_ROW_COUNTS + +#ifdef SQL_PARAM_ARRAY_SELECTS + // Enumerating the driver's properties regarding the availability of result sets in a parameterized + // execution. Has the following values: + // + // SQL_PAS_BATCH = There is one result set available per set of parameters. This is conceptually + // equivalent to the driver generating a batch of SQL statements, one for each parameter set in + // the array. + // + // SQL_PAS_NO_BATCH = There is only one result set available, which represents the cumulative result set + // resulting from the execution of the statement for the complete array of parameters. This is + // conceptually equivalent to treating the statement together with the complete parameter array as + // one atomic unit. + // + // SQL_PAS_NO_SELECT = A driver does not allow a result - set generating statement to be executed with + // an array of parameters. + intParams[SQL_PARAM_ARRAY_SELECTS] = SQL_PAS_NO_SELECT; +#endif //SQL_PARAM_ARRAY_SELECTS + //======================= Short Params ======================== #ifdef SQL_MAX_CONCURRENT_ACTIVITIES // The maximum number of active statements that the driver can diff --git a/modules/platforms/cpp/odbc/src/message.cpp b/modules/platforms/cpp/odbc/src/message.cpp new file mode 100644 index 0000000000000..741540526fb19 --- /dev/null +++ b/modules/platforms/cpp/odbc/src/message.cpp @@ -0,0 +1,366 @@ +/* + * 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. + */ + +#include "ignite/odbc/message.h" +#include "ignite/odbc/utility.h" + +namespace ignite +{ + namespace odbc + { + HandshakeRequest::HandshakeRequest(int64_t version, bool distributedJoins, bool enforceJoinOrder): + version(version), + distributedJoins(distributedJoins), + enforceJoinOrder(enforceJoinOrder) + { + // No-op. + } + + HandshakeRequest::~HandshakeRequest() + { + // No-op. + } + + void HandshakeRequest::Write(impl::binary::BinaryWriterImpl& writer) const + { + writer.WriteInt8(REQUEST_TYPE_HANDSHAKE); + + writer.WriteInt64(version); + + writer.WriteBool(distributedJoins); + writer.WriteBool(enforceJoinOrder); + } + + QueryExecuteRequest::QueryExecuteRequest(const std::string& schema, const std::string& sql, const app::ParameterSet& params): + cache(schema), + sql(sql), + params(params) + { + // No-op. + } + + QueryExecuteRequest::~QueryExecuteRequest() + { + // No-op. + } + + void QueryExecuteRequest::Write(impl::binary::BinaryWriterImpl& writer) const + { + writer.WriteInt8(REQUEST_TYPE_EXECUTE_SQL_QUERY); + + if (cache.empty()) + writer.WriteNull(); + else + writer.WriteObject(cache); + + writer.WriteObject(sql); + + params.Write(writer); + } + + QueryExecuteBatchtRequest::QueryExecuteBatchtRequest(const std::string& schema, const std::string& sql, + const app::ParameterSet& params, SqlUlen begin, SqlUlen end, bool last): + schema(schema), + sql(sql), + params(params), + begin(begin), + end(end), + last(last) + { + // No-op. + } + + QueryExecuteBatchtRequest::~QueryExecuteBatchtRequest() + { + // No-op. + } + + void QueryExecuteBatchtRequest::Write(impl::binary::BinaryWriterImpl& writer) const + { + writer.WriteInt8(REQUEST_TYPE_EXECUTE_SQL_QUERY_BATCH); + + if (schema.empty()) + writer.WriteNull(); + else + writer.WriteObject(schema); + + writer.WriteObject(sql); + + params.Write(writer, begin, end, last); + } + + QueryCloseRequest::QueryCloseRequest(int64_t queryId): queryId(queryId) + { + // No-op. + } + + QueryCloseRequest::~QueryCloseRequest() + { + // No-op. + } + + void QueryCloseRequest::Write(impl::binary::BinaryWriterImpl& writer) const + { + writer.WriteInt8(REQUEST_TYPE_CLOSE_SQL_QUERY); + writer.WriteInt64(queryId); + } + + QueryFetchRequest::QueryFetchRequest(int64_t queryId, int32_t pageSize): + queryId(queryId), + pageSize(pageSize) + { + // No-op. + } + + QueryFetchRequest::~QueryFetchRequest() + { + // No-op. + } + + void QueryFetchRequest::Write(impl::binary::BinaryWriterImpl& writer) const + { + writer.WriteInt8(REQUEST_TYPE_FETCH_SQL_QUERY); + writer.WriteInt64(queryId); + writer.WriteInt32(pageSize); + } + + QueryGetColumnsMetaRequest::QueryGetColumnsMetaRequest(const std::string& schema, const std::string& table, const std::string& column): + schema(schema), + table(table), + column(column) + { + // No-op. + } + + QueryGetColumnsMetaRequest::~QueryGetColumnsMetaRequest() + { + // No-op. + } + + void QueryGetColumnsMetaRequest::Write(impl::binary::BinaryWriterImpl& writer) const + { + writer.WriteInt8(REQUEST_TYPE_GET_COLUMNS_METADATA); + + utility::WriteString(writer, schema); + utility::WriteString(writer, table); + utility::WriteString(writer, column); + } + + QueryGetTablesMetaRequest::QueryGetTablesMetaRequest(const std::string& catalog, const std::string& schema, const std::string& table, const std::string& tableTypes): + catalog(catalog), + schema(schema), + table(table), + tableTypes(tableTypes) + { + // No-op. + } + + QueryGetTablesMetaRequest::~QueryGetTablesMetaRequest() + { + // No-op. + } + + void QueryGetTablesMetaRequest::Write(impl::binary::BinaryWriterImpl& writer) const + { + writer.WriteInt8(REQUEST_TYPE_GET_TABLES_METADATA); + + utility::WriteString(writer, catalog); + utility::WriteString(writer, schema); + utility::WriteString(writer, table); + utility::WriteString(writer, tableTypes); + } + + void QueryGetParamsMetaRequest::Write(impl::binary::BinaryWriterImpl& writer) const + { + writer.WriteInt8(REQUEST_TYPE_GET_PARAMS_METADATA); + + utility::WriteString(writer, cacheName); + utility::WriteString(writer, sqlQuery); + } + + Response::Response(): + status(RESPONSE_STATUS_FAILED), + error() + { + // No-op. + } + + Response::~Response() + { + // No-op. + } + + void Response::Read(impl::binary::BinaryReaderImpl& reader) + { + status = reader.ReadInt8(); + + if (status == RESPONSE_STATUS_SUCCESS) + ReadOnSuccess(reader); + else + utility::ReadString(reader, error); + } + + void Response::ReadOnSuccess(impl::binary::BinaryReaderImpl&) + { + // No-op. + } + + HandshakeResponse::HandshakeResponse(): + accepted(false), + protoVerSince(), + currentVer() + { + // No-op. + } + + HandshakeResponse::~HandshakeResponse() + { + // No-op. + } + + void HandshakeResponse::ReadOnSuccess(impl::binary::BinaryReaderImpl& reader) + { + accepted = reader.ReadBool(); + + if (!accepted) + { + utility::ReadString(reader, protoVerSince); + utility::ReadString(reader, currentVer); + } + } + + QueryCloseResponse::QueryCloseResponse(): queryId(0) + { + // No-op. + } + + QueryCloseResponse::~QueryCloseResponse() + { + // No-op. + } + + void QueryCloseResponse::ReadOnSuccess(impl::binary::BinaryReaderImpl& reader) + { + queryId = reader.ReadInt64(); + } + + QueryExecuteResponse::QueryExecuteResponse(): queryId(0), meta() + { + // No-op. + } + + QueryExecuteResponse::~QueryExecuteResponse() + { + // No-op. + } + + void QueryExecuteResponse::ReadOnSuccess(impl::binary::BinaryReaderImpl& reader) + { + queryId = reader.ReadInt64(); + + meta::ReadColumnMetaVector(reader, meta); + } + + QueryExecuteBatchResponse::QueryExecuteBatchResponse(): + affectedRows(0), + errorSetIdx(-1), + errorMessage() + { + // No-op. + } + + QueryExecuteBatchResponse::~QueryExecuteBatchResponse() + { + // No-op. + } + + void QueryExecuteBatchResponse::ReadOnSuccess(impl::binary::BinaryReaderImpl& reader) + { + bool success = reader.ReadBool(); + affectedRows = reader.ReadInt64(); + + if (!success) + { + errorSetIdx = reader.ReadInt64(); + errorMessage = reader.ReadObject(); + } + } + + QueryFetchResponse::QueryFetchResponse(ResultPage& resultPage): queryId(0), resultPage(resultPage) + { + // No-op. + } + + QueryFetchResponse::~QueryFetchResponse() + { + // No-op. + } + + void QueryFetchResponse::ReadOnSuccess(impl::binary::BinaryReaderImpl& reader) + { + queryId = reader.ReadInt64(); + + resultPage.Read(reader); + } + + QueryGetColumnsMetaResponse::QueryGetColumnsMetaResponse() + { + // No-op. + } + + QueryGetColumnsMetaResponse::~QueryGetColumnsMetaResponse() + { + // No-op. + } + + void QueryGetColumnsMetaResponse::ReadOnSuccess(impl::binary::BinaryReaderImpl& reader) + { + meta::ReadColumnMetaVector(reader, meta); + } + + QueryGetTablesMetaResponse::QueryGetTablesMetaResponse() + { + // No-op. + } + + QueryGetTablesMetaResponse::~QueryGetTablesMetaResponse() + { + // No-op. + } + + void QueryGetTablesMetaResponse::ReadOnSuccess(impl::binary::BinaryReaderImpl& reader) + { + meta::ReadTableMetaVector(reader, meta); + } + + QueryGetParamsMetaResponse::QueryGetParamsMetaResponse() + { + // No-op. + } + + QueryGetParamsMetaResponse::~QueryGetParamsMetaResponse() + { + // No-op. + } + + void QueryGetParamsMetaResponse::ReadOnSuccess(impl::binary::BinaryReaderImpl& reader) + { + utility::ReadByteArray(reader, typeIds); + } + } +} + diff --git a/modules/platforms/cpp/odbc/src/odbc.cpp b/modules/platforms/cpp/odbc/src/odbc.cpp index 542e64cca668f..af8c73489902c 100644 --- a/modules/platforms/cpp/odbc/src/odbc.cpp +++ b/modules/platforms/cpp/odbc/src/odbc.cpp @@ -417,9 +417,9 @@ namespace ignite using odbc::Statement; using odbc::app::ApplicationDataBuffer; - LOG_MSG("SQLBindCol called: index=" << colNum << ", type=" << targetType << - ", targetValue=" << reinterpret_cast(targetValue) << - ", bufferLength=" << bufferLength << + LOG_MSG("SQLBindCol called: index=" << colNum << ", type=" << targetType << + ", targetValue=" << reinterpret_cast(targetValue) << + ", bufferLength=" << bufferLength << ", lengthInd=" << reinterpret_cast(strLengthOrIndicator)); Statement *statement = reinterpret_cast(stmt); @@ -618,7 +618,7 @@ namespace ignite if (!statement) return SQL_INVALID_HANDLE; - statement->BindParameter(paramIdx, ioType, bufferType, paramSqlType, columnSize, decDigits, buffer, bufferLen, resLen); + statement->BindParameter(paramIdx,ioType , bufferType, paramSqlType, columnSize, decDigits, buffer, bufferLen, resLen); return statement->GetDiagnosticRecords().GetReturnCode(); } @@ -897,7 +897,13 @@ namespace ignite return SQL_INVALID_HANDLE; if (paramCnt) - *paramCnt = static_cast(statement->GetParametersNumber()); + { + uint16_t num = 0; + + statement->GetParametersNumber(num); + + *paramCnt = static_cast(num); + } return statement->GetDiagnosticRecords().GetReturnCode(); } diff --git a/modules/platforms/cpp/odbc/src/query/batch_query.cpp b/modules/platforms/cpp/odbc/src/query/batch_query.cpp new file mode 100644 index 0000000000000..dd929026750b1 --- /dev/null +++ b/modules/platforms/cpp/odbc/src/query/batch_query.cpp @@ -0,0 +1,198 @@ +/* + * 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. + */ + +#include "ignite/odbc/connection.h" +#include "ignite/odbc/message.h" +#include "ignite/odbc/query/batch_query.h" +#include "ignite/odbc/log.h" + +namespace ignite +{ + namespace odbc + { + namespace query + { + BatchQuery::BatchQuery(diagnostic::Diagnosable& diag, Connection& connection, + const std::string& sql, const app::ParameterSet& params) : + Query(diag, BATCH), + connection(connection), + sql(sql), + params(params), + resultMeta(), + rowsAffected(0), + setsProcessed(0), + executed(false), + dataRetrieved(false) + { + // No-op. + } + + BatchQuery::~BatchQuery() + { + // No-op. + } + + SqlResult BatchQuery::Execute() + { + if (executed) + { + diag.AddStatusRecord(SQL_STATE_HY010_SEQUENCE_ERROR, "Query cursor is in open state already."); + + return SQL_RESULT_ERROR; + } + + int32_t maxPageSize = connection.GetConfiguration().GetPageSize(); + int32_t rowNum = params.GetParamSetSize(); + SqlResult res; + + int32_t processed = 0; + + do { + int32_t currentPageSize = std::min(maxPageSize, rowNum - processed); + bool lastPage = currentPageSize == rowNum - processed; + + res = MakeRequestExecuteBatch(processed, processed + currentPageSize, lastPage); + + processed += currentPageSize; + } while (res == SQL_RESULT_SUCCESS && processed < rowNum); + + params.SetParamsProcessed(static_cast(setsProcessed)); + + return res; + } + + const meta::ColumnMetaVector& BatchQuery::GetMeta() const + { + return resultMeta; + } + + SqlResult BatchQuery::FetchNextRow(app::ColumnBindingMap& columnBindings) + { + if (!executed) + { + diag.AddStatusRecord(SQL_STATE_HY010_SEQUENCE_ERROR, "Query was not executed."); + + return SQL_RESULT_ERROR; + } + + if (dataRetrieved) + return SQL_RESULT_NO_DATA; + + app::ColumnBindingMap::iterator it = columnBindings.find(1); + + if (it != columnBindings.end()) + it->second.PutInt64(rowsAffected); + + dataRetrieved = true; + + return SQL_RESULT_SUCCESS; + } + + SqlResult BatchQuery::GetColumn(uint16_t columnIdx, app::ApplicationDataBuffer& buffer) + { + if (!executed) + { + diag.AddStatusRecord(SQL_STATE_HY010_SEQUENCE_ERROR, "Query was not executed."); + + return SQL_RESULT_ERROR; + } + + if (dataRetrieved) + return SQL_RESULT_NO_DATA; + + if (columnIdx != 1) + { + std::stringstream builder; + builder << "Column with id " << columnIdx << " is not available in result set."; + + diag.AddStatusRecord(SQL_STATE_HY000_GENERAL_ERROR, builder.str()); + + return SQL_RESULT_ERROR; + } + + buffer.PutInt64(rowsAffected); + + return SQL_RESULT_SUCCESS; + } + + SqlResult BatchQuery::Close() + { + return SQL_RESULT_SUCCESS; + } + + bool BatchQuery::DataAvailable() const + { + return false; + } + + int64_t BatchQuery::AffectedRows() const + { + return rowsAffected; + } + + SqlResult BatchQuery::MakeRequestExecuteBatch(SqlUlen begin, SqlUlen end, bool last) + { + const std::string& schema = connection.GetCache(); + + QueryExecuteBatchtRequest req(schema, sql, params, begin, end, last); + QueryExecuteBatchResponse rsp; + + try + { + connection.SyncMessage(req, rsp); + } + catch (const IgniteError& err) + { + diag.AddStatusRecord(SQL_STATE_HYT01_CONNECTIOIN_TIMEOUT, err.GetText()); + + return SQL_RESULT_ERROR; + } + + if (rsp.GetStatus() != RESPONSE_STATUS_SUCCESS) + { + LOG_MSG("Error: " << rsp.GetError()); + + diag.AddStatusRecord(SQL_STATE_HY000_GENERAL_ERROR, rsp.GetError()); + + return SQL_RESULT_ERROR; + } + + rowsAffected += rsp.GetAffectedRows(); + LOG_MSG("rowsAffected: " << rowsAffected); + + if (!rsp.GetErrorMessage().empty()) + { + LOG_MSG("Error: " << rsp.GetErrorMessage()); + + setsProcessed += rsp.GetErrorSetIdx(); + LOG_MSG("setsProcessed: " << setsProcessed); + + diag.AddStatusRecord(SQL_STATE_HY000_GENERAL_ERROR, rsp.GetErrorMessage(), + static_cast(setsProcessed), 0); + + return SQL_RESULT_SUCCESS_WITH_INFO; + } + + setsProcessed += end - begin; + LOG_MSG("setsProcessed: " << setsProcessed); + + return SQL_RESULT_SUCCESS; + } + } + } +} + diff --git a/modules/platforms/cpp/odbc/src/query/data_query.cpp b/modules/platforms/cpp/odbc/src/query/data_query.cpp index c9762ad2aaa14..f2ee7ec4758ad 100644 --- a/modules/platforms/cpp/odbc/src/query/data_query.cpp +++ b/modules/platforms/cpp/odbc/src/query/data_query.cpp @@ -19,6 +19,7 @@ #include "ignite/odbc/message.h" #include "ignite/odbc/log.h" #include "ignite/odbc/query/data_query.h" +#include "ignite/odbc/query/batch_query.h" namespace ignite { @@ -26,9 +27,8 @@ namespace ignite { namespace query { - DataQuery::DataQuery(diagnostic::Diagnosable& diag, - Connection& connection, const std::string& sql, - const app::ParameterBindingMap& params) : + DataQuery::DataQuery(diagnostic::Diagnosable& diag, Connection& connection, + const std::string& sql, const app::ParameterSet& params) : Query(diag, DATA), connection(connection), sql(sql), @@ -39,7 +39,7 @@ namespace ignite DataQuery::~DataQuery() { - Close(); + InternalClose(); } SqlResult DataQuery::Execute() @@ -140,6 +140,11 @@ namespace ignite } SqlResult DataQuery::Close() + { + return InternalClose(); + } + + SqlResult DataQuery::InternalClose() { if (!cursor.get()) return SQL_RESULT_SUCCESS; diff --git a/modules/platforms/cpp/odbc/src/statement.cpp b/modules/platforms/cpp/odbc/src/statement.cpp index 2395d66864363..328bac958d987 100644 --- a/modules/platforms/cpp/odbc/src/statement.cpp +++ b/modules/platforms/cpp/odbc/src/statement.cpp @@ -16,6 +16,7 @@ */ #include "ignite/odbc/system/odbc_constants.h" +#include "ignite/odbc/query/batch_query.h" #include "ignite/odbc/query/data_query.h" #include "ignite/odbc/query/column_metadata_query.h" #include "ignite/odbc/query/table_metadata_query.h" @@ -39,9 +40,8 @@ namespace ignite currentQuery(), rowsFetched(0), rowStatuses(0), - paramBindOffset(0), columnBindOffset(0), - currentParamIdx(0) + parameters() { // No-op. } @@ -91,8 +91,6 @@ namespace ignite void Statement::SafeBindColumn(uint16_t columnIdx, const app::ApplicationDataBuffer& buffer) { columnBindings[columnIdx] = buffer; - - columnBindings[columnIdx].SetPtrToOffsetPtr(&columnBindOffset); } void Statement::SafeUnbindColumn(uint16_t columnIdx) @@ -143,38 +141,45 @@ namespace ignite void Statement::BindParameter(uint16_t paramIdx, int16_t ioType, int16_t bufferType, int16_t paramSqlType, SqlUlen columnSize, int16_t decDigits, void* buffer, SqlLen bufferLen, SqlLen* resLen) { - IGNITE_ODBC_API_CALL(InternalBindParameter(paramIdx, ioType, bufferType, paramSqlType, columnSize, decDigits, buffer, bufferLen, resLen)); + IGNITE_ODBC_API_CALL(InternalBindParameter(paramIdx, ioType, bufferType, paramSqlType, columnSize, + decDigits, buffer, bufferLen, resLen)); } - SqlResult Statement::InternalBindParameter(uint16_t paramIdx, int16_t ioType, int16_t bufferType, int16_t paramSqlType, - SqlUlen columnSize, int16_t decDigits, void* buffer, SqlLen bufferLen, SqlLen* resLen) + + SqlResult Statement::InternalBindParameter(uint16_t paramIdx, int16_t ioType, int16_t bufferType, + int16_t paramSqlType, SqlUlen columnSize, int16_t decDigits, + void* buffer, SqlLen bufferLen, SqlLen* resLen) { - using namespace odbc::type_traits; - using odbc::Statement; - using odbc::app::ApplicationDataBuffer; - using odbc::app::Parameter; - using odbc::type_traits::IsSqlTypeSupported; + using namespace type_traits; + using app::ApplicationDataBuffer; + using app::Parameter; if (paramIdx == 0) { - AddStatusRecord(SQL_STATE_24000_INVALID_CURSOR_STATE, - "The value specified for the argument ParameterNumber was less than 1."); + std::stringstream builder; + builder << "The value specified for the argument ParameterNumber was less than 1. [ParameterNumber=" << paramIdx << ']'; + + AddStatusRecord(SQL_STATE_24000_INVALID_CURSOR_STATE, builder.str()); return SQL_RESULT_ERROR; } if (ioType != SQL_PARAM_INPUT) { - AddStatusRecord(SQL_STATE_HY105_INVALID_PARAMETER_TYPE, - "The value specified for the argument InputOutputType was not SQL_PARAM_INPUT."); + std::stringstream builder; + builder << "The value specified for the argument InputOutputType was not SQL_PARAM_INPUT. [ioType=" << ioType << ']'; + + AddStatusRecord(SQL_STATE_HY105_INVALID_PARAMETER_TYPE, builder.str()); return SQL_RESULT_ERROR; } if (!IsSqlTypeSupported(paramSqlType)) { - AddStatusRecord(SQL_STATE_HYC00_OPTIONAL_FEATURE_NOT_IMPLEMENTED, - "Data type is not supported."); + std::stringstream builder; + builder << "Data type is not supported. [typeId=" << paramSqlType << ']'; + + AddStatusRecord(SQL_STATE_HYC00_OPTIONAL_FEATURE_NOT_IMPLEMENTED, builder.str()); return SQL_RESULT_ERROR; } @@ -183,8 +188,10 @@ namespace ignite if (driverType == IGNITE_ODBC_C_TYPE_UNSUPPORTED) { - AddStatusRecord(odbc::SQL_STATE_HY003_INVALID_APPLICATION_BUFFER_TYPE, - "The argument TargetType was not a valid data type."); + std::stringstream builder; + builder << "The argument TargetType was not a valid data type. [TargetType=" << bufferType << ']'; + + AddStatusRecord(SQL_STATE_HY003_INVALID_APPLICATION_BUFFER_TYPE, builder.str()); return SQL_RESULT_ERROR; } @@ -205,19 +212,17 @@ namespace ignite void Statement::SafeBindParameter(uint16_t paramIdx, const app::Parameter& param) { - paramBindings[paramIdx] = param; - - paramBindings[paramIdx].GetBuffer().SetPtrToOffsetPtr(¶mBindOffset); + parameters.BindParameter(paramIdx, param); } void Statement::SafeUnbindParameter(uint16_t paramIdx) { - paramBindings.erase(paramIdx); + parameters.UnbindParameter(paramIdx); } void Statement::SafeUnbindAllParameters() { - paramBindings.clear(); + parameters.UnbindAll(); } void Statement::SetAttribute(int attr, void* value, SQLINTEGER valueLen) @@ -231,7 +236,7 @@ namespace ignite { case SQL_ATTR_ROW_ARRAY_SIZE: { - SQLULEN val = reinterpret_cast(value); + SqlUlen val = reinterpret_cast(value); LOG_MSG("SQL_ATTR_ROW_ARRAY_SIZE: " << val); @@ -274,6 +279,20 @@ namespace ignite break; } + case SQL_ATTR_PARAMSET_SIZE: + { + parameters.SetParamSetSize(reinterpret_cast(value)); + + break; + } + + case SQL_ATTR_PARAMS_PROCESSED_PTR: + { + parameters.SetParamsProcessedPtr(reinterpret_cast(value)); + + break; + } + default: { AddStatusRecord(SQL_STATE_HYC00_OPTIONAL_FEATURE_NOT_IMPLEMENTED, @@ -311,6 +330,9 @@ namespace ignite *val = static_cast(this); + if (valueLen) + *valueLen = SQL_IS_POINTER; + break; } @@ -320,14 +342,20 @@ namespace ignite *val = static_cast(1); + if (valueLen) + *valueLen = SQL_IS_INTEGER; + break; } case SQL_ATTR_ROWS_FETCHED_PTR: { - SQLULEN** val = reinterpret_cast(buf); + SqlUlen** val = reinterpret_cast(buf); + + *val = reinterpret_cast(GetRowsFetchedPtr()); - *val = reinterpret_cast(GetRowsFetchedPtr()); + if (valueLen) + *valueLen = SQL_IS_POINTER; break; } @@ -338,6 +366,9 @@ namespace ignite *val = reinterpret_cast(GetRowStatusesPtr()); + if (valueLen) + *valueLen = SQL_IS_POINTER; + break; } @@ -345,16 +376,46 @@ namespace ignite { SQLULEN** val = reinterpret_cast(buf); - *val = reinterpret_cast(GetParamBindOffsetPtr()); + *val = reinterpret_cast(parameters.GetParamBindOffsetPtr()); + + if (valueLen) + *valueLen = SQL_IS_POINTER; break; } case SQL_ATTR_ROW_BIND_OFFSET_PTR: { - SQLULEN** val = reinterpret_cast(buf); + SqlUlen** val = reinterpret_cast(buf); - *val = reinterpret_cast(GetColumnBindOffsetPtr()); + *val = reinterpret_cast(GetColumnBindOffsetPtr()); + + if (valueLen) + *valueLen = SQL_IS_POINTER; + + break; + } + + case SQL_ATTR_PARAMSET_SIZE: + { + SqlUlen* val = reinterpret_cast(buf); + + *val = static_cast(parameters.GetParamSetSize()); + + if (valueLen) + *valueLen = SQL_IS_UINTEGER; + + break; + } + + case SQL_ATTR_PARAMS_PROCESSED_PTR: + { + SqlUlen** val = reinterpret_cast(buf); + + *val = parameters.GetParamsProcessedPtr(); + + if (valueLen) + *valueLen = SQL_IS_POINTER; break; } @@ -371,23 +432,45 @@ namespace ignite return SQL_RESULT_SUCCESS; } - uint16_t Statement::GetParametersNumber() + void Statement::GetParametersNumber(uint16_t& paramNum) { - IGNITE_ODBC_API_CALL_ALWAYS_SUCCESS; - - return static_cast(paramBindings.size()); + IGNITE_ODBC_API_CALL(InternalGetParametersNumber(paramNum)); } - void Statement::SetParamBindOffsetPtr(int* ptr) + SqlResult Statement::InternalGetParametersNumber(uint16_t& paramNum) { - IGNITE_ODBC_API_CALL_ALWAYS_SUCCESS; + if (!currentQuery.get()) + { + AddStatusRecord(SQL_STATE_HY010_SEQUENCE_ERROR, "Query is not prepared."); + + return SQL_RESULT_ERROR; + } + + if (currentQuery->GetType() != query::Query::DATA) + { + paramNum = 0; + + return SQL_RESULT_SUCCESS; + } - paramBindOffset = ptr; + if (!parameters.IsMetadataSet()) + { + SqlResult res = UpdateParamsMeta(); + + if (res != SQL_RESULT_SUCCESS) + return res; + } + + paramNum = parameters.GetExpectedParamNum(); + + return SQL_RESULT_SUCCESS; } - int* Statement::GetParamBindOffsetPtr() + void Statement::SetParamBindOffsetPtr(int* ptr) { - return paramBindOffset; + IGNITE_ODBC_API_CALL_ALWAYS_SUCCESS; + + parameters.SetParamBindOffsetPtr(ptr); } void Statement::GetColumnData(uint16_t columnIdx, app::ApplicationDataBuffer& buffer) @@ -421,10 +504,10 @@ namespace ignite if (currentQuery.get()) currentQuery->Close(); - currentQuery.reset(new query::DataQuery(*this, connection, query, paramBindings)); - // Resetting parameters types as we are changing the query. - paramTypes.clear(); + parameters.Prepare(); + + currentQuery.reset(new query::DataQuery(*this, connection, query, parameters)); return SQL_RESULT_SUCCESS; } @@ -458,20 +541,31 @@ namespace ignite return SQL_RESULT_ERROR; } - bool paramDataReady = true; - - app::ParameterBindingMap::iterator it; - for (it = paramBindings.begin(); it != paramBindings.end(); ++it) + if (parameters.GetParamSetSize() > 1 && currentQuery->GetType() == query::Query::DATA) { - app::Parameter& param = it->second; + query::DataQuery& qry = static_cast(*currentQuery); - param.ResetStoredData(); + currentQuery.reset(new query::BatchQuery(*this, connection, qry.GetSql(), parameters)); + } + else if (parameters.GetParamSetSize() == 1 && currentQuery->GetType() == query::Query::BATCH) + { + query::BatchQuery& qry = static_cast(*currentQuery); - paramDataReady &= param.IsDataReady(); + currentQuery.reset(new query::DataQuery(*this, connection, qry.GetSql(), parameters)); } - if (!paramDataReady) + if (parameters.IsDataAtExecNeeded()) + { + if (currentQuery->GetType() == query::Query::BATCH) + { + AddStatusRecord(SQL_STATE_HYC00_OPTIONAL_FEATURE_NOT_IMPLEMENTED, + "Data-at-execution is not supported together with batching."); + + return SQL_RESULT_ERROR; + } + return SQL_RESULT_NEED_DATA; + } return currentQuery->Execute(); } @@ -597,8 +691,10 @@ namespace ignite { if (!type_traits::IsSqlTypeSupported(sqlType)) { - AddStatusRecord(SQL_STATE_HYC00_OPTIONAL_FEATURE_NOT_IMPLEMENTED, - "Data type is not supported."); + std::stringstream builder; + builder << "Data type is not supported. [typeId=" << sqlType << ']'; + + AddStatusRecord(SQL_STATE_HYC00_OPTIONAL_FEATURE_NOT_IMPLEMENTED, builder.str()); return SQL_RESULT_ERROR; } @@ -709,6 +805,12 @@ namespace ignite return SQL_RESULT_ERROR; } + if (columnBindOffset) + { + for (app::ColumnBindingMap::iterator it = columnBindings.begin(); it != columnBindings.end(); ++it) + it->second.SetByteOffset(*columnBindOffset); + } + SqlResult res = currentQuery->FetchNextRow(columnBindings); if (res == SQL_RESULT_SUCCESS) @@ -877,35 +979,24 @@ namespace ignite return SQL_RESULT_ERROR; } - app::ParameterBindingMap::iterator it; + app::Parameter *selected = parameters.GetSelectedParameter(); - if (currentParamIdx) + if (selected && !selected->IsDataReady()) { - it = paramBindings.find(currentParamIdx); + AddStatusRecord(SQL_STATE_22026_DATA_LENGTH_MISMATCH, + "Less data was sent for a parameter than was specified with " + "the StrLen_or_IndPtr argument in SQLBindParameter."); - if (it != paramBindings.end() && !it->second.IsDataReady()) - { - AddStatusRecord(SQL_STATE_22026_DATA_LENGTH_MISMATCH, - "Less data was sent for a parameter than was specified with " - "the StrLen_or_IndPtr argument in SQLBindParameter."); - - return SQL_RESULT_ERROR; - } + return SQL_RESULT_ERROR; } - for (it = paramBindings.begin(); it != paramBindings.end(); ++it) - { - uint16_t paramIdx = it->first; - app::Parameter& param = it->second; - - if (!param.IsDataReady()) - { - *paramPtr = param.GetBuffer().GetData(); + selected = parameters.SelectNextParameter(); - currentParamIdx = paramIdx; + if (selected) + { + *paramPtr = selected->GetBuffer().GetData(); - return SQL_RESULT_NEED_DATA; - } + return SQL_RESULT_NEED_DATA; } SqlResult res = currentQuery->Execute(); @@ -932,7 +1023,7 @@ namespace ignite return SQL_RESULT_ERROR; } - if (currentParamIdx == 0) + if (!parameters.IsParameterSelected()) { AddStatusRecord(SQL_STATE_HY010_SEQUENCE_ERROR, "Parameter is not selected with the SQLParamData."); @@ -940,9 +1031,9 @@ namespace ignite return SQL_RESULT_ERROR; } - app::ParameterBindingMap::iterator it = paramBindings.find(currentParamIdx); + app::Parameter* param = parameters.GetSelectedParameter(); - if (it == paramBindings.end()) + if (!param) { AddStatusRecord(SQL_STATE_HY000_GENERAL_ERROR, "Selected parameter has been unbound."); @@ -950,9 +1041,7 @@ namespace ignite return SQL_RESULT_ERROR; } - app::Parameter& param = it->second; - - param.PutData(data, len); + param->PutData(data, len); return SQL_RESULT_SUCCESS; } @@ -982,10 +1071,7 @@ namespace ignite return SQL_RESULT_ERROR; } - int8_t type = 0; - - if (paramNum > 0 && static_cast(paramNum) <= paramTypes.size()) - type = paramTypes[paramNum - 1]; + int8_t type = parameters.GetParamType(paramNum, 0); LOG_MSG("Type: " << type); @@ -996,10 +1082,7 @@ namespace ignite if (res != SQL_RESULT_SUCCESS) return res; - if (paramNum < 1 || static_cast(paramNum) > paramTypes.size()) - type = impl::binary::IGNITE_HDR_NULL; - else - type = paramTypes[paramNum - 1]; + type = parameters.GetParamType(paramNum, impl::binary::IGNITE_HDR_NULL); } if (dataType) @@ -1052,11 +1135,11 @@ namespace ignite return SQL_RESULT_ERROR; } - paramTypes = rsp.GetTypeIds(); + parameters.UpdateParamsTypes(rsp.GetTypeIds()); - for (size_t i = 0; i < paramTypes.size(); ++i) + for (size_t i = 0; i < rsp.GetTypeIds().size(); ++i) { - LOG_MSG("[" << i << "] Parameter type: " << paramTypes[i]); + LOG_MSG("[" << i << "] Parameter type: " << rsp.GetTypeIds()[i]); } return SQL_RESULT_SUCCESS; From cbe5df51c423568782e31245c7f1aa06c9ba3be1 Mon Sep 17 00:00:00 2001 From: Igor Sapego Date: Tue, 13 Jun 2017 19:47:00 +0300 Subject: [PATCH 149/357] IGNITE-5478: ODBC: SQLNumParams now returns number of required parameters. (cherry picked from commit b1c56a1) (cherry picked from commit 4a8f295) --- .../cpp/odbc-test/src/queries_test.cpp | 30 +++++++++++++++++++ .../cpp/odbc/include/ignite/odbc/statement.h | 4 +-- modules/platforms/cpp/odbc/src/odbc.cpp | 7 ++--- 3 files changed, 35 insertions(+), 6 deletions(-) diff --git a/modules/platforms/cpp/odbc-test/src/queries_test.cpp b/modules/platforms/cpp/odbc-test/src/queries_test.cpp index 8e528b4c0b910..082def8ea0290 100644 --- a/modules/platforms/cpp/odbc-test/src/queries_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/queries_test.cpp @@ -245,6 +245,25 @@ struct QueriesTestSuiteFixture BOOST_CHECK(ret == SQL_NO_DATA); } + void CheckParamsNum(const std::string& req, SQLSMALLINT expectedParamsNum) + { + std::vector req0(req.begin(), req.end()); + + SQLRETURN ret = SQLPrepare(stmt, &req0[0], static_cast(req0.size())); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + SQLSMALLINT paramsNum = -1; + + ret = SQLNumParams(stmt, ¶msNum); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECK_EQUAL(paramsNum, expectedParamsNum); + } + int CountRows(SQLHSTMT stmt) { int res = 0; @@ -1771,4 +1790,15 @@ BOOST_AUTO_TEST_CASE(TestTablesMeta) BOOST_CHECK(ret == SQL_NO_DATA); } +BOOST_AUTO_TEST_CASE(TestParamsNum) +{ + Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;SCHEMA=cache"); + + CheckParamsNum("SELECT * FROM TestType", 0); + CheckParamsNum("SELECT * FROM TestType WHERE _key=?", 1); + CheckParamsNum("SELECT * FROM TestType WHERE _key=? AND _val=?", 2); + CheckParamsNum("INSERT INTO TestType(_key, strField) VALUES(1, 'some')", 0); + CheckParamsNum("INSERT INTO TestType(_key, strField) VALUES(?, ?)", 2); +} + BOOST_AUTO_TEST_SUITE_END() diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h b/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h index 132cf78e45238..93018cd5c41ac 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h @@ -129,9 +129,9 @@ namespace ignite void GetAttribute(int attr, void* buf, SQLINTEGER bufLen, SQLINTEGER *valueLen); /** - * Get number of binded parameters. + * Get number parameters required by the prepared statement. * - * @param paramNum Number of binded parameters. + * @param paramNum Number of parameters. */ void GetParametersNumber(uint16_t& paramNum); diff --git a/modules/platforms/cpp/odbc/src/odbc.cpp b/modules/platforms/cpp/odbc/src/odbc.cpp index af8c73489902c..a85b694e9b3b8 100644 --- a/modules/platforms/cpp/odbc/src/odbc.cpp +++ b/modules/platforms/cpp/odbc/src/odbc.cpp @@ -898,11 +898,10 @@ namespace ignite if (paramCnt) { - uint16_t num = 0; + uint16_t paramNum = 0; + statement->GetParametersNumber(paramNum); - statement->GetParametersNumber(num); - - *paramCnt = static_cast(num); + *paramCnt = static_cast(paramNum); } return statement->GetDiagnosticRecords().GetReturnCode(); From 9ad513e68b89e907f7db36a3f3f0daca0e5986e6 Mon Sep 17 00:00:00 2001 From: Igor Sapego Date: Fri, 23 Jun 2017 17:06:40 +0300 Subject: [PATCH 150/357] IGNITE-5478: Fix for cherry pick (cherry picked from commit a2a4ec1ee9794cb542f146a07c6c67002cad444e) --- modules/platforms/cpp/odbc-test/src/queries_test.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/platforms/cpp/odbc-test/src/queries_test.cpp b/modules/platforms/cpp/odbc-test/src/queries_test.cpp index 082def8ea0290..ee470b3f7a114 100644 --- a/modules/platforms/cpp/odbc-test/src/queries_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/queries_test.cpp @@ -1792,7 +1792,7 @@ BOOST_AUTO_TEST_CASE(TestTablesMeta) BOOST_AUTO_TEST_CASE(TestParamsNum) { - Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;SCHEMA=cache"); + Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache"); CheckParamsNum("SELECT * FROM TestType", 0); CheckParamsNum("SELECT * FROM TestType WHERE _key=?", 1); From bfec212b1ece0e9e791de6dfb642324834fa77ca Mon Sep 17 00:00:00 2001 From: AMRepo Date: Fri, 23 Jun 2017 00:24:57 +0300 Subject: [PATCH 151/357] 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 152/357] 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 f59007f2c1221d8dd22abb0a9692b4abb31e87ad Mon Sep 17 00:00:00 2001 From: AMRepo Date: Fri, 23 Jun 2017 00:24:57 +0300 Subject: [PATCH 153/357] Partially reverted GG-12352. --- .../service/GridServiceProcessor.java | 2 +- .../spi/IgniteSpiOperationTimeoutHelper.java | 4 +-- .../tcp/TcpCommunicationSpi.java | 2 -- .../ignite/spi/discovery/tcp/ServerImpl.java | 16 ++---------- .../apache/ignite/spi/GridTcpForwarder.java | 26 +++++++++++++++++++ .../tcp/TcpCommunicationSpiDropNodesTest.java | 1 - .../TcpCommunicationSpiFaultyClientTest.java | 5 ---- .../discovery/tcp/TcpDiscoverySelfTest.java | 1 + 8 files changed, 32 insertions(+), 25 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 4c9f8e800465a..5c7a299b2bd4f 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 @@ -1659,7 +1659,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 269a98525cd42..10d2141ffdf15 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; @@ -2918,7 +2917,6 @@ protected GridCommunicationClient createTcpClient(ClusterNode node, int connIdx) } 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 80a0f18359cdf..410a351586768 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 @@ -1147,8 +1147,6 @@ else if (U.currentTimeMillis() - noResStart > spi.joinTimeout) boolean openSock = false; - boolean wasHandshake = false; - Socket sock = null; try { @@ -1166,8 +1164,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; @@ -1247,7 +1243,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); @@ -1255,21 +1251,13 @@ 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; 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 71a758024b6a8..3315c17046af5 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 c18a89f04e041..b0e543df495c1 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; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java index 043208c955b44..cd1776ca2bc1c 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java @@ -219,6 +219,7 @@ else if (gridName.contains("testNodeShutdownOnRingMessageWorkerFailureFailedNode return cfg; } + /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { discoMap = null; From 3e2a8dd7497cc21f9ad176213bcefa35869eb198 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Mon, 26 Jun 2017 12:27:58 +0300 Subject: [PATCH 154/357] Minor fix for GG-12197 "Ignore events for discarded update in CLOCK mode". --- .../processors/cache/GridCacheMapEntry.java | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 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 6ae2a7cc3a77a..bfa20d9bed139 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 @@ -2280,13 +2280,14 @@ else if (ttl != CU.TTL_ZERO) if (updateCntr != null) updateCntr0 = updateCntr; - cctx.continuousQueries().skipUpdateEvent( - lsnrs, - key, - partition(), - updateCntr0, - primary, - topVer); + if (lsnrs != null) + cctx.continuousQueries().skipUpdateEvent( + lsnrs, + key, + partition(), + updateCntr0, + primary, + topVer); } return new GridCacheUpdateAtomicResult(false, From 03e507f5021f3aee9728e5cbcbc85ed731e5e538 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Mon, 26 Jun 2017 19:37:27 +0300 Subject: [PATCH 155/357] Fixed tests. --- .../ignite/internal/IgniteClientReconnectAbstractTest.java | 5 +++++ ...iteTxCacheWriteSynchronizationModesMultithreadedTest.java | 5 +++++ 2 files changed, 10 insertions(+) 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..a793760f44fbf 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 @@ -69,6 +69,9 @@ public abstract class IgniteClientReconnectAbstractTest extends GridCommonAbstra /** */ private static final long RECONNECT_TIMEOUT = 10_000; + /** Reconnect should occurs before failure detection time is out. */ + public static final long FAILURE_DETECTION_TIMEOUT = RECONNECT_TIMEOUT +2000; + /** */ protected boolean clientMode; @@ -76,6 +79,8 @@ public abstract class IgniteClientReconnectAbstractTest extends GridCommonAbstra @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(gridName); + cfg.setFailureDetectionTimeout(FAILURE_DETECTION_TIMEOUT); + TestTcpDiscoverySpi disco = new TestTcpDiscoverySpi(); disco.setIpFinder(ipFinder); 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 ad6add47b29caf44dadfe2e32acd4d245ee256ab Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Tue, 27 Jun 2017 10:34:42 +0300 Subject: [PATCH 156/357] Fixed tests. --- ...IgniteBinaryMetadataUpdateNodeRestartTest.java | 10 ++++++++++ .../IgniteCacheNearRestartRollbackSelfTest.java | 15 +++++++++++++++ 2 files changed, 25 insertions(+) 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. From 897f4c00c4945eda3f9f4a41d064ded2f6f27ccc Mon Sep 17 00:00:00 2001 From: sboikov Date: Tue, 27 Jun 2017 12:55:11 +0300 Subject: [PATCH 157/357] 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 158/357] 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 3f33a902ed0d0a3e27be548209fe8e7933da57a9 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Mon, 26 Jun 2017 12:27:58 +0300 Subject: [PATCH 159/357] Minor fix for GG-12197 "Ignore events for discarded update in CLOCK mode". --- .../processors/cache/GridCacheMapEntry.java | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 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 6ae2a7cc3a77a..bfa20d9bed139 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 @@ -2280,13 +2280,14 @@ else if (ttl != CU.TTL_ZERO) if (updateCntr != null) updateCntr0 = updateCntr; - cctx.continuousQueries().skipUpdateEvent( - lsnrs, - key, - partition(), - updateCntr0, - primary, - topVer); + if (lsnrs != null) + cctx.continuousQueries().skipUpdateEvent( + lsnrs, + key, + partition(), + updateCntr0, + primary, + topVer); } return new GridCacheUpdateAtomicResult(false, From 3668b91817c1da7958cc3d7c4dddf890a2237772 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Tue, 27 Jun 2017 10:34:42 +0300 Subject: [PATCH 160/357] Partially reverted GG-12352. --- .../service/GridServiceProcessor.java | 2 +- .../spi/IgniteSpiOperationTimeoutHelper.java | 4 +-- .../tcp/TcpCommunicationSpi.java | 2 -- .../ignite/spi/discovery/tcp/ServerImpl.java | 16 ++---------- .../IgniteClientReconnectAbstractTest.java | 5 ++++ ...teBinaryMetadataUpdateNodeRestartTest.java | 10 +++++++ ...gniteCacheNearRestartRollbackSelfTest.java | 15 +++++++++++ ...SynchronizationModesMultithreadedTest.java | 5 ++++ .../apache/ignite/spi/GridTcpForwarder.java | 26 +++++++++++++++++++ .../tcp/TcpCommunicationSpiDropNodesTest.java | 1 - .../TcpCommunicationSpiFaultyClientTest.java | 5 ---- .../discovery/tcp/TcpDiscoverySelfTest.java | 1 + 12 files changed, 67 insertions(+), 25 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 4c9f8e800465a..5c7a299b2bd4f 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 @@ -1659,7 +1659,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 269a98525cd42..10d2141ffdf15 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; @@ -2918,7 +2917,6 @@ protected GridCommunicationClient createTcpClient(ClusterNode node, int connIdx) } 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 80a0f18359cdf..410a351586768 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 @@ -1147,8 +1147,6 @@ else if (U.currentTimeMillis() - noResStart > spi.joinTimeout) boolean openSock = false; - boolean wasHandshake = false; - Socket sock = null; try { @@ -1166,8 +1164,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; @@ -1247,7 +1243,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); @@ -1255,21 +1251,13 @@ 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; 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/internal/IgniteClientReconnectAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAbstractTest.java index 4d49366e1ad23..a793760f44fbf 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 @@ -69,6 +69,9 @@ public abstract class IgniteClientReconnectAbstractTest extends GridCommonAbstra /** */ private static final long RECONNECT_TIMEOUT = 10_000; + /** Reconnect should occurs before failure detection time is out. */ + public static final long FAILURE_DETECTION_TIMEOUT = RECONNECT_TIMEOUT +2000; + /** */ protected boolean clientMode; @@ -76,6 +79,8 @@ public abstract class IgniteClientReconnectAbstractTest extends GridCommonAbstra @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(gridName); + cfg.setFailureDetectionTimeout(FAILURE_DETECTION_TIMEOUT); + TestTcpDiscoverySpi disco = new TestTcpDiscoverySpi(); disco.setIpFinder(ipFinder); 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(); } 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 71a758024b6a8..3315c17046af5 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 c18a89f04e041..b0e543df495c1 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; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java index 043208c955b44..cd1776ca2bc1c 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java @@ -219,6 +219,7 @@ else if (gridName.contains("testNodeShutdownOnRingMessageWorkerFailureFailedNode return cfg; } + /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { discoMap = null; From 172f41c489c0ca5b7613163cd325b0c01d5b28b1 Mon Sep 17 00:00:00 2001 From: sboikov Date: Tue, 27 Jun 2017 12:55:11 +0300 Subject: [PATCH 161/357] gg-12133 Deadlock for metadata update from GridCacheMapEntry.innerUpdate (cherry picked from commit 897f4c0) --- .../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 ++++++++++++++++++ ...adlockDetectionMessageMarshallingTest.java | 2 +- .../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 +- 31 files changed, 290 insertions(+), 62 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 06dcb8664445c..2cb4c3eebe1a7 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 @@ -322,7 +322,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) @@ -1509,7 +1509,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) @@ -2334,14 +2334,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); } /** @@ -2441,7 +2441,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 406da2d284b63..a80cfbe2555d8 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 @@ -977,7 +977,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 1ba0102408eb0..8b029465fbf30 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 @@ -2475,7 +2475,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; Throwable err = null; 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 4717e8724d038..c4989a218df34 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) { @@ -722,7 +722,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 4490a6854d79b..21a0b7f2037d3 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 @@ -155,7 +155,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/internal/processors/cache/transactions/TxDeadlockDetectionMessageMarshallingTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionMessageMarshallingTest.java index eafd09f2fdbf9..f32148a272cbf 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionMessageMarshallingTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionMessageMarshallingTest.java @@ -76,7 +76,7 @@ public void testMessageUnmarshallWithoutCacheContext() throws Exception { final AtomicBoolean res = new AtomicBoolean(); clientCtx.gridIO().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 TxLocksResponse) { try { ((TxLocksResponse)msg).finishUnmarshal(clientCtx, clientCtx.deploy().globalLoader()); 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 ee9976c22846a..cbfe1de8735aa 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 @@ -189,7 +189,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 0521b7780756788d92bfa35ef00f56b5bb01367d Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Tue, 27 Jun 2017 20:43:07 +0300 Subject: [PATCH 162/357] GG-12370: Fixed massive NODE_FAILED events lead to excessive momentary memory consumption. --- .../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 07805f3f1b50b..4f5feaefc1554 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, evt, cache, null, msg); } } - else - exchangeFuture(msg.exchangeId(), null, null, null, null).onAffinityChangeMessage(customEvt.eventNode(), msg); + else if (msg.exchangeId().topologyVersion().topologyVersion() >= affinityTopologyVersion(cctx.discovery().localJoinEvent()).topologyVersion()) + exchangeFuture(msg.exchangeId(), null, 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 6abe5bf5bd732bf9f79df577e159243520dd5c0b Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Wed, 28 Jun 2017 09:47:45 +0300 Subject: [PATCH 163/357] Fixed compilation. --- .../internal/processors/hadoop/shuffle/HadoopShuffle.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffle.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffle.java index 8ffea8c75075d..81001a6ff0dec 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffle.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffle.java @@ -63,7 +63,7 @@ public class HadoopShuffle extends HadoopComponent { super.start(ctx); ctx.kernalContext().io().addMessageListener(GridTopic.TOPIC_HADOOP_MSG, new GridMessageListener() { - @Override public void onMessage(UUID nodeId, Object msg) { + @Override public void onMessage(UUID nodeId, Object msg, byte plc) { onMessageReceived(nodeId, (HadoopMessage)msg); } }); From 6b54f991c13efb3630527a3010c4f90105a4fc76 Mon Sep 17 00:00:00 2001 From: agura Date: Wed, 21 Jun 2017 19:09:37 +0300 Subject: [PATCH 164/357] ignite-5574 For offheap_tiered cache first try read value from offheap in order to avoid redundant synchronization on entry --- .../processors/cache/GridCacheAdapter.java | 14 +- .../cache/IgniteCacheNoSyncForGetTest.java | 163 ++++++++++++++++-- 2 files changed, 157 insertions(+), 20 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 c1b1a5cdfd4f6..f2c6410831482 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 @@ -820,7 +820,13 @@ public String toString() { if (nearKey && !ctx.isNear()) return null; - if (modes.heap) { + if (modes.offheap || modes.swap) { + GridCacheSwapManager swapMgr = ctx.isNear() ? ctx.near().dht().context().swap() : ctx.swap(); + + cacheVal = swapMgr.readValue(cacheKey, modes.offheap, modes.swap); + } + + if (cacheVal == null && modes.heap) { GridCacheEntryEx e = nearKey ? peekEx(cacheKey) : (ctx.isNear() ? ctx.near().dht().peekEx(cacheKey) : peekEx(cacheKey)); @@ -831,12 +837,6 @@ public String toString() { modes.swap = false; } } - - if (modes.offheap || modes.swap) { - GridCacheSwapManager swapMgr = ctx.isNear() ? ctx.near().dht().context().swap() : ctx.swap(); - - cacheVal = swapMgr.readValue(cacheKey, modes.offheap, modes.swap); - } } else cacheVal = localCachePeek0(cacheKey, modes.heap, modes.offheap, modes.swap, plc); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNoSyncForGetTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNoSyncForGetTest.java index 3e624a3cf0391..f65ad11328fec 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNoSyncForGetTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNoSyncForGetTest.java @@ -34,6 +34,7 @@ import org.apache.ignite.cache.CacheEntry; import org.apache.ignite.cache.CacheEntryProcessor; import org.apache.ignite.cache.CacheMemoryMode; +import org.apache.ignite.cache.CachePeekMode; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteInternalFuture; @@ -108,7 +109,22 @@ public void testAtomicGetOffheap() throws Exception { for (boolean getAll0 : getAll) { for (boolean expiryPlc0 : cfgExpiryPlc) for (boolean withExpiryPlc0 : withExpiryPlc) - doGet(ATOMIC, OFFHEAP_TIERED, getAll0, expiryPlc0, withExpiryPlc0); + doGet(ATOMIC, OFFHEAP_TIERED, false, getAll0, expiryPlc0, withExpiryPlc0); + } + } + + /** + * @throws Exception If failed. + */ + public void testAtomicPeekOffheap() throws Exception { + boolean getAll[] = {true, false}; + boolean cfgExpiryPlc[] = {true, false}; + boolean withExpiryPlc[] = {true, false}; + + for (boolean getAll0 : getAll) { + for (boolean expiryPlc0 : cfgExpiryPlc) + for (boolean withExpiryPlc0 : withExpiryPlc) + doGet(ATOMIC, OFFHEAP_TIERED, true, getAll0, expiryPlc0, withExpiryPlc0); } } @@ -123,24 +139,40 @@ public void testTxGetOffheap() throws Exception { for (boolean getAll0 : getAll) { for (boolean expiryPlc0 : cfgExpiryPlc) for (boolean withExpiryPlc0 : withExpiryPlc) - doGet(TRANSACTIONAL, OFFHEAP_TIERED, getAll0, expiryPlc0, withExpiryPlc0); + doGet(TRANSACTIONAL, OFFHEAP_TIERED, false, getAll0, expiryPlc0, withExpiryPlc0); + } + } + + /** + * @throws Exception If failed. + */ + public void testTxPeekOffheap() throws Exception { + boolean getAll[] = {true, false}; + boolean cfgExpiryPlc[] = {true, false}; + boolean withExpiryPlc[] = {true, false}; + + for (boolean getAll0 : getAll) { + for (boolean expiryPlc0 : cfgExpiryPlc) + for (boolean withExpiryPlc0 : withExpiryPlc) + doGet(TRANSACTIONAL, OFFHEAP_TIERED, false, getAll0, expiryPlc0, withExpiryPlc0); } } /** * @param atomicityMode Cache atomicity mode. * @param memoryMode Cache memory mode. - * @param getAll Test getAll flag. + * @param all Test all flag. * @param cfgExpiryPlc Configured expiry policy flag. * @param withExpiryPlc Custom expiry policy flag. * @throws Exception If failed. */ private void doGet(CacheAtomicityMode atomicityMode, CacheMemoryMode memoryMode, - final boolean getAll, + final boolean peek, + final boolean all, final boolean cfgExpiryPlc, final boolean withExpiryPlc) throws Exception { - log.info("Test get [getAll=" + getAll + ", cfgExpiryPlc=" + cfgExpiryPlc + ']'); + log.info("Test get [all=" + all + ", cfgExpiryPlc=" + cfgExpiryPlc + ']'); Ignite srv = ignite(0); @@ -163,7 +195,7 @@ private void doGet(CacheAtomicityMode atomicityMode, IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { @Override public Void call() throws Exception { - if (getAll) + if (all) cache.invokeAll(data.keySet(), new HangEntryProcessor()); else cache.invoke(1, new HangEntryProcessor()); @@ -177,13 +209,19 @@ private void doGet(CacheAtomicityMode atomicityMode, assertTrue(wait); - if (getAll) { - assertEquals(data, client.compute().affinityCall(cache.getName(), 1, - new GetAllClosure(data.keySet(), cache.getName(), withExpiryPlc))); + if (all) { + IgniteCallable clo = peek ? + new PeekAllClosure(data.keySet(), cache.getName(), withExpiryPlc) : + new GetAllClosure(data.keySet(), cache.getName(), withExpiryPlc); + + assertEquals(data, client.compute().affinityCall(cache.getName(), 1, clo)); } else { - assertEquals(1, client.compute().affinityCall(cache.getName(), 1, - new GetClosure(1, cache.getName(), withExpiryPlc))); + IgniteCallable clo = peek ? + new PeekClosure(1, cache.getName(), withExpiryPlc) : + new GetClosure(1, cache.getName(), withExpiryPlc); + + assertEquals(1, client.compute().affinityCall(cache.getName(), 1, clo)); } hangLatch.countDown(); @@ -204,7 +242,7 @@ private void doGet(CacheAtomicityMode atomicityMode, IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { @Override public Void call() throws Exception { - if (getAll) + if (all) cache.invokeAll(data.keySet(), new HangEntryProcessor()); else cache.invoke(1, new HangEntryProcessor()); @@ -223,7 +261,7 @@ private void doGet(CacheAtomicityMode atomicityMode, if (withExpiryPlc) srvCache = srvCache.withExpiryPolicy(ModifiedExpiryPolicy.factoryOf(Duration.FIVE_MINUTES).create()); - if (getAll) { + if (all) { assertEquals(data, srvCache.getAll(data.keySet())); assertEquals(data.size(), srvCache.getEntries(data.keySet()).size()); } @@ -390,4 +428,103 @@ public static class GetAllClosure implements IgniteCallable { return vals; } } + + /** + * + */ + public static class PeekClosure implements IgniteCallable { + /** */ + @IgniteInstanceResource + private Ignite ignite; + + /** */ + private final int key; + + /** */ + private final String cacheName; + + /** */ + private final boolean withExpiryPlc; + + /** + * @param key Key. + * @param cacheName Cache name. + * @param withExpiryPlc Custom expiry policy flag. + */ + PeekClosure(int key, String cacheName, boolean withExpiryPlc) { + this.key = key; + this.cacheName = cacheName; + this.withExpiryPlc = withExpiryPlc; + } + + /** {@inheritDoc} */ + @Override public Object call() throws Exception { + IgniteCache cache = ignite.cache(cacheName); + + if (withExpiryPlc) + cache = cache.withExpiryPolicy(ModifiedExpiryPolicy.factoryOf(Duration.FIVE_MINUTES).create()); + + Object val = cache.localPeek(key, CachePeekMode.ALL); + + CacheEntry e = cache.getEntry(key); + + assertEquals(val, e.getValue()); + + return val; + } + } + + /** + * + */ + public static class PeekAllClosure implements IgniteCallable { + /** */ + @IgniteInstanceResource + private Ignite ignite; + + /** */ + private final Set keys; + + /** */ + private final String cacheName; + + /** */ + private final boolean withExpiryPlc; + + /** + * @param keys Keys. + * @param cacheName Cache name. + * @param withExpiryPlc Custom expiry policy flag. + */ + PeekAllClosure(Set keys, String cacheName, boolean withExpiryPlc) { + this.keys = keys; + this.cacheName = cacheName; + this.withExpiryPlc = withExpiryPlc; + } + + /** {@inheritDoc} */ + @Override public Object call() throws Exception { + IgniteCache cache = ignite.cache(cacheName); + + if (withExpiryPlc) + cache = cache.withExpiryPolicy(ModifiedExpiryPolicy.factoryOf(Duration.FIVE_MINUTES).create()); + + Map vals = new HashMap(); + + for (Object key : keys) + vals.put(key, cache.localPeek(key, CachePeekMode.ALL)); + + Collection entries = cache.getEntries(keys); + + assertEquals(vals.size(), entries.size()); + + for (CacheEntry entry : entries) { + Object val = vals.get(entry.getKey()); + + assertEquals(val, entry.getValue()); + } + + return vals; + } + } } From 7d5217260b293b7224340349b5e44792999600f3 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Wed, 28 Jun 2017 11:47:21 +0300 Subject: [PATCH 165/357] Rethrow handshake exceptions as it done for ConnectionException. --- .../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 10d2141ffdf15..0f910df798947 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 @@ -3009,7 +3009,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++; @@ -3036,7 +3036,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); @@ -3048,7 +3049,7 @@ else if (X.hasCause(e, SocketTimeoutException.class)) } } - if (X.hasCause(errs, ConnectException.class)) + if (X.hasCause(errs, ConnectException.class, HandshakeException.class)) throw errs; } @@ -3101,7 +3102,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(); @@ -3111,7 +3112,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(); @@ -3127,7 +3128,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; } @@ -3136,7 +3137,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); @@ -3222,7 +3223,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(); @@ -3260,7 +3261,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; @@ -3480,6 +3481,19 @@ private NodeIdMessage nodeIdMessage() { return S.toString(TcpCommunicationSpi.class, this); } + /** 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 81cdea40743131cac9dae49150c1038073595f7e Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Wed, 28 Jun 2017 13:00:30 +0300 Subject: [PATCH 166/357] Fixed tests. --- .../IgniteClientReconnectAbstractTest.java | 10 +++++----- ...iteClientReconnectContinuousProcessorTest.java | 15 +-------------- 2 files changed, 6 insertions(+), 19 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 a793760f44fbf..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; @@ -69,9 +70,6 @@ public abstract class IgniteClientReconnectAbstractTest extends GridCommonAbstra /** */ private static final long RECONNECT_TIMEOUT = 10_000; - /** Reconnect should occurs before failure detection time is out. */ - public static final long FAILURE_DETECTION_TIMEOUT = RECONNECT_TIMEOUT +2000; - /** */ protected boolean clientMode; @@ -79,8 +77,6 @@ public abstract class IgniteClientReconnectAbstractTest extends GridCommonAbstra @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(gridName); - cfg.setFailureDetectionTimeout(FAILURE_DETECTION_TIMEOUT); - TestTcpDiscoverySpi disco = new TestTcpDiscoverySpi(); disco.setIpFinder(ipFinder); @@ -157,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) @@ -177,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. From 473abcafca568c7c898b0b1ae91fe964084fdf43 Mon Sep 17 00:00:00 2001 From: agura Date: Wed, 28 Jun 2017 14:49:49 +0300 Subject: [PATCH 167/357] 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 4f383ae772631987c3f4ac29bb654b4811fbc407 Mon Sep 17 00:00:00 2001 From: agura Date: Wed, 28 Jun 2017 14:49:49 +0300 Subject: [PATCH 168/357] 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 4b11bb27d5caeb7ba7bc2929685d7c548f4c327d Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Wed, 28 Jun 2017 15:45:42 +0300 Subject: [PATCH 169/357] Tests fixed. --- .../cache/IgniteDynamicCacheStartSelfTest.java | 15 +++++++++++++++ 1 file changed, 15 insertions(+) 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 9b92dac5756619ed5218db269acdae1ef02ef8ae Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Wed, 28 Jun 2017 18:20:02 +0300 Subject: [PATCH 170/357] Tests fixed. --- .../ignite/spi/communication/tcp/TcpCommunicationSpi.java | 2 ++ 1 file changed, 2 insertions(+) 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 0f910df798947..122700e84785c 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; @@ -2917,6 +2918,7 @@ protected GridCommunicationClient createTcpClient(ClusterNode node, int connIdx) } if (failureDetectionTimeoutEnabled() && (e instanceof HandshakeTimeoutException || + X.hasCause(e, SocketException.class) || timeoutHelper.checkFailureTimeoutReached(e))) { String msg = "Handshake timed out (failure detection timeout is reached) " + From 0295a1a7ecb2ef57a917ddc9015ff8b71a6ddb14 Mon Sep 17 00:00:00 2001 From: mcherkasov Date: Fri, 23 Jun 2017 21:00:45 +0300 Subject: [PATCH 171/357] 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 16fed552fc8f91de550207eeebd5850e685960a8 Mon Sep 17 00:00:00 2001 From: AMRepo Date: Thu, 29 Jun 2017 10:49:34 +0300 Subject: [PATCH 172/357] Fix tests. --- ...PartitionedBackupNodeFailureRecoveryTest.java | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePartitionedBackupNodeFailureRecoveryTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePartitionedBackupNodeFailureRecoveryTest.java index 6654fd93a77a8..e1ef59c5b79be 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePartitionedBackupNodeFailureRecoveryTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePartitionedBackupNodeFailureRecoveryTest.java @@ -33,6 +33,8 @@ import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.processors.cache.IgniteCacheAbstractTest; +import org.apache.ignite.internal.util.typedef.PA; +import org.apache.ignite.testframework.GridTestUtils; import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.PRIMARY; import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC; @@ -148,16 +150,22 @@ public void testBackUpFail() throws Exception { IgniteEx backUp = startGrid(2); - IgniteCache cache3 = backUp.cache(null); + final IgniteCache cache3 = backUp.cache(null); lock.lock(); try { - Integer backUpVal = cache3.localPeek(finalKey); + boolean res = GridTestUtils.waitForCondition(new PA() { + @Override public boolean apply() { + Integer actl = cache3.localPeek(finalKey); - Integer exp = cntr.get(); + Integer exp = cntr.get(); - assertEquals(exp, backUpVal); + return exp.equals(actl); + } + }, 1000); + + assertTrue(res); } finally { lock.unlock(); From 65d93e48c264f4bfff0a94856fdfeb83375a3976 Mon Sep 17 00:00:00 2001 From: mcherkasov Date: Fri, 23 Jun 2017 21:00:45 +0300 Subject: [PATCH 173/357] 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 174/357] 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 175/357] 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 176/357] 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 177/357] 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 178/357] 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 179/357] 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 62876fefc8dbd334db9dc6741c2bc91b3409343d Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Fri, 30 Jun 2017 14:45:18 +0300 Subject: [PATCH 180/357] 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 24df237db6a75..aaba91e15d304 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -136,6 +136,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 122700e84785c..7c49a35b5c62a 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 @@ -349,6 +349,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() { @@ -2989,9 +2993,10 @@ protected GridCommunicationClient createTcpClient(ClusterNode node, int connIdx) 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' " + @@ -3041,8 +3046,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 75c442a8309d554f6894f30f63ed0b964e7a0aa1 Mon Sep 17 00:00:00 2001 From: Slava Koptilin Date: Mon, 26 Jun 2017 15:37:26 +0300 Subject: [PATCH 181/357] Backported 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 ab4d593375130..d28a99109b29a --- 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 551923e1e5139c484fcb716e7f10907fa71021d9 Mon Sep 17 00:00:00 2001 From: nikolay_tikhonov Date: Fri, 30 Jun 2017 14:38:54 +0300 Subject: [PATCH 182/357] Backported "IGNITE-5424 GridServiceProxy does not unwraps exception message from InvocationTargetException." --- .../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 d16a4c48dcf86..3a40b907f91b2 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; @@ -413,7 +414,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 7b5abf5d2f90d..7a80fd14a3d54 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"); + } + } + + } From 752b1368adbb5c77b5d5caca3c07a72decff5111 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Fri, 30 Jun 2017 14:45:18 +0300 Subject: [PATCH 183/357] 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 fd3f947a1b49144a5b11aaf3f1a1b7a28ba961fd Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Fri, 30 Jun 2017 18:36:50 +0300 Subject: [PATCH 184/357] 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 6ab152a85f70c05847823f65f8e095ab9eb6b1f7 Mon Sep 17 00:00:00 2001 From: sboikov Date: Wed, 19 Apr 2017 12:46:31 +0300 Subject: [PATCH 185/357] 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 549c486e959aa..12e197124a5da 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 1525c6cf2cb015289392eb54fec4029e9b53b438 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Fri, 30 Jun 2017 18:36:50 +0300 Subject: [PATCH 186/357] 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 187/357] 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 05fac7e8a0bf9e08cd758bed7bd35ec85b914592 Mon Sep 17 00:00:00 2001 From: Evgenii Zhuravlev Date: Wed, 19 Apr 2017 14:01:21 +0300 Subject: [PATCH 188/357] Backported IGNITE-4925 Fix IgniteCacheBinaryObjectsScanSelfTest.testScanNoClasses - Fixes #1750. (cherry picked from commit b47f29d) --- .../cache/IgniteCacheBinaryObjectsScanSelfTest.java | 11 +++++++++-- .../cache/IgniteCacheEntryListenerAbstractTest.java | 2 +- 2 files changed, 10 insertions(+), 3 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheBinaryObjectsScanSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheBinaryObjectsScanSelfTest.java index 77438821630b1..69cda8538e81c 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheBinaryObjectsScanSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheBinaryObjectsScanSelfTest.java @@ -17,10 +17,12 @@ package org.apache.ignite.internal.processors.cache; +import java.util.ArrayList; import java.util.List; import javax.cache.Cache; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; import org.apache.ignite.cache.query.ScanQuery; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; @@ -127,9 +129,14 @@ public void testScanNoClasses() throws Exception { assertEquals(PERSON_CLS_NAME, entry.getValue().getClass().getName()); } - entries = cache.query(new ScanQuery<>(1)).getAll(); + entries = new ArrayList<>(); - assertFalse(entries.isEmpty()); + int partCnt = client.affinity("testCache").partitions(); + + for (int i = 0; i < partCnt; i++) + entries.addAll(cache.query(new ScanQuery<>(i)).getAll()); + + assertEquals(100, entries.size()); for (Cache.Entry entry : entries) { assertEquals(PERSON_KEY_CLS_NAME, entry.getKey().getClass().getName()); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java index 1f58765b85364..1772ef95cc938 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java @@ -339,7 +339,7 @@ private void awaitLatch() { if (!eagerTtl()) { U.sleep(1100); - assertNull(primaryCache(key, cache.getName()).get(key(key))); + assertNull(primaryCache(key(key), cache.getName()).get(key(key))); evtsLatch.await(5000, MILLISECONDS); From ef0a874ceb5c8bfa53e16337f6fd1699afaf2a39 Mon Sep 17 00:00:00 2001 From: nikolay_tikhonov Date: Fri, 30 Jun 2017 20:39:01 +0300 Subject: [PATCH 189/357] 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 190/357] 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 50887fed508e03a8b7df32569afb6d84ab3eb670 Mon Sep 17 00:00:00 2001 From: Igor Sapego Date: Tue, 4 Jul 2017 20:01:01 +0300 Subject: [PATCH 191/357] IGNITE-5663: ODBC: Closing cursor do not reset prepared statement anymore --- .../cpp/odbc-test/src/queries_test.cpp | 62 +++++++++++++++++++ modules/platforms/cpp/odbc/src/statement.cpp | 3 - 2 files changed, 62 insertions(+), 3 deletions(-) diff --git a/modules/platforms/cpp/odbc-test/src/queries_test.cpp b/modules/platforms/cpp/odbc-test/src/queries_test.cpp index 9aca77de784ed..f0e7c76133ed9 100644 --- a/modules/platforms/cpp/odbc-test/src/queries_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/queries_test.cpp @@ -1754,4 +1754,66 @@ BOOST_AUTO_TEST_CASE(TestParamsNum) CheckParamsNum("INSERT INTO TestType(_key, strField) VALUES(?, ?)", 2); } +BOOST_AUTO_TEST_CASE(TestExecuteAfterCursorClose) +{ + TestType in(1, 2, 3, 4, "5", 6.0f, 7.0, true, Guid(8, 9), BinaryUtils::MakeDateGmt(1987, 6, 5), + BinaryUtils::MakeTimestampGmt(1998, 12, 27, 1, 2, 3, 456)); + + cache1.Put(1, in); + + Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;SCHEMA=cache"); + + int64_t key = 0; + char strField[1024] = { 0 }; + SQLLEN strFieldLen = 0; + + // Binding columns. + SQLRETURN ret = SQLBindCol(stmt, 1, SQL_C_SLONG, &key, 0, 0); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + // Binding columns. + ret = SQLBindCol(stmt, 2, SQL_C_CHAR, &strField, sizeof(strField), &strFieldLen); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + // Just selecting everything to make sure everything is OK + SQLCHAR selectReq[] = "SELECT _key, strField FROM TestType"; + + ret = SQLPrepare(stmt, selectReq, sizeof(selectReq)); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLExecute(stmt); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLFreeStmt(stmt, SQL_CLOSE); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLExecute(stmt); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLFetch(stmt); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECK_EQUAL(key, 1); + + BOOST_CHECK_EQUAL(std::string(strField, strFieldLen), "5"); + + ret = SQLFetch(stmt); + + BOOST_CHECK_EQUAL(ret, SQL_NO_DATA); +} + BOOST_AUTO_TEST_SUITE_END() diff --git a/modules/platforms/cpp/odbc/src/statement.cpp b/modules/platforms/cpp/odbc/src/statement.cpp index 6154f9126568e..78b8a1ef8a2da 100644 --- a/modules/platforms/cpp/odbc/src/statement.cpp +++ b/modules/platforms/cpp/odbc/src/statement.cpp @@ -684,9 +684,6 @@ namespace ignite SqlResult result = currentQuery->Close(); - if (result == SQL_RESULT_SUCCESS) - currentQuery.reset(); - return result; } From da290cee855ef45a90ad539515e039f2826a6c00 Mon Sep 17 00:00:00 2001 From: Igor Sapego Date: Wed, 5 Jul 2017 13:21:12 +0300 Subject: [PATCH 192/357] IGNITE-5663: Fix for test --- modules/platforms/cpp/odbc-test/src/queries_test.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/platforms/cpp/odbc-test/src/queries_test.cpp b/modules/platforms/cpp/odbc-test/src/queries_test.cpp index f0e7c76133ed9..4c0284f978ca9 100644 --- a/modules/platforms/cpp/odbc-test/src/queries_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/queries_test.cpp @@ -1761,7 +1761,7 @@ BOOST_AUTO_TEST_CASE(TestExecuteAfterCursorClose) cache1.Put(1, in); - Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;SCHEMA=cache"); + Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache"); int64_t key = 0; char strField[1024] = { 0 }; From 3536a58982e4c264bb72b2ccc1953049d2b5c67f Mon Sep 17 00:00:00 2001 From: Alexey Kukushkin Date: Wed, 5 Jul 2017 19:36:41 +0300 Subject: [PATCH 193/357] IGNITE-4901 Decrease logging level for DataStremer retry --- .../datastreamer/DataStreamProcessor.java | 4 +- .../datastreamer/DataStreamerImpl.java | 14 +- .../DataStreamerImplSelfTest.java | 121 +++++++++++++++++- 3 files changed, 131 insertions(+), 8 deletions(-) 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 6f35a52d5b673..dec228d520ec1 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 @@ -21,10 +21,10 @@ import java.util.UUID; import java.util.concurrent.DelayQueue; import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.IgniteException; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.managers.communication.GridIoManager; import org.apache.ignite.internal.managers.communication.GridMessageListener; import org.apache.ignite.internal.managers.deployment.GridDeployment; @@ -336,7 +336,7 @@ private void localUpdate(final UUID nodeId, AffinityTopologyVersion topVer = fut.topologyVersion(); if (!allowOverwrite && !topVer.equals(req.topologyVersion())) { - Exception err = new IgniteCheckedException( + Exception err = new ClusterTopologyCheckedException( "DataStreamer will retry data transfer at stable topology " + "[reqTop=" + req.topologyVersion() + ", topVer=" + topVer + ", node=remote]"); 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 515314eb08728..8978fbf34dffe 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 @@ -1774,10 +1774,16 @@ void onResponse(DataStreamerResponse res, UUID nodeId) { try { GridPeerDeployAware jobPda0 = jobPda; - err = new IgniteCheckedException("DataStreamer request failed [node=" + nodeId + "]", - (Throwable)U.unmarshal(ctx, - errBytes, - U.resolveClassLoader(jobPda0 != null ? jobPda0.classLoader() : null, ctx.config()))); + final Throwable cause = U.unmarshal( + ctx, + errBytes, + U.resolveClassLoader(jobPda0 != null ? jobPda0.classLoader() : null, ctx.config())); + + final String msg = "DataStreamer request failed [node=" + nodeId + "]"; + + err = cause instanceof ClusterTopologyCheckedException ? + new ClusterTopologyCheckedException(msg, cause) : + new IgniteCheckedException(msg, cause); } catch (IgniteCheckedException e) { f.onDone(null, new IgniteCheckedException("Failed to unmarshal response.", e)); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImplSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImplSelfTest.java index a6a9f5488cefc..b26ccf24526eb 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImplSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImplSelfTest.java @@ -18,6 +18,7 @@ package org.apache.ignite.internal.processors.datastreamer; import java.io.Serializable; +import java.io.StringWriter; import java.util.Map; import java.util.Random; import java.util.concurrent.Callable; @@ -26,17 +27,29 @@ import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteDataStreamer; +import org.apache.ignite.IgniteException; import org.apache.ignite.cache.CacheServerNotFoundException; +import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.managers.communication.GridIoMessage; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.G; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteFuture; +import org.apache.ignite.lang.IgniteInClosure; +import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; 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 org.apache.log4j.Appender; +import org.apache.log4j.Logger; +import org.apache.log4j.SimpleLayout; +import org.apache.log4j.WriterAppender; import static org.apache.ignite.cache.CacheMode.PARTITIONED; import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; @@ -51,12 +64,18 @@ public class DataStreamerImplSelfTest extends GridCommonAbstractTest { /** Number of keys to load via data streamer. */ private static final int KEYS_COUNT = 1000; + /** Next nodes after MAX_CACHE_COUNT start without cache */ + private static final int MAX_CACHE_COUNT = 4; + /** Started grid counter. */ private static int cnt; /** No nodes filter. */ private static volatile boolean noNodesFilter; + /** Indicates whether we need to make the topology stale */ + private static boolean needStaleTop = false; + /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { super.afterTest(); @@ -73,8 +92,9 @@ public class DataStreamerImplSelfTest extends GridCommonAbstractTest { cfg.setDiscoverySpi(discoSpi); - // Forth node goes without cache. - if (cnt < 4) + cfg.setCommunicationSpi(new StaleTopologyCommunicationSpi()); + + if (cnt < MAX_CACHE_COUNT) cfg.setCacheConfiguration(cacheConfiguration()); cnt++; @@ -232,6 +252,44 @@ public void testNoDataNodesOnFlush() throws Exception { } } + /** + * Cluster topology mismatch shall result in DataStreamer retrying cache update with the latest topology and + * no error logged to the console. + * + * @throws Exception if failed + */ + public void testRetryWhenTopologyMismatch() throws Exception { + final int KEY = 1; + final String VAL = "1"; + + cnt = 0; + + StringWriter logWriter = new StringWriter(); + Appender logAppender = new WriterAppender(new SimpleLayout(), logWriter); + + Logger.getRootLogger().addAppender(logAppender); + + startGrids(MAX_CACHE_COUNT - 1); // cache-enabled nodes + + try (Ignite ignite = startGrid(MAX_CACHE_COUNT); + IgniteDataStreamer streamer = ignite.dataStreamer(null)) { + + needStaleTop = true; // simulate stale topology for the next action + + streamer.addData(KEY, VAL); + } finally { + needStaleTop = false; + + logWriter.flush(); + + Logger.getRootLogger().removeAppender(logAppender); + + logAppender.close(); + } + + assertFalse(logWriter.toString().contains("DataStreamer will retry data transfer at stable topology")); + } + /** * Gets cache configuration. * @@ -284,4 +342,63 @@ public Integer val() { return obj instanceof TestObject && ((TestObject)obj).val == val; } } + + /** + * Simulate stale (not up-to-date) topology + */ + private static class StaleTopologyCommunicationSpi extends TcpCommunicationSpi { + /** {@inheritDoc} */ + @Override public void sendMessage(ClusterNode node, Message msg, IgniteInClosure ackC) { + // Send stale topology only in the first request to avoid indefinitely getting failures. + if (needStaleTop) { + if (msg instanceof GridIoMessage) { + GridIoMessage ioMsg = (GridIoMessage)msg; + + Message appMsg = ioMsg.message(); + + if (appMsg != null && appMsg instanceof DataStreamerRequest) { + DataStreamerRequest req = (DataStreamerRequest)appMsg; + + AffinityTopologyVersion validTop = req.topologyVersion(); + + // Simulate situation when a node did not receive the latest "node joined" topology update causing + // topology mismatch + AffinityTopologyVersion staleTop = new AffinityTopologyVersion( + validTop.topologyVersion() - 1, + validTop.minorTopologyVersion()); + + appMsg = new DataStreamerRequest( + req.requestId(), + req.responseTopicBytes(), + req.cacheName(), + req.updaterBytes(), + req.entries(), + req.ignoreDeploymentOwnership(), + req.skipStore(), + req.keepBinary(), + req.deploymentMode(), + req.sampleClassName(), + req.userVersion(), + req.participants(), + req.classLoaderId(), + req.forceLocalDeployment(), + staleTop); + + msg = new GridIoMessage( + GridTestUtils.getFieldValue(ioMsg, "plc"), + GridTestUtils.getFieldValue(ioMsg, "topic"), + GridTestUtils.getFieldValue(ioMsg, "topicOrd"), + appMsg, + GridTestUtils.getFieldValue(ioMsg, "ordered"), + ioMsg.timeout(), + ioMsg.skipOnTimeout()); + + needStaleTop = false; + } + } + } + + super.sendMessage(node, msg, ackC); + } + } } \ No newline at end of file From 6d3a3ff2d99697882232070e715928336a9180cd Mon Sep 17 00:00:00 2001 From: Alexey Kukushkin Date: Wed, 5 Jul 2017 20:05:02 +0300 Subject: [PATCH 194/357] Fixed merge conflicts --- .../cache/GridCachePartitionExchangeManager.java | 2 +- .../spi/communication/tcp/TcpCommunicationSpi.java | 13 ------------- 2 files changed, 1 insertion(+), 14 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 8ddbb59b540f3..c62ffd2aec79c 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,7 +269,7 @@ else if (customEvt.customMessage() instanceof CacheAffinityChangeMessage) { exchFut = exchangeFuture(exchId, evt, cache, null, msg); } } - elseif (msg.exchangeId().topologyVersion().topologyVersion() >= affinityTopologyVersion(cctx.discovery().localJoinEvent()).topologyVersion()) + else if (msg.exchangeId().topologyVersion().topologyVersion() >= affinityTopologyVersion(cctx.discovery().localJoinEvent()).topologyVersion()) exchangeFuture(msg.exchangeId(), null, null, null, null).onAffinityChangeMessage(customEvt.eventNode(), msg); } } 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 f96cd1d79f538..0d80f4778df36 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 @@ -3504,19 +3504,6 @@ private static class HandshakeException extends IgniteCheckedException { } } - /** 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 acfc400b22738fa46397d392f88d49614e687969 Mon Sep 17 00:00:00 2001 From: nikolay_tikhonov Date: Wed, 5 Jul 2017 20:42:48 +0300 Subject: [PATCH 195/357] Merge branch 'ignite-1.7.12' into ignite-1.9.4 Signed-off-by: nikolay_tikhonov --- .../spi/communication/tcp/TcpCommunicationSpi.java | 13 ------------- 1 file changed, 13 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 ed672e2fe4730..1776b01ff8da5 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 @@ -3504,19 +3504,6 @@ private static class HandshakeException extends IgniteCheckedException { } } - /** 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 8dea19ba41bb9eda16f47933b2c46a081116d5f7 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Thu, 6 Jul 2017 12:02:07 +0300 Subject: [PATCH 196/357] Minor fix. --- .../processors/cache/GridCachePartitionExchangeManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 8ddbb59b540f3..c62ffd2aec79c 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,7 +269,7 @@ else if (customEvt.customMessage() instanceof CacheAffinityChangeMessage) { exchFut = exchangeFuture(exchId, evt, cache, null, msg); } } - elseif (msg.exchangeId().topologyVersion().topologyVersion() >= affinityTopologyVersion(cctx.discovery().localJoinEvent()).topologyVersion()) + else if (msg.exchangeId().topologyVersion().topologyVersion() >= affinityTopologyVersion(cctx.discovery().localJoinEvent()).topologyVersion()) exchangeFuture(msg.exchangeId(), null, null, null, null).onAffinityChangeMessage(customEvt.eventNode(), msg); } } From f208f434f944196d531a1b51066dfe8d6394d739 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Thu, 6 Jul 2017 15:17:50 +0300 Subject: [PATCH 197/357] Test fixed "IGNITE-5390: Bug in IgniteCacheTxStoreSessionWriteBehindCoalescingTest." --- .../cache/IgniteCacheAbstractTest.java | 17 +++++++++ ...ientWriteBehindStoreNonCoalescingTest.java | 38 +++++++++++-------- 2 files changed, 39 insertions(+), 16 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractTest.java index 45b6e9d07e757..f9ae6c24dfdf9 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractTest.java @@ -152,6 +152,9 @@ protected CacheConfiguration cacheConfiguration(String gridName) throws Exceptio cfg.setReadThrough(true); cfg.setWriteThrough(true); cfg.setLoadPreviousValue(true); + + cfg.setWriteBehindEnabled(writeBehindEnabled()); + cfg.setWriteBehindCoalescing(writeBehindCoalescing()); } if (cacheMode() == PARTITIONED) @@ -167,6 +170,20 @@ protected Factory cacheStoreFactory() { return null; } + /** + * @return write behind enabled flag. + */ + protected boolean writeBehindEnabled() { + return false; + } + + /** + * @return write behind coalescing flag. + */ + protected boolean writeBehindCoalescing() { + return true; + } + /** * @return Cache loader factory. */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/IgnteCacheClientWriteBehindStoreNonCoalescingTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/IgnteCacheClientWriteBehindStoreNonCoalescingTest.java index 8ea109dd2edfd..297a1cca8e714 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/IgnteCacheClientWriteBehindStoreNonCoalescingTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/IgnteCacheClientWriteBehindStoreNonCoalescingTest.java @@ -19,9 +19,8 @@ import java.util.ArrayList; import java.util.Collection; -import java.util.HashSet; import java.util.Map; -import java.util.Set; +import java.util.Random; import javax.cache.Cache; import javax.cache.configuration.Factory; import javax.cache.processor.EntryProcessor; @@ -77,6 +76,18 @@ public class IgnteCacheClientWriteBehindStoreNonCoalescingTest extends IgniteCac return new TestIncrementStoreFactory(); } + /** {@inheritDoc} */ + @Override protected boolean writeBehindEnabled() { + return true; + } + + /** {@inheritDoc} */ + @Override protected boolean writeBehindCoalescing() { + return false; + } + + private static Random rnd = new Random(); + /** * @throws Exception If failed. */ @@ -88,35 +99,30 @@ public void testNonCoalescingIncrementing() throws Exception { assertEquals(cache.getConfiguration(CacheConfiguration.class).getCacheStoreFactory().getClass(), TestIncrementStoreFactory.class); - Set keys = new HashSet<>(); - - for (int i = 0; i < 1000; i++) { - keys.add(i); - + for (int i = 0; i < CacheConfiguration.DFLT_WRITE_BEHIND_FLUSH_SIZE * 2; i++) { cache.put(i, i); } Collection> futs = new ArrayList<>(); - for (int i = 0; i < 100; i++) - futs.add(updateKeys(cache, keys)); + for (int i = 0; i < 1000; i++) + futs.add(updateKey(cache)); for (IgniteFuture fut : futs) fut.get(); } /** - * Update specified keys in async mode. + * Update random key in async mode. * * @param cache Cache to use. - * @param keys Keys to update. * @return IgniteFuture. */ - private IgniteFuture updateKeys(IgniteCache cache, Set keys) { + private IgniteFuture updateKey(IgniteCache cache) { IgniteCache asyncCache = cache.withAsync(); // Using EntryProcessor.invokeAll to increment every value in place. - asyncCache.invokeAll(keys, new EntryProcessor() { + asyncCache.invoke(rnd.nextInt(100), new EntryProcessor() { @Override public Object process(MutableEntry entry, Object... arguments) throws EntryProcessorException { entry.setValue(entry.getValue() + 1); @@ -155,10 +161,10 @@ public static class TestIncrementStore extends CacheStoreAdapter /** {@inheritDoc} */ @Override public void write(Cache.Entry entry) { - Object oldValue = storeMap.put(entry.getKey(), entry.getValue()); + Object oldVal = storeMap.put(entry.getKey(), entry.getValue()); - if (oldValue instanceof Integer && entry.getValue() instanceof Integer) { - Integer oldInt = (Integer)oldValue; + if (oldVal != null) { + Integer oldInt = (Integer)oldVal; Integer newInt = (Integer)entry.getValue(); assertTrue( From 355a5283559c885f57c4557bba2c6d9170a9b5fc Mon Sep 17 00:00:00 2001 From: mcherkasov Date: Fri, 30 Jun 2017 20:23:55 +0300 Subject: [PATCH 198/357] 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 199/357] 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 9165a0f93b5173b543cc6b4fad5fde37bd215f91 Mon Sep 17 00:00:00 2001 From: Slava Koptilin Date: Fri, 7 Jul 2017 15:35:33 +0300 Subject: [PATCH 200/357] ignite-5562: assert statements were changed to the 'if' blocks --- .../spi/discovery/tcp/internal/TcpDiscoveryStatistics.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) 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 a69dbd9aa89aa..42d2ae6f8617e 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 @@ -437,7 +437,8 @@ public synchronized String maxProcessingTimeMessageClass() { * @param initTime Time socket was initialized in. */ public synchronized void onServerSocketInitialized(long initTime) { - assert initTime >= 0; + if (initTime < 0) + initTime = 0; if (maxSrvSockInitTime < initTime) maxSrvSockInitTime = initTime; @@ -449,7 +450,8 @@ public synchronized void onServerSocketInitialized(long initTime) { * @param initTime Time socket was initialized in. */ public synchronized void onClientSocketInitialized(long initTime) { - assert initTime >= 0; + if (initTime < 0) + initTime = 0; clientSockCreatedCnt++; From d9fc20a61d5ac0a6e63b26faa7fa0af753b2fa06 Mon Sep 17 00:00:00 2001 From: Dmitriy Govorukhin Date: Fri, 7 Apr 2017 14:28:22 +0300 Subject: [PATCH 201/357] 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 16067300c9124b79bfee42139eb881ae585c0914 Mon Sep 17 00:00:00 2001 From: Dmitriy Govorukhin Date: Fri, 7 Apr 2017 14:28:22 +0300 Subject: [PATCH 202/357] 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 c82e25d67a2f6825a27d26933199a436f6eabba2 Mon Sep 17 00:00:00 2001 From: Dmitriy Govorukhin Date: Fri, 7 Apr 2017 14:28:22 +0300 Subject: [PATCH 203/357] 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 204/357] 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 205/357] 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 206/357] 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 207/357] 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 c3e2eebeccbdc4bb3a7a0a70d09a8a7b63399c2c Mon Sep 17 00:00:00 2001 From: Evgenii Zhuravlev Date: Tue, 18 Jul 2017 18:50:48 +0300 Subject: [PATCH 208/357] IGNITE 5776 Add option to turn on filter reachable addresses in TcpCommunicationSpi --- .../tcp/TcpCommunicationSpi.java | 77 ++++++++++++++----- 1 file changed, 56 insertions(+), 21 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 0d80f4778df36..7a5f7c1f16f3a 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 @@ -313,6 +313,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter /** Default value for {@code TCP_NODELAY} socket option (value is true). */ public static final boolean DFLT_TCP_NODELAY = true; + /** Default value for {@code FILTER_REACHABLE_ADDRESSES} socket option (value is false). */ + public static final boolean DFLT_FILTER_REACHABLE_ADDRESSES = false; + /** Default received messages threshold for sending ack. */ public static final int DFLT_ACK_SND_THRESHOLD = 32; @@ -996,6 +999,9 @@ class ConnectClosure implements IgniteInClosure { /** {@code TCP_NODELAY} option value for created sockets. */ private boolean tcpNoDelay = DFLT_TCP_NODELAY; + /** {@code FILTER_REACHABLE_ADDRESSES} option value for created sockets. */ + private boolean filterReachableAddresses = DFLT_FILTER_REACHABLE_ADDRESSES; + /** Number of received messages after which acknowledgment is sent. */ private int ackSndThreshold = DFLT_ACK_SND_THRESHOLD; @@ -1515,6 +1521,33 @@ public void setTcpNoDelay(boolean tcpNoDelay) { return tcpNoDelay; } + /** + * Gets value for {@code FILTER_REACHABLE_ADDRESSES} socket option. + * + * @return {@code True} if needed to filter reachable addresses. + */ + public boolean isFilterReachableAddresses() { + return filterReachableAddresses; + } + + /** + * Setting this option to {@code true} enables filter for reachable + * addresses on creating tcp client. + *

    + * Usually its advised to set this value to {@code false}. + *

    + * If not provided, default value is {@link #DFLT_FILTER_REACHABLE_ADDRESSES}. + * + * @param filterReachableAddresses {@code True} to filter reachable addresses. + * @return {@code this} for chaining. + */ + @IgniteSpiConfiguration(optional = true) + public TcpCommunicationSpi setFilterReachableAddresses(boolean filterReachableAddresses) { + this.filterReachableAddresses = filterReachableAddresses; + + return this; + } + /** * Sets receive buffer size for sockets created or accepted by this SPI. *

    @@ -2780,35 +2813,37 @@ protected GridCommunicationClient createTcpClient(ClusterNode node, int connIdx) if (isExtAddrsExist) addrs.addAll(extAddrs); - Set allInetAddrs = U.newHashSet(addrs.size()); + if (filterReachableAddresses) { + Set allInetAddrs = U.newHashSet(addrs.size()); - for (InetSocketAddress addr : addrs) { - // Skip unresolved as addr.getAddress() can return null. - if(!addr.isUnresolved()) - 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); + List reachableInetAddrs = U.filterReachable(allInetAddrs); - if (reachableInetAddrs.size() < allInetAddrs.size()) { - LinkedHashSet addrs0 = U.newLinkedHashSet(addrs.size()); + if (reachableInetAddrs.size() < allInetAddrs.size()) { + LinkedHashSet addrs0 = U.newLinkedHashSet(addrs.size()); - List unreachableInetAddr = new ArrayList<>(allInetAddrs.size() - reachableInetAddrs.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); + else + unreachableInetAddr.add(addr); + } - addrs0.addAll(unreachableInetAddr); + addrs0.addAll(unreachableInetAddr); - addrs = addrs0; - } + addrs = addrs0; + } - if (log.isDebugEnabled()) - log.debug("Addresses to connect for node [rmtNode=" + node.id() + ", addrs=" + addrs.toString() + ']'); + if (log.isDebugEnabled()) + log.debug("Addresses to connect for node [rmtNode=" + node.id() + ", addrs=" + addrs.toString() + ']'); + } boolean conn = false; GridCommunicationClient client = null; From 97d3f42c1c95a6aafce1d0c300ccfe6708398c17 Mon Sep 17 00:00:00 2001 From: shtykh_roman Date: Wed, 7 Sep 2016 08:35:31 +0300 Subject: [PATCH 209/357] 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 c2062d52a227dda5afee560d80c3bb4dd2ce09eb Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Wed, 19 Jul 2017 08:41:46 +0300 Subject: [PATCH 210/357] Remove empty test_utils.cpp --- modules/platforms/cpp/core-test/src/test_utils.cpp | 0 1 file changed, 0 insertions(+), 0 deletions(-) delete mode 100644 modules/platforms/cpp/core-test/src/test_utils.cpp diff --git a/modules/platforms/cpp/core-test/src/test_utils.cpp b/modules/platforms/cpp/core-test/src/test_utils.cpp deleted file mode 100644 index e69de29bb2d1d..0000000000000 From 45cbba4853bab1ba4ffe2ea0d3add99a9d454aab Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Wed, 19 Jul 2017 10:44:04 +0300 Subject: [PATCH 211/357] IGNITE-5768 - Retry resolving class name from marshaller cache and .classname file. --- .../apache/ignite/IgniteSystemProperties.java | 3 + .../internal/MarshallerContextImpl.java | 61 ++++++++++++++----- 2 files changed, 50 insertions(+), 14 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 aaba91e15d304..d3c9a6b64c650 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -580,6 +580,9 @@ public final class IgniteSystemProperties { */ public static final String IGNITE_ENABLE_FORCIBLE_NODE_KILL = "IGNITE_ENABLE_FORCIBLE_NODE_KILL"; + /** Ignite marshaller cache reread pause. */ + public static final String IGNITE_MARSHALLER_CACHE_REREAD_PAUSE = "IGNITE_MARSHALLER_CACHE_REREAD_PAUSE"; + /** * Enforces singleton. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java index f3e368d859a04..dae0d4edad1d1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java @@ -38,6 +38,7 @@ import javax.cache.event.CacheEntryUpdatedListener; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; +import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.internal.processors.cache.CachePartialUpdateCheckedException; import org.apache.ignite.internal.processors.cache.GridCacheAdapter; import org.apache.ignite.internal.processors.cache.GridCacheTryPutFailedException; @@ -47,10 +48,18 @@ import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.plugin.PluginProvider; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_MARSHALLER_CACHE_REREAD_PAUSE; + /** * Marshaller context implementation. */ public class MarshallerContextImpl extends MarshallerContextAdapter { + /** Cache reread tries. */ + private static final int CACHE_REREAD_TRIES = 5; + + /** Cache reread pause. */ + private static final int CACHE_REREAD_PAUSE = IgniteSystemProperties.getInteger(IGNITE_MARSHALLER_CACHE_REREAD_PAUSE, 20); + /** */ private static final GridStripedLock fileLock = new GridStripedLock(32); @@ -193,35 +202,59 @@ public void onKernalStop() { throw new IllegalStateException("Failed to initialize marshaller context (grid is stopping)."); } - String clsName = cache0.getTopologySafe(id); + String clsName = null; + + for (int i = 0; i < CACHE_REREAD_TRIES; i++) { + clsName = cache0.getTopologySafe(id); + + if (clsName != null) + break; + + U.sleep(CACHE_REREAD_PAUSE); + } if (clsName == null) { String fileName = id + ".classname"; - Lock lock = fileLock(fileName); + // Class name may be not in the file yet. + for (int i = 0; i < 2; i++) { + Lock lock = fileLock(fileName); + + lock.lock(); - lock.lock(); + File file; - try { - File file = new File(workDir, fileName); + try { + file = new File(workDir, fileName); - try (FileInputStream in = new FileInputStream(file)) { - FileLock fileLock = fileLock(in.getChannel(), true); + try (FileInputStream in = new FileInputStream(file)) { + FileLock fileLock = fileLock(in.getChannel(), true); - assert fileLock != null : fileName; + assert fileLock != null : fileName; - try (BufferedReader reader = new BufferedReader(new InputStreamReader(in, StandardCharsets.UTF_8))) { - clsName = reader.readLine(); + try (BufferedReader reader = new BufferedReader(new InputStreamReader(in, StandardCharsets.UTF_8))) { + clsName = reader.readLine(); + } + } + catch (IOException ignore) { + // Will fail on last try. } } - catch (IOException e) { + finally { + lock.unlock(); + } + + if (clsName != null) + break; + + // Fail on second unsuccessful try. + if (i == 1) { throw new IgniteCheckedException("Class definition was not found " + "at marshaller cache and local file. " + "[id=" + id + ", file=" + file.getAbsolutePath() + ']'); } - } - finally { - lock.unlock(); + + U.sleep(20); } // Must explicitly put entry to cache to invoke other continuous queries. From f24969f7e908645444df622642967a5f7fd3db23 Mon Sep 17 00:00:00 2001 From: Evgenii Zhuravlev Date: Wed, 19 Jul 2017 19:30:07 +0300 Subject: [PATCH 212/357] 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 213/357] 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 a58688f6cc6c5b114dcdd1b2fde43b7e1e5e0732 Mon Sep 17 00:00:00 2001 From: vsisko Date: Wed, 26 Jul 2017 15:40:19 +0700 Subject: [PATCH 214/357] IGNITE-5781 Visor throws ClassCastException if cache store implementation is other than CacheJdbcPojoStore. (cherry picked from commit f3adb95) --- .../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 c87ad05d824e6..46d72c3e4e7ba 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 @@ -109,7 +109,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 5b8963e38254fcac3bdd255a95adf9f733ce08f2 Mon Sep 17 00:00:00 2001 From: vsisko Date: Wed, 26 Jul 2017 15:40:19 +0700 Subject: [PATCH 215/357] IGNITE-5781 Visor throws ClassCastException if cache store implementation is other than CacheJdbcPojoStore. (cherry picked from commit f3adb95) --- .../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 c87ad05d824e6..46d72c3e4e7ba 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 @@ -109,7 +109,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 b7d1fb25ceba20b82631bb2e926a0ad52bf19e9d Mon Sep 17 00:00:00 2001 From: sboikov Date: Thu, 20 Jul 2017 17:43:17 +0300 Subject: [PATCH 216/357] Do not process partition exchange messages in striped pool. (cherry picked from commit 3a33706) --- .../dht/preloader/GridDhtPartitionsAbstractMessage.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsAbstractMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsAbstractMessage.java index 6e69161cb6ff2..34521e5cd2714 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsAbstractMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsAbstractMessage.java @@ -65,6 +65,11 @@ protected GridDhtPartitionsAbstractMessage() { this.lastVer = lastVer; } + /** {@inheritDoc} */ + @Override public int partition() { + return Integer.MIN_VALUE; + } + /** {@inheritDoc} */ @Override public boolean addDeploymentInfo() { return false; From c338bb9f5ac8f34dccbac1f7058765c5ce4549a4 Mon Sep 17 00:00:00 2001 From: sboikov Date: Thu, 20 Jul 2017 17:49:55 +0300 Subject: [PATCH 217/357] Removed unnecessary discoCache.updateAlives. (cherry picked from commit 07a0698) --- .../dht/preloader/GridDhtPartitionsExchangeFuture.java | 2 -- 1 file changed, 2 deletions(-) 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 d3e3701936786..ba47ffd435e06 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 @@ -460,8 +460,6 @@ public void init() throws IgniteInterruptedCheckedException { assert !dummy && !forcePreload : this; try { - discoCache.updateAlives(cctx.discovery()); - srvNodes = new ArrayList<>(discoCache.serverNodes()); remaining.addAll(F.nodeIds(F.view(srvNodes, F.remoteNodes(cctx.localNodeId())))); From e7fbe8fba6bcd427bebf6ab6fa8746bc47b0d42f Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Mon, 31 Jul 2017 17:20:29 +0300 Subject: [PATCH 218/357] IGNITE-4800: Lucene query may fails with NPE. --- .../query/h2/opt/GridLuceneDirectory.java | 44 ++- .../query/h2/opt/GridLuceneFile.java | 78 +++- .../query/h2/opt/GridLuceneIndex.java | 3 +- .../query/h2/opt/GridLuceneInputStream.java | 27 +- .../query/h2/opt/GridLuceneOutputStream.java | 4 +- .../cache/GridCacheFullTextQuerySelfTest.java | 367 ++++++++++++++++++ ...CacheFullTextQueryNodeJoiningSelfTest.java | 4 + .../IgniteCacheQuerySelfTestSuite.java | 6 + 8 files changed, 497 insertions(+), 36 deletions(-) create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFullTextQuerySelfTest.java diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneDirectory.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneDirectory.java index 480922cbb53b4..44887180528f4 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneDirectory.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneDirectory.java @@ -25,7 +25,9 @@ import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicLong; +import org.apache.ignite.IgniteException; import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory; +import org.apache.ignite.internal.util.typedef.F; import org.apache.lucene.store.Directory; import org.apache.lucene.store.IndexInput; import org.apache.lucene.store.IndexOutput; @@ -66,10 +68,7 @@ public GridLuceneDirectory(GridUnsafeMemory mem) { // and the code below is resilient to map changes during the array population. Set fileNames = fileMap.keySet(); - List names = new ArrayList<>(fileNames.size()); - - for (String name : fileNames) - names.add(name); + List names = new ArrayList<>(fileNames); return names.toArray(new String[names.size()]); } @@ -115,21 +114,25 @@ public GridLuceneDirectory(GridUnsafeMemory mem) { @Override public void deleteFile(String name) throws IOException { ensureOpen(); - doDeleteFile(name); + doDeleteFile(name, false); } /** * Deletes file. * * @param name File name. + * @param onClose If on close directory; * @throws IOException If failed. */ - private void doDeleteFile(String name) throws IOException { + private void doDeleteFile(String name, boolean onClose) throws IOException { GridLuceneFile file = fileMap.remove(name); if (file != null) { file.delete(); + // All files should be closed when Directory is closing. + assert !onClose || !file.hasRefs() : "Possible memory leak, resource is not closed: " + file.toString(); + sizeInBytes.addAndGet(-file.getSizeInBytes()); } else @@ -142,7 +145,10 @@ private void doDeleteFile(String name) throws IOException { GridLuceneFile file = newRAMFile(); - GridLuceneFile existing = fileMap.remove(name); + // Lock for using in stream. Will be unlocked on stream closing. + file.lockRef(); + + GridLuceneFile existing = fileMap.put(name, file); if (existing != null) { sizeInBytes.addAndGet(-existing.getSizeInBytes()); @@ -150,8 +156,6 @@ private void doDeleteFile(String name) throws IOException { existing.delete(); } - fileMap.put(name, file); - return new GridLuceneOutputStream(file); } @@ -174,6 +178,16 @@ protected GridLuceneFile newRAMFile() { if (file == null) throw new FileNotFoundException(name); + // Lock for using in stream. Will be unlocked on stream closing. + file.lockRef(); + + if (!fileMap.containsKey(name)) { + // Unblock for deferred delete. + file.releaseRef(); + + throw new FileNotFoundException(name); + } + return new GridLuceneInputStream(name, file); } @@ -181,16 +195,24 @@ protected GridLuceneFile newRAMFile() { @Override public void close() { isOpen = false; + IgniteException errs = null; + for (String fileName : fileMap.keySet()) { try { - doDeleteFile(fileName); + doDeleteFile(fileName, true); } catch (IOException e) { - throw new IllegalStateException(e); + if (errs == null) + errs = new IgniteException("Error closing index directory."); + + errs.addSuppressed(e); } } assert fileMap.isEmpty(); + + if (errs != null && !F.isEmpty(errs.getSuppressed())) + throw errs; } /** diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneFile.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneFile.java index 3985f0987b74f..4191135fc17db 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneFile.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneFile.java @@ -17,22 +17,16 @@ package org.apache.ignite.internal.processors.query.h2.opt; -import java.io.Serializable; import java.util.Arrays; -import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; import static org.apache.ignite.internal.processors.query.h2.opt.GridLuceneOutputStream.BUFFER_SIZE; /** * Lucene file. */ -public class GridLuceneFile implements Serializable { - /** */ - private static final long serialVersionUID = 0L; - - /** */ - public static final AtomicInteger filesCnt = new AtomicInteger(); - +public class GridLuceneFile { /** */ private LongArray buffers = new LongArray(); @@ -45,6 +39,12 @@ public class GridLuceneFile implements Serializable { /** */ private volatile long sizeInBytes; + /** */ + private final AtomicLong refCnt = new AtomicLong(); + + /** */ + private final AtomicBoolean deleted = new AtomicBoolean(); + /** * File used as buffer, in no RAMDirectory * @@ -52,8 +52,6 @@ public class GridLuceneFile implements Serializable { */ GridLuceneFile(GridLuceneDirectory dir) { this.dir = dir; - - filesCnt.incrementAndGet(); } /** @@ -92,52 +90,90 @@ final long addBuffer() { return buf; } + /** + * Increment ref counter. + */ + void lockRef() { + refCnt.incrementAndGet(); + } + + /** + * Decrement ref counter. + */ + void releaseRef() { + refCnt.decrementAndGet(); + + deferredDelete(); + } + + /** + * Checks if there is file stream opened. + * + * @return {@code True} if file has external references. + */ + boolean hasRefs() { + long refs = refCnt.get(); + + assert refs >= 0; + + return refs != 0; + } + /** * Gets address of buffer. * * @param idx Index. * @return Pointer. */ - protected final synchronized long getBuffer(int idx) { + final synchronized long getBuffer(int idx) { return buffers.get(idx); } /** * @return Number of buffers. */ - protected final synchronized int numBuffers() { + final synchronized int numBuffers() { return buffers.size(); } /** - * Expert: allocate a new buffer. - * Subclasses can allocate differently. + * Expert: allocate a new buffer. Subclasses can allocate differently. * * @return allocated buffer. */ - protected long newBuffer() { + private long newBuffer() { return dir.memory().allocate(BUFFER_SIZE); } /** * Deletes file and deallocates memory.. */ - public synchronized void delete() { - if (buffers == null) + public void delete() { + if (!deleted.compareAndSet(false, true)) return; + deferredDelete(); + } + + /** + * Deferred delete. + */ + synchronized void deferredDelete() { + if (!deleted.get() || hasRefs()) + return; + + assert refCnt.get() == 0; + for (int i = 0; i < buffers.idx; i++) dir.memory().release(buffers.arr[i], BUFFER_SIZE); buffers = null; - - filesCnt.decrementAndGet(); } /** * @return Size in bytes. */ - public long getSizeInBytes() { + long getSizeInBytes() { return sizeInBytes; } diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneIndex.java index 716c9cb2c2af4..2242f1447d8f9 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneIndex.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneIndex.java @@ -290,7 +290,8 @@ public GridCloseableIterator> query(String qry, /** {@inheritDoc} */ @Override public void close() { U.closeQuiet(writer); - U.closeQuiet(dir); + + dir.close(); } /** diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneInputStream.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneInputStream.java index eda97f36f487b..7740a03f9eb54 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneInputStream.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneInputStream.java @@ -20,6 +20,7 @@ import java.io.EOFException; import java.io.IOException; import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory; +import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.IndexInput; import org.apache.lucene.store.IndexOutput; @@ -28,7 +29,7 @@ /** * A memory-resident {@link IndexInput} implementation. */ -public class GridLuceneInputStream extends IndexInput { +public class GridLuceneInputStream extends IndexInput implements Cloneable { /** */ private GridLuceneFile file; @@ -53,6 +54,11 @@ public class GridLuceneInputStream extends IndexInput { /** */ private final GridUnsafeMemory mem; + /** */ + private volatile boolean closed; + + /** */ + private boolean isClone; /** * Constructor. * @@ -80,7 +86,24 @@ public GridLuceneInputStream(String name, GridLuceneFile f) throws IOException { /** {@inheritDoc} */ @Override public void close() { - // nothing to do here + if (!isClone) { + closed = true; + + file.releaseRef(); + } + } + + /** {@inheritDoc} */ + @Override public IndexInput clone() { + GridLuceneInputStream clone = (GridLuceneInputStream) super.clone(); + + if(closed) + throw new AlreadyClosedException(toString()); + + clone.isClone = true; + + return clone; + } /** {@inheritDoc} */ diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneOutputStream.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneOutputStream.java index 8d3d79cd27770..81f2f60671477 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneOutputStream.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneOutputStream.java @@ -82,6 +82,8 @@ public void reset() { /** {@inheritDoc} */ @Override public void close() throws IOException { flush(); + + file.releaseRef(); } /** {@inheritDoc} */ @@ -174,7 +176,7 @@ private void setFileLength() { * * @return Bytes used. */ - public long sizeInBytes() { + long sizeInBytes() { return (long)file.numBuffers() * (long)BUFFER_SIZE; } diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFullTextQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFullTextQuerySelfTest.java new file mode 100644 index 0000000000000..a26d5c52d52d9 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFullTextQuerySelfTest.java @@ -0,0 +1,367 @@ +/* + * 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.ArrayList; +import java.util.Calendar; +import java.util.Collection; +import java.util.Date; +import java.util.HashSet; +import java.util.List; +import java.util.Random; +import java.util.Set; +import javax.cache.Cache; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.binary.BinaryObject; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.affinity.Affinity; +import org.apache.ignite.cache.query.Query; +import org.apache.ignite.cache.query.QueryCursor; +import org.apache.ignite.cache.query.TextQuery; +import org.apache.ignite.cache.query.annotations.QuerySqlField; +import org.apache.ignite.cache.query.annotations.QueryTextField; +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.lang.IgnitePredicate; +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.common.GridCommonAbstractTest; + +import static org.apache.ignite.cache.CacheMode.PARTITIONED; +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; + +/** + * FullTest queries left test. + */ +public class GridCacheFullTextQuerySelfTest extends GridCommonAbstractTest { + /** Cache size. */ + private static final int MAX_ITEM_COUNT = 100; + + /** Cache name */ + private static final String PERSON_CACHE = "Person"; + + /** */ + private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + TcpDiscoverySpi disco = new TcpDiscoverySpi(); + + disco.setIpFinder(ipFinder); + + cfg.setDiscoverySpi(disco); + + cfg.setIncludeEventTypes(); + + cfg.setConnectorConfiguration(null); + + CacheConfiguration cacheCfg = defaultCacheConfiguration(); + + cacheCfg.setName(PERSON_CACHE) + .setCacheMode(PARTITIONED) + .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL) + .setWriteSynchronizationMode(FULL_SYNC) + .setBackups(0) + .setIndexedTypes(Integer.class, Person.class); + + cfg.setCacheConfiguration(cacheCfg); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + startGrids(2); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + } + + /** + * JUnit. + * + * @throws Exception In case of error. + */ + public void testLocalTextQueryWithKeepBinary() throws Exception { + checkTextQuery(true, true); + } + + /** + * JUnit. + * + * @throws Exception In case of error. + */ + public void testLocalTextQuery() throws Exception { + checkTextQuery(true, false); + } + + /** + * JUnit. + * + * @throws Exception In case of error. + */ + public void testTextQueryWithKeepBinary() throws Exception { + checkTextQuery(false, true); + } + + /** + * JUnit. + * + * @throws Exception In case of error. + */ + public void testTextQuery() throws Exception { + checkTextQuery(false, false); + } + + /** + * @param loc local query flag. + * @param keepBinary keep binary flag. + */ + private void checkTextQuery(boolean loc, boolean keepBinary) throws Exception { + final IgniteEx ignite = grid(0); + + // 1. Populate cache with data, calculating expected count in parallel. + Set exp = populateCache(ignite, loc, MAX_ITEM_COUNT, new IgnitePredicate() { + @Override + public boolean apply(Integer x) { + return String.valueOf(x).startsWith("1"); + } + }); + + // 2. Validate results. + TextQuery qry = new TextQuery<>(Person.class, "1*").setLocal(loc); + + validateQueryResults(ignite, qry, exp, keepBinary); + + clearCache(ignite); + } + + /** + * Clear cache with check. + */ + private static void clearCache(IgniteEx ignite) { + IgniteCache cache = ignite.cache(PERSON_CACHE); + + cache.clear(); + + List all = cache.query(new TextQuery<>(Person.class, "1*")).getAll(); + + assertTrue(all.isEmpty()); + } + + /** + * Fill cache. + * + * @throws IgniteCheckedException if failed. + */ + private static Set populateCache(IgniteEx ignite, boolean loc, int cnt, + IgnitePredicate expectedEntryFilter) throws IgniteCheckedException { + IgniteInternalCache cache = ignite.cachex(PERSON_CACHE); + + assertNotNull(cache); + + Random rand = new Random(); + + HashSet exp = new HashSet<>(); + + Affinity aff = cache.affinity(); + + ClusterNode localNode = cache.context().localNode(); + + for (int i = 0; i < cnt; i++) { + int val = rand.nextInt(cnt); + + cache.put(val, new Person(String.valueOf(val), val)); + + if (expectedEntryFilter.apply(val) && (!loc || aff.isPrimary(localNode, val))) + exp.add(val); + } + + return exp; + } + + /** + * Check query results. + * + * @throws IgniteCheckedException if failed. + */ + private static void validateQueryResults(IgniteEx ignite, Query qry, Set exp, + boolean keepBinary) throws IgniteCheckedException { + IgniteCache cache = ignite.cache(PERSON_CACHE); + + if (keepBinary) { + IgniteCache cache0 = cache.withKeepBinary(); + + try (QueryCursor> cursor = cache0.query(qry)) { + Set exp0 = new HashSet<>(exp); + + List> all = new ArrayList<>(); + + for (Cache.Entry entry : cursor.getAll()) { + all.add(entry); + + assertEquals(entry.getKey().toString(), entry.getValue().field("name")); + + assertEquals(entry.getKey(), entry.getValue().field("age")); + + exp0.remove(entry.getKey()); + } + + checkForMissedKeys(ignite, exp0, all); + } + + try (QueryCursor> cursor = cache0.query(qry)) { + Set exp0 = new HashSet<>(exp); + + List> all = new ArrayList<>(); + + for (Cache.Entry entry : cursor.getAll()) { + all.add(entry); + + assertEquals(entry.getKey().toString(), entry.getValue().field("name")); + + assertEquals(entry.getKey(), entry.getValue().field("age")); + + exp0.remove(entry.getKey()); + } + + checkForMissedKeys(ignite, exp0, all); + } + } + else { + try (QueryCursor> cursor = cache.query(qry)) { + Set exp0 = new HashSet<>(exp); + + List> all = new ArrayList<>(); + + for (Cache.Entry entry : cursor.getAll()) { + all.add(entry); + + assertEquals(entry.getKey().toString(), entry.getValue().name); + + assertEquals(entry.getKey(), Integer.valueOf(entry.getValue().age)); + + exp0.remove(entry.getKey()); + } + + checkForMissedKeys(ignite, exp0, all); + } + + try (QueryCursor> cursor = cache.query(qry)) { + Set exp0 = new HashSet<>(exp); + + List> all = new ArrayList<>(); + + for (Cache.Entry entry : cursor.getAll()) { + all.add(entry); + + assertEquals(entry.getKey().toString(), entry.getValue().name); + + assertEquals(entry.getKey().intValue(), entry.getValue().age); + + exp0.remove(entry.getKey()); + } + + checkForMissedKeys(ignite, exp0, all); + } + } + } + + /** + * Check if there is missed keys. + * + * @throws IgniteCheckedException if failed. + */ + private static void checkForMissedKeys(IgniteEx ignite, Collection exp, + List> all) throws IgniteCheckedException { + if (exp.size() == 0) + return; + + IgniteInternalCache cache = ignite.cachex(PERSON_CACHE); + + assertNotNull(cache); + + StringBuilder sb = new StringBuilder(); + + Affinity aff = cache.affinity(); + + for (Integer key : exp) { + Integer part = aff.partition(key); + + sb.append( + String.format("Query did not return expected key '%d' (exists: %s), partition '%d', partition nodes: ", + key, cache.get(key) != null, part)); + + Collection partNodes = aff.mapPartitionToPrimaryAndBackups(part); + + for (ClusterNode node : partNodes) + sb.append(node).append(" "); + + sb.append(";\n"); + } + + sb.append("Returned keys: "); + + for (Cache.Entry e : all) + sb.append(e.getKey()).append(" "); + + sb.append(";\n"); + + fail(sb.toString()); + } + + /** + * Test model class. + */ + public static class Person implements Serializable { + /** */ + @QueryTextField + String name; + + /** */ + @QuerySqlField(index = true) + int age; + + /** */ + @QuerySqlField final Date birthday; + + /** + * Constructor + */ + public Person(String name, int age) { + this.name = name; + this.age = age % 2000; + + Calendar cal = Calendar.getInstance(); + cal.add(Calendar.YEAR, -age); + + birthday = cal.getTime(); + } + } +} \ No newline at end of file diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheFullTextQueryNodeJoiningSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheFullTextQueryNodeJoiningSelfTest.java index 5921ba0832f4e..8809631b3d8bf 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheFullTextQueryNodeJoiningSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheFullTextQueryNodeJoiningSelfTest.java @@ -117,6 +117,10 @@ public void testFullTextQueryNodeJoin() throws Exception { Ignite started = startGrid(GRID_CNT); + //TODO: remove next line when IGNITE-2229 issue will be fixed. + // see https://issues.apache.org/jira/browse/IGNITE-2229 + awaitPartitionMapExchange(); + for (int i = 0; i < 100; i++) { QueryCursor, IndexedEntity>> res = started.cache(null) .query(new TextQuery, IndexedEntity>(IndexedEntity.class, "indexed")); diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java index 4fb729d56c1ed..6ae26e2c9c096 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java +++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java @@ -33,6 +33,7 @@ import org.apache.ignite.internal.processors.cache.CacheReplicatedQueryMetricsLocalSelfTest; import org.apache.ignite.internal.processors.cache.CacheSqlQueryValueCopySelfTest; import org.apache.ignite.internal.processors.cache.GridCacheCrossCacheQuerySelfTest; +import org.apache.ignite.internal.processors.cache.GridCacheFullTextQuerySelfTest; import org.apache.ignite.internal.processors.cache.GridCacheQueryIndexDisabledSelfTest; import org.apache.ignite.internal.processors.cache.GridCacheQueryIndexingDisabledSelfTest; import org.apache.ignite.internal.processors.cache.GridCacheQueryInternalKeysSelfTest; @@ -53,6 +54,7 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinTest; import org.apache.ignite.internal.processors.cache.IgniteCacheDuplicateEntityConfigurationSelfTest; import org.apache.ignite.internal.processors.cache.IgniteCacheFieldsQueryNoDataSelfTest; +import org.apache.ignite.internal.processors.cache.IgniteCacheFullTextQueryNodeJoiningSelfTest; import org.apache.ignite.internal.processors.cache.IgniteCacheInsertSqlQuerySelfTest; import org.apache.ignite.internal.processors.cache.IgniteCacheJoinPartitionedAndReplicatedTest; import org.apache.ignite.internal.processors.cache.IgniteCacheJoinQueryWithAffinityKeyTest; @@ -214,6 +216,10 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(CacheQueryOffheapEvictDataLostTest.class); + // Full text queries. + suite.addTestSuite(GridCacheFullTextQuerySelfTest.class); + suite.addTestSuite(IgniteCacheFullTextQueryNodeJoiningSelfTest.class); + // Ignite cache and H2 comparison. suite.addTestSuite(BaseH2CompareQueryTest.class); suite.addTestSuite(H2CompareBigQueryTest.class); From 8c992fb8ba33a0c0ac5c0fb741ee8ffd515c0f31 Mon Sep 17 00:00:00 2001 From: Evgenii Zhuravlev Date: Tue, 1 Aug 2017 17:46:27 +0300 Subject: [PATCH 219/357] 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 eb3471075035d4d6ed3f54b3c26e929a6a097262 Mon Sep 17 00:00:00 2001 From: Igor Sapego Date: Tue, 1 Aug 2017 18:30:54 +0300 Subject: [PATCH 220/357] Fix for C++ tests --- modules/platforms/cpp/odbc-test/src/queries_test.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/modules/platforms/cpp/odbc-test/src/queries_test.cpp b/modules/platforms/cpp/odbc-test/src/queries_test.cpp index a5793483ed8e9..e13d45c371b26 100644 --- a/modules/platforms/cpp/odbc-test/src/queries_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/queries_test.cpp @@ -1803,8 +1803,8 @@ BOOST_AUTO_TEST_CASE(TestParamsNum) BOOST_AUTO_TEST_CASE(TestExecuteAfterCursorClose) { - TestType in(1, 2, 3, 4, "5", 6.0f, 7.0, true, Guid(8, 9), BinaryUtils::MakeDateGmt(1987, 6, 5), - BinaryUtils::MakeTimestampGmt(1998, 12, 27, 1, 2, 3, 456)); + TestType in(1, 2, 3, 4, "5", 6.0f, 7.0, true, Guid(8, 9), common::MakeDateGmt(1987, 6, 5), + common::MakeTimestampGmt(1998, 12, 27, 1, 2, 3, 456)); cache1.Put(1, in); From 9f12a2d4bf333840df50d2a56478349f530b65c8 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Wed, 2 Aug 2017 14:23:26 +0300 Subject: [PATCH 221/357] IGNITE-4800: Test fixed. --- .../h2/GridIndexingSpiAbstractSelfTest.java | 157 +++++++++++------- 1 file changed, 93 insertions(+), 64 deletions(-) diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java index d6a5fb1f396e4..3d92e365ee90a 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java @@ -22,7 +22,6 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; -import java.util.Iterator; import java.util.List; import java.util.Map; import org.apache.ignite.IgniteCheckedException; @@ -35,6 +34,7 @@ import org.apache.ignite.internal.processors.query.GridQueryIndexType; import org.apache.ignite.internal.processors.query.GridQueryProperty; import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor; +import org.apache.ignite.internal.util.lang.GridCloseableIterator; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteBiTuple; @@ -42,6 +42,7 @@ import org.apache.ignite.plugin.extensions.communication.MessageWriter; import org.apache.ignite.spi.IgniteSpiCloseableIterator; import org.apache.ignite.spi.IgniteSpiException; +import org.apache.ignite.spi.indexing.IndexingSpi; import org.apache.ignite.testframework.GridStringLogger; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; @@ -110,7 +111,7 @@ protected void startIndexing(IgniteH2Indexing spi) throws Exception { * @param name Name. */ private CacheConfiguration cacheCfg(String name) { - CacheConfiguration cfg = new CacheConfiguration<>(); + CacheConfiguration cfg = new CacheConfiguration<>(); cfg.setName(name); @@ -230,16 +231,16 @@ public void testSpi() throws Exception { assertEquals(0, spi.size(typeAB.space(), typeAB)); assertEquals(0, spi.size(typeBA.space(), typeBA)); - assertFalse(spi.queryLocalSql(typeAA.space(), "select * from A.A", null, Collections.emptySet(), typeAA.name(), null, null).hasNext()); - assertFalse(spi.queryLocalSql(typeAB.space(), "select * from A.B", null, Collections.emptySet(), typeAB.name(), null, null).hasNext()); - assertFalse(spi.queryLocalSql(typeBA.space(), "select * from B.A", null, Collections.emptySet(), typeBA.name(), null, null).hasNext()); + assertFalse(hasLocalQueryResults(spi, typeAA.space(), "select * from A.A", typeAA.name())); + assertFalse(hasLocalQueryResults(spi, typeAB.space(), "select * from A.B", typeAB.name())); + assertFalse(hasLocalQueryResults(spi, typeBA.space(), "select * from B.A", typeBA.name())); - assertFalse(spi.queryLocalSql(typeBA.space(), "select * from B.A, A.B, A.A", null, - Collections.emptySet(), typeBA.name(), null, null).hasNext()); + assertFalse(hasLocalQueryResults(spi, typeBA.space(), "select * from B.A, A.B, A.A", typeBA.name())); - try { - spi.queryLocalSql(typeBA.space(), "select aa.*, ab.*, ba.* from A.A aa, A.B ab, B.A ba", null, - Collections.emptySet(), typeBA.name(), null, null).hasNext(); + try (GridCloseableIterator> res = + spi.queryLocalSql(typeBA.space(), "select aa.*, ab.*, ba.* from A.A aa, A.B ab, B.A ba", null, + Collections.emptySet(), typeBA.name(), null, null)) { + res.hasNext(); fail("Enumerations of aliases in select block must be prohibited"); } @@ -247,11 +248,9 @@ public void testSpi() throws Exception { // all fine } - assertFalse(spi.queryLocalSql(typeAB.space(), "select ab.* from A.B ab", null, - Collections.emptySet(), typeAB.name(), null, null).hasNext()); + assertFalse(hasLocalQueryResults(spi, typeAB.space(), "select ab.* from A.B ab", typeAB.name())); - assertFalse(spi.queryLocalSql(typeBA.space(), "select ba.* from B.A as ba", null, - Collections.emptySet(), typeBA.name(), null, null).hasNext()); + assertFalse(hasLocalQueryResults(spi, typeBA.space(), "select ba.* from B.A as ba", typeBA.name())); // Nothing to remove. spi.remove("A", key(1), aa(1, "", 10)); @@ -304,80 +303,92 @@ public void testSpi() throws Exception { assertEquals(1, spi.size(typeBA.space(), typeBA)); // Query data. - Iterator>> res = - spi.queryLocalSql(typeAA.space(), "from a order by age", null, Collections.emptySet(), typeAA.name(), null, null); - - assertTrue(res.hasNext()); - assertEquals(aa(3, "Borya", 18).value(null, false), value(res.next())); - assertTrue(res.hasNext()); - assertEquals(aa(2, "Valera", 19).value(null, false), value(res.next())); - assertFalse(res.hasNext()); + try (GridCloseableIterator>> res = + spi.queryLocalSql(typeAA.space(), "from a order by age", null, Collections.emptySet(), + typeAA.name(), null, null)) { - res = spi.queryLocalSql(typeAA.space(), "select aa.* from a aa order by aa.age", null, - Collections.emptySet(), typeAA.name(), null, null); + assertTrue(res.hasNext()); + assertEquals(aa(3, "Borya", 18).value(null, false), value(res.next())); + assertTrue(res.hasNext()); + assertEquals(aa(2, "Valera", 19).value(null, false), value(res.next())); + assertFalse(res.hasNext()); + } - assertTrue(res.hasNext()); - assertEquals(aa(3, "Borya", 18).value(null, false), value(res.next())); - assertTrue(res.hasNext()); - assertEquals(aa(2, "Valera", 19).value(null, false), value(res.next())); - assertFalse(res.hasNext()); + try (GridCloseableIterator>> res = + spi.queryLocalSql(typeAA.space(), "select aa.* from a aa order by aa.age", null, + Collections.emptySet(), typeAA.name(), null, null)) { - res = spi.queryLocalSql(typeAB.space(), "from b order by name", null, Collections.emptySet(), typeAB.name(), null, null); + assertTrue(res.hasNext()); + assertEquals(aa(3, "Borya", 18).value(null, false), value(res.next())); + assertTrue(res.hasNext()); + assertEquals(aa(2, "Valera", 19).value(null, false), value(res.next())); + assertFalse(res.hasNext()); + } - assertTrue(res.hasNext()); - assertEquals(ab(1, "Vasya", 20, "Some text about Vasya goes here.").value(null, false), value(res.next())); - assertTrue(res.hasNext()); - assertEquals(ab(4, "Vitalya", 20, "Very Good guy").value(null, false), value(res.next())); - assertFalse(res.hasNext()); + try (GridCloseableIterator>> res = + spi.queryLocalSql(typeAB.space(), "from b order by name", null, Collections.emptySet(), + typeAB.name(), null, null)) { - res = spi.queryLocalSql(typeAB.space(), "select bb.* from b as bb order by bb.name", null, - Collections.emptySet(), typeAB.name(), null, null); + assertTrue(res.hasNext()); + assertEquals(ab(1, "Vasya", 20, "Some text about Vasya goes here.").value(null, false), value(res.next())); + assertTrue(res.hasNext()); + assertEquals(ab(4, "Vitalya", 20, "Very Good guy").value(null, false), value(res.next())); + assertFalse(res.hasNext()); + } - assertTrue(res.hasNext()); - assertEquals(ab(1, "Vasya", 20, "Some text about Vasya goes here.").value(null, false), value(res.next())); - assertTrue(res.hasNext()); - assertEquals(ab(4, "Vitalya", 20, "Very Good guy").value(null, false), value(res.next())); - assertFalse(res.hasNext()); + try (GridCloseableIterator>> res = + spi.queryLocalSql(typeAB.space(), "select bb.* from b as bb order by bb.name", null, + Collections.emptySet(), typeAB.name(), null, null)) { + assertTrue(res.hasNext()); + assertEquals(ab(1, "Vasya", 20, "Some text about Vasya goes here.").value(null, false), value(res.next())); + assertTrue(res.hasNext()); + assertEquals(ab(4, "Vitalya", 20, "Very Good guy").value(null, false), value(res.next())); + assertFalse(res.hasNext()); + } - res = spi.queryLocalSql(typeBA.space(), "from a", null, Collections.emptySet(), typeBA.name(), null, null); + try (GridCloseableIterator>> res = + spi.queryLocalSql(typeBA.space(), "from a", null, Collections.emptySet(), typeBA.name(), null, null)) { - assertTrue(res.hasNext()); - assertEquals(ba(2, "Kolya", 25, true).value(null, false), value(res.next())); - assertFalse(res.hasNext()); + assertTrue(res.hasNext()); + assertEquals(ba(2, "Kolya", 25, true).value(null, false), value(res.next())); + assertFalse(res.hasNext()); + } // Text queries - Iterator>> txtRes = spi.queryLocalText(typeAB.space(), "good", - typeAB, null); + try (GridCloseableIterator>> txtRes = spi.queryLocalText(typeAB.space(), "good", + typeAB, null)) { - assertTrue(txtRes.hasNext()); - assertEquals(ab(4, "Vitalya", 20, "Very Good guy").value(null, false), value(txtRes.next())); - assertFalse(txtRes.hasNext()); + assertTrue(txtRes.hasNext()); + assertEquals(ab(4, "Vitalya", 20, "Very Good guy").value(null, false), value(txtRes.next())); + assertFalse(txtRes.hasNext()); + } // Fields query GridQueryFieldsResult fieldsRes = spi.queryLocalSqlFields("A", "select a.a.name n1, a.a.age a1, b.a.name n2, " + - "b.a.age a2 from a.a, b.a where a.a.id = b.a.id ", Collections.emptySet(), null, false, 0, null); + "b.a.age a2 from a.a, b.a where a.a.id = b.a.id ", Collections.emptySet(), null, false, 0, null); String[] aliases = {"N1", "A1", "N2", "A2"}; - Object[] vals = { "Valera", 19, "Kolya", 25}; + Object[] vals = {"Valera", 19, "Kolya", 25}; - IgniteSpiCloseableIterator> it = fieldsRes.iterator(); + try (IgniteSpiCloseableIterator> it = fieldsRes.iterator()) { - assertTrue(it.hasNext()); + assertTrue(it.hasNext()); - List fields = it.next(); + List fields = it.next(); - assertEquals(4, fields.size()); + assertEquals(4, fields.size()); - int i = 0; + int i = 0; - for (Object f : fields) { - assertEquals(aliases[i], fieldsRes.metaData().get(i).fieldName()); - assertEquals(vals[i++], f); - } + for (Object f : fields) { + assertEquals(aliases[i], fieldsRes.metaData().get(i).fieldName()); + assertEquals(vals[i++], f); + } - assertFalse(it.hasNext()); + assertFalse(it.hasNext()); + } // Remove spi.remove(typeAA.space(), key(2), aa(2, "Valera", 19)); @@ -430,6 +441,23 @@ public void testSpi() throws Exception { assertEquals(-1, spi.size(typeAA.space(), typeAA)); } + /** + * @param spi {@link IndexingSpi} implementation + * @param spaceName Space name. + * @param qry SQL query string. + * @param type Query return type. + * @return {@code False} if resultset is empty, {@code True} otherwise. + * @throws IgniteCheckedException + */ + private static boolean hasLocalQueryResults(IgniteH2Indexing spi, String spaceName, String qry, + String type) throws IgniteCheckedException { + + try (GridCloseableIterator> res = + spi.queryLocalSql(spaceName, qry, null, Collections.emptySet(), type, null, null)) { + return res.hasNext(); + } + } + /** * Test long queries write explain warnings into log. * @@ -610,7 +638,8 @@ String space() { /** {@inheritDoc} */ @SuppressWarnings("unchecked") - @Override public void setValue(String field, Object key, Object val, Object propVal) throws IgniteCheckedException { + @Override public void setValue(String field, Object key, Object val, + Object propVal) throws IgniteCheckedException { assert !F.isEmpty(field); assert key instanceof Integer; From f0f1c82c8f9877d952f639ffe56803043e53415b Mon Sep 17 00:00:00 2001 From: nikolay_tikhonov Date: Fri, 4 Aug 2017 14:58:52 +0300 Subject: [PATCH 222/357] 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 223/357] 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 9b9eabd97494f291444f42159cb5977b407dda8c Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Wed, 9 Aug 2017 18:58:02 +0700 Subject: [PATCH 224/357] IGNITE-5987 Added -nq (visor will not quit in batch mode) option for Visor Cmd. --- .../apache/ignite/visor/commands/VisorConsole.scala | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/VisorConsole.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/VisorConsole.scala index ad8c2edaaf8dd..31b4ff06b00dc 100644 --- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/VisorConsole.scala +++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/VisorConsole.scala @@ -93,6 +93,7 @@ class VisorConsole { println(" -cfg= - connect with specified configuration.") println(" -b= - batch mode with file.") println(" -e=cmd1;cmd2;... - batch mode with commands.") + println(" -nq - batch mode will not quit after execution (useful for alerts monitoring).") visor.quit() } @@ -104,6 +105,10 @@ class VisorConsole { val cfgFile = argValue("cfg", argLst) val batchFile = argValue("b", argLst) val batchCommand = argValue("e", argLst) + val noBatchQuit = hasArgName("nq", argLst) + + if (noBatchQuit && batchFile.isEmpty && batchCommand.isEmpty) + visor.warn("Option \"-nq\" will be ignored because batch mode options \"-b\" or \"-e\" were not specified.") cfgFile.foreach(cfg => { if (cfg.trim.isEmpty) { @@ -150,7 +155,10 @@ class VisorConsole { case Some(cmd) => visor.batchMode = true - new ByteArrayInputStream((cmd + "\nquit\n").getBytes("UTF-8")) + val script = if (noBatchQuit) cmd else cmd + "\nquit\n" + + new ByteArrayInputStream(script.getBytes("UTF-8")) + case None => new FileInputStream(FileDescriptor.in) } @@ -160,7 +168,7 @@ class VisorConsole { new TerminalSupport(false) {} } catch { - case ignored: ClassNotFoundException => null + case _: ClassNotFoundException => null } val reader = new ConsoleReader(inputStream, System.out, term) From 737260b070c3178d91ef04f7b200da87c9a1874f Mon Sep 17 00:00:00 2001 From: Evgenii Zhuravlev Date: Thu, 10 Aug 2017 18:54:57 +0300 Subject: [PATCH 225/357] 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 bad6c4cfe4c315b2418161e667a395176404d295 Mon Sep 17 00:00:00 2001 From: Evgenii Zhuravlev Date: Thu, 10 Aug 2017 18:54:57 +0300 Subject: [PATCH 226/357] 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 7506a9125351a..ed588c4d5e7b9 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 @@ -2257,7 +2257,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() + ']'); } @@ -2428,7 +2428,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 d4f4a323a80193dffabff96b45d0f44e5ee22d9b Mon Sep 17 00:00:00 2001 From: nikolay_tikhonov Date: Mon, 14 Aug 2017 16:44:51 +0300 Subject: [PATCH 227/357] Merge branch 'ignite-1.7.14' into ignite-1.8.10 Signed-off-by: nikolay_tikhonov --- .../CacheContinuousQueryConcurrentPartitionUpdateTest.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) 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 index c1675f328d706..36d1dc44aabf4 100644 --- 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 @@ -82,7 +82,7 @@ public class CacheContinuousQueryConcurrentPartitionUpdateTest extends GridCommo /** * @throws Exception If failed. */ - public void testConcurrentUpdatePartitionAtomic() throws Exception { + public void _testConcurrentUpdatePartitionAtomic() throws Exception { concurrentUpdatePartition(ATOMIC, false); } @@ -96,7 +96,7 @@ public void testConcurrentUpdatePartitionTx() throws Exception { /** * @throws Exception If failed. */ - public void testConcurrentUpdatePartitionAtomicClient() throws Exception { + public void _testConcurrentUpdatePartitionAtomicClient() throws Exception { concurrentUpdatePartition(ATOMIC, true); } @@ -205,7 +205,7 @@ private void concurrentUpdatePartition(CacheAtomicityMode atomicityMode, boolean /** * @throws Exception If failed. */ - public void testConcurrentUpdatesAndQueryStartAtomic() throws Exception { + public void _testConcurrentUpdatesAndQueryStartAtomic() throws Exception { concurrentUpdatesAndQueryStart(ATOMIC); } From b7bf1c09e8404d10eac57c13c9c6720c040d0c8e Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Fri, 14 Jul 2017 20:14:47 +0300 Subject: [PATCH 228/357] 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 b63c7911adedd0952edb5c2062a2dad8927976ac Mon Sep 17 00:00:00 2001 From: mcherkasov Date: Tue, 15 Aug 2017 17:36:02 +0300 Subject: [PATCH 229/357] Muted IgniteCacheMessageWriteTimeoutTest#testMessageQueueLimit test. --- .../cache/distributed/IgniteCacheMessageWriteTimeoutTest.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) 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 0dd4079733081..8f3f7723efb60 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 @@ -75,7 +75,9 @@ public class IgniteCacheMessageWriteTimeoutTest extends GridCommonAbstractTest { * @throws Exception If failed. */ public void testMessageQueueLimit() throws Exception { - for (int i = 0; i < 3; i++) { + fail("https://ggsystems.atlassian.net/browse/GG-12398"); + + for (int i = 0; i < 15; i++) { log.info("Iteration: " + i); startGridsMultiThreaded(3); From 60e2de794b52cc7fe3525c5e6de831106bcb9d5f Mon Sep 17 00:00:00 2001 From: mcherkasov Date: Wed, 16 Aug 2017 00:24:07 +0300 Subject: [PATCH 230/357] 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 928d445f63cf47ad52185b73ccfef559f22717e8 Mon Sep 17 00:00:00 2001 From: nikolay_tikhonov Date: Wed, 16 Aug 2017 14:06:37 +0300 Subject: [PATCH 231/357] Fixed flaky test "IgniteCacheEntryListener*" Signed-off-by: nikolay_tikhonov --- .../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 7cf7d0761db13..26c361498fbab 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 @@ -481,8 +481,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 aa5ca8fd154a853c37c2121d88a92e3404eead3a Mon Sep 17 00:00:00 2001 From: Nikolay Izhikov Date: Wed, 16 Aug 2017 15:45:16 +0300 Subject: [PATCH 232/357] IGNITE-5897 Fix session init/end logic. This fixes tests. Signed-off-by: nikolay_tikhonov --- .../cache/store/CacheStoreManager.java | 4 +++- .../store/GridCacheStoreManagerAdapter.java | 20 +++++++++---------- .../cache/transactions/IgniteTxAdapter.java | 9 +++++++-- 3 files changed, 19 insertions(+), 14 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java index 8d6b63dce3c8a..b096edf6a8b95 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java @@ -165,9 +165,11 @@ public boolean removeAll(@Nullable IgniteInternalTx tx, Collection keys) /** * @param tx Transaction. * @param commit Commit. + * @param last {@code True} if this is last store in transaction. + * @param storeSessionEnded {@code True} if session for underlying store already ended. * @throws IgniteCheckedException If failed. */ - public void sessionEnd(IgniteInternalTx tx, boolean commit, boolean last) throws IgniteCheckedException; + public void sessionEnd(IgniteInternalTx tx, boolean commit, boolean last, boolean storeSessionEnded) throws IgniteCheckedException; /** * End session initiated by write-behind store. 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 685e8f6e03181..14ec92237c3d4 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 @@ -777,7 +777,8 @@ private void loadAllFromStore(@Nullable IgniteInternalTx tx, } /** {@inheritDoc} */ - @Override public final void sessionEnd(IgniteInternalTx tx, boolean commit, boolean last) throws IgniteCheckedException { + @Override public final void sessionEnd(IgniteInternalTx tx, boolean commit, boolean last, + boolean storeSessionEnded) throws IgniteCheckedException { assert store != null; sessionInit0(tx); @@ -788,7 +789,7 @@ private void loadAllFromStore(@Nullable IgniteInternalTx tx, lsnr.onSessionEnd(locSes, commit); } - if (!sesHolder.get().ended(store)) + if (!sesHolder.get().ended(store) && !storeSessionEnded) store.sessionEnd(commit); } catch (Throwable e) { @@ -857,7 +858,7 @@ private void sessionInit0(@Nullable IgniteInternalTx tx) throws IgniteCheckedExc sesHolder.set(ses); try { - if (sesLsnrs != null && !ses.started(this)) { + if (!ses.started(store) && sesLsnrs != null) { for (CacheStoreSessionListener lsnr : sesLsnrs) lsnr.onSessionStart(locSes); } @@ -920,11 +921,8 @@ private static class SessionData { private Object attachment; /** */ - private final Set started = - new GridSetWrapper<>(new IdentityHashMap()); - - /** */ - private final Set ended = new GridSetWrapper<>(new IdentityHashMap()); + private final Set started = + new GridSetWrapper<>(new IdentityHashMap()); /** * @param tx Current transaction. @@ -987,8 +985,8 @@ private void cacheName(String cacheName) { /** * @return If session is started. */ - private boolean started(CacheStoreManager mgr) { - return !started.add(mgr); + private boolean started(CacheStore store) { + return !started.add(store); } /** @@ -996,7 +994,7 @@ private boolean started(CacheStoreManager mgr) { * @return Whether session already ended on this store instance. */ private boolean ended(CacheStore store) { - return !ended.add(store); + return !started.remove(store); } /** {@inheritDoc} */ 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..cd5babe7118b0 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 @@ -27,6 +27,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.IdentityHashMap; import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; @@ -40,6 +41,7 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.cache.CacheWriteSynchronizationMode; +import org.apache.ignite.cache.store.CacheStore; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; @@ -61,6 +63,7 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheVersionConflictContext; import org.apache.ignite.internal.processors.cache.version.GridCacheVersionedEntryEx; import org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException; +import org.apache.ignite.internal.util.GridSetWrapper; import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.lang.GridMetadataAwareAdapter; import org.apache.ignite.internal.util.lang.GridTuple; @@ -1238,13 +1241,15 @@ protected boolean isWriteToStoreFromDhtValid(Collection store * @param commit Commit flag. * @throws IgniteCheckedException In case of error. */ - protected void sessionEnd(Collection stores, boolean commit) throws IgniteCheckedException { + protected void sessionEnd(final Collection stores, boolean commit) throws IgniteCheckedException { Iterator it = stores.iterator(); + Set visited = new GridSetWrapper<>(new IdentityHashMap()); + while (it.hasNext()) { CacheStoreManager store = it.next(); - store.sessionEnd(this, commit, !it.hasNext()); + store.sessionEnd(this, commit, !it.hasNext(), !visited.add(store.store())); } } From fdd74de0c3635e12f74d70564bea9945776bfbca Mon Sep 17 00:00:00 2001 From: Alexander Fedotov Date: Wed, 16 Aug 2017 18:03:57 +0300 Subject: [PATCH 233/357] IGNITE-GG-12549 SQL: Add support for Java 8 Time API classes in date\time functions - add tests --- ...yWithJsr310Java8DateTimeApiFieldsTest.java | 409 ++++++++++++++++++ ...heQueryJsr310Java8DateTimeApiBaseTest.java | 71 +++ ...Jsr310Java8DateTimeApiKeyAndValueTest.java | 301 +++++++++++++ ...sr310Java8DateTimeApiSupportTestSuite.java | 40 ++ parent/pom.xml | 42 ++ 5 files changed, 863 insertions(+) create mode 100644 modules/indexing/src/test/java8/org/apache/ignite/internal/processors/query/h2/CacheQueryEntityWithJsr310Java8DateTimeApiFieldsTest.java create mode 100644 modules/indexing/src/test/java8/org/apache/ignite/internal/processors/query/h2/CacheQueryJsr310Java8DateTimeApiBaseTest.java create mode 100644 modules/indexing/src/test/java8/org/apache/ignite/internal/processors/query/h2/CacheQueryJsr310Java8DateTimeApiKeyAndValueTest.java create mode 100644 modules/indexing/src/test/java8/org/apache/ignite/testsuites/CacheQueryJsr310Java8DateTimeApiSupportTestSuite.java diff --git a/modules/indexing/src/test/java8/org/apache/ignite/internal/processors/query/h2/CacheQueryEntityWithJsr310Java8DateTimeApiFieldsTest.java b/modules/indexing/src/test/java8/org/apache/ignite/internal/processors/query/h2/CacheQueryEntityWithJsr310Java8DateTimeApiFieldsTest.java new file mode 100644 index 0000000000000..045fa3e40788a --- /dev/null +++ b/modules/indexing/src/test/java8/org/apache/ignite/internal/processors/query/h2/CacheQueryEntityWithJsr310Java8DateTimeApiFieldsTest.java @@ -0,0 +1,409 @@ +/* + * 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.query.h2; + +import java.io.Serializable; +import java.sql.Date; +import java.sql.Time; +import java.sql.Timestamp; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.util.List; +import java.util.Objects; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteException; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.cache.query.annotations.QuerySqlField; +import org.apache.ignite.cache.query.annotations.QuerySqlFunction; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.testframework.GridTestUtils; + +/** + * Tests queries against entities with JSR-310 Java 8 Date and Time API fields. + */ +public class CacheQueryEntityWithJsr310Java8DateTimeApiFieldsTest extends CacheQueryJsr310Java8DateTimeApiBaseTest { + /** + * Entity containing JSR-310 fields. + */ + private static class EntityWithJsr310Fields implements Serializable { + + /** Serial version UID. */ + private static final long serialVersionUID = 1L; + + /** ID. */ + @QuerySqlField(index = true) + private Long id; + + /** {@link LocalTime} field. */ + @QuerySqlField(index = true) + private LocalTime locTime; + + /** {@link LocalDate} field. */ + @QuerySqlField(index = true) + private LocalDate locDate; + + /** {@link LocalDateTime} field. */ + @QuerySqlField(index = true) + private LocalDateTime locDateTime; + + /** + * Default constructor. + */ + EntityWithJsr310Fields() { + } + + /** + * Copy constructor. + * + * @param entity Entity to copy from. + */ + EntityWithJsr310Fields(EntityWithJsr310Fields entity) { + id = entity.id; + locTime = LocalTime.from(entity.locTime); + locDate = LocalDate.from(entity.locDate); + locDateTime = LocalDateTime.from(entity.locDateTime); + } + + /** + * Constructor. + * + * @param id ID. + * @param locTime {@link LocalTime} value. + * @param locDate {@link LocalDate} value. + * @param locDateTime {@link LocalDateTime} value. + */ + EntityWithJsr310Fields(Long id, LocalTime locTime, LocalDate locDate, LocalDateTime locDateTime) { + this.id = id; + this.locTime = locTime; + this.locDate = locDate; + this.locDateTime = locDateTime; + } + + /** + * Returns the ID. + * + * @return ID. + */ + public Long getId() { + return id; + } + + /** + * Sets the ID. + * + * @param id ID. + */ + public void setId(Long id) { + this.id = id; + } + + /** + * Returns the {@link LocalDateTime} field value + * + * @return {@link LocalDateTime} field value; + */ + public LocalDateTime getLocalDateTime() { + return locDateTime; + } + + /** + * Returns the {@link LocalDateTime} field value. + * + * @param locDateTime {@link LocalDateTime} value. + */ + public void setLocalDateTime(LocalDateTime locDateTime) { + this.locDateTime = locDateTime; + } + + /** + * Returns the {@link LocalDate} field value. + * + * @return {@link LocalDate} field value. + */ + public LocalDate getLocalDate() { + return locDate; + } + + /** + * Sets the {@link LocalDate} field value. + * + * @param locDate {@link LocalDate} value. + */ + public void setLocalDate(LocalDate locDate) { + this.locDate = locDate; + } + + /** + * Returns the {@link LocalTime} field value. + * + * @return {@link LocalTime} field value. + */ + public LocalTime getLocalTime() { + return locTime; + } + + /** + * Sets the {@link LocalTime} field value. + * + * @param locTime {@link LocalTime} value. + */ + public void setLocalTime(LocalTime locTime) { + this.locTime = locTime; + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + + if (o == null || getClass() != o.getClass()) + return false; + + EntityWithJsr310Fields fields = (EntityWithJsr310Fields)o; + + return Objects.equals(id, fields.id) && Objects.equals(locDateTime, fields.locDateTime) && + Objects.equals(locDate, fields.locDate) && Objects.equals(locTime, fields.locTime); + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return Objects.hash(id, locDateTime, locDate, locTime); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return "EntityWithJsr310Fields{" + "id=" + id + ", locDateTime=" + locDateTime + ", locDate=" + locDate + + ", locTime=" + locTime + '}'; + } + } + + /** + * Custom SQL functions. + */ + public static class SqlFuncitons { + /** + * Converts the specified {@link LocalDateTime} object to a {@link Timestamp} object. + * + * @param localDateTime {@link LocalDateTime} object to convert. + * @return Converted object. + */ + @QuerySqlFunction(deterministic = true) + public static Timestamp localDateTimeToTimestamp(LocalDateTime localDateTime) { + return Timestamp.valueOf(localDateTime); + } + } + + /** Cache. */ + private IgniteCache cache; + + /** Entity with JSR-310 fields instance. */ + private final EntityWithJsr310Fields entity = + new EntityWithJsr310Fields(1L, LOCAL_TIME, LOCAL_DATE, LOCAL_DATE_TIME); + + /** + * Creates a cache configuration. + * + * @return Cache configuration. + */ + private static CacheConfiguration createCacheConfig() { + return CacheQueryEntityWithJsr310Java8DateTimeApiFieldsTest.createCacheConfig( + "entityWithJava8DataTimeFields", Long.class, EntityWithJsr310Fields.class + ).setSqlFunctionClasses(SqlFuncitons.class); + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + Ignite ignite = startGridsMultiThreaded(1, true); + cache = ignite.getOrCreateCache(createCacheConfig()); + + cache.put(entity.getId(), entity); + } + + /** + * Tests insertion of an entity. + * + * @throws Exception If failed. + */ + public void testInsertEntityFields() throws Exception { + cache.remove(entity.getId()); + + assertEquals(0, cache.size()); + + SqlFieldsQuery qry = new SqlFieldsQuery( + "insert into EntityWithJsr310Fields(_key, id, locTime, locDate, locDateTime) values(?, ?, ?, ?, ?)" + ).setArgs( + entity.getId(), entity.getId(), entity.getLocalTime(), entity.getLocalDate(), entity.getLocalDateTime() + ); + + List> qryResults = cache.query(qry).getAll(); + + assertEquals(1, qryResults.size()); + assertEquals(1L, qryResults.get(0).get(0)); + assertEquals(1, cache.size()); + assertEquals(entity, cache.get(entity.getId())); + } + + /** + * Tests selection of an entity. + * + * @throws Exception If failed. + */ + public void testSelectEntity() throws Exception { + SqlFieldsQuery qry = new SqlFieldsQuery("select _val from EntityWithJsr310Fields"); + + List> qryResults = cache.query(qry).getAll(); + + assertEquals(1, qryResults.size()); + assertEquals(entity, qryResults.get(0).get(0)); + } + + /** + * Tests that DATEDIFF SQL function fails for {@link LocalDateTime} + * fields with the time part set to midnight. + * + * @throws Exception If failed. + */ + public void testDateDiffForLocalDateTimeFieldAtMidnightThrows() throws Exception { + GridTestUtils.assertThrows( + log, + () -> { + SqlFieldsQuery qry = new SqlFieldsQuery( + "select DATEDIFF('DAY', locDateTime, CURRENT_DATE ()) from EntityWithJsr310Fields"); + + List> qryResults = cache.query(qry).getAll(); + + assertEquals(1, qryResults.size()); + assertTrue((Long)qryResults.get(0).get(0) >= DAYS_BEFORE_NOW); + + return null; + }, + IgniteException.class, null + ); + } + + /** + * Tests that DATEDIFF SQL function works for a {@link LocalDateTime} + * with the time part set to midnight field converted to + * a {@link Timestamp} object via a custom SQL function. + * + * @throws Exception If failed. + */ + public void testDateDiffForLocalDateTimeFieldAtMidnight() throws Exception { + SqlFieldsQuery qry = new SqlFieldsQuery( + "select DATEDIFF('DAY', localDateTimeToTimestamp(locDateTime), CURRENT_DATE ()) from EntityWithJsr310Fields"); + + List> qryResults = cache.query(qry).getAll(); + + assertEquals(1, qryResults.size()); + assertTrue((Long)qryResults.get(0).get(0) >= DAYS_BEFORE_NOW); + } + + /** + * Tests that selection of a {@link LocalTime} field. + * + * @throws Exception If failed. + */ + public void testSelectLocalTimeField() throws Exception { + SqlFieldsQuery qry = new SqlFieldsQuery("select locTime from EntityWithJsr310Fields"); + + List> qryResults = cache.query(qry).getAll(); + + assertEquals(1, qryResults.size()); + assertEquals(LocalTime.class, qryResults.get(0).get(0).getClass()); + } + + /** + * Tests that selection of a {@link LocalDate} field. + * + * @throws Exception If failed. + */ + public void testSelectLocalDateField() throws Exception { + SqlFieldsQuery qry = new SqlFieldsQuery("select locDate from EntityWithJsr310Fields"); + + List> qryResults = cache.query(qry).getAll(); + + assertEquals(1, qryResults.size()); + assertEquals(LocalDate.class, qryResults.get(0).get(0).getClass()); + } + + /** + * Tests that selection of a {@link LocalDateTime} field. + * + * @throws Exception If failed. + */ + public void testSelectLocalDateTimeField() throws Exception { + SqlFieldsQuery qry = new SqlFieldsQuery("select locDateTime from EntityWithJsr310Fields"); + + List> qryResults = cache.query(qry).getAll(); + + assertEquals(1, qryResults.size()); + assertEquals(LocalDateTime.class, qryResults.get(0).get(0).getClass()); + } + + /** + * Tests selection of an entity by a {@link LocalTime} field. + */ + public void testSelectByAllJsr310Fields() { + SqlFieldsQuery qry = new SqlFieldsQuery( + "select _val from EntityWithJsr310Fields where locTime = ? and locDate = ? and locDateTime = ?" + ).setArgs(entity.getLocalTime(), entity.getLocalDate(), entity.getLocalDateTime()); + + List> qryResults = cache.query(qry).getAll(); + + assertEquals(1, qryResults.size()); + assertEquals(entity, qryResults.get(0).get(0)); + } + + /** + * Tests updating of all JSR-310 fields. + */ + public void testUpdateAllJsr310Fields() { + EntityWithJsr310Fields expEntity = new EntityWithJsr310Fields(entity); + + expEntity.setLocalTime(expEntity.getLocalTime().plusHours(1)); + expEntity.setLocalDate(expEntity.getLocalDate().plusDays(1)); + expEntity.setLocalDateTime(LocalDateTime.of(expEntity.getLocalDate(), expEntity.getLocalTime())); + + SqlFieldsQuery qry = new SqlFieldsQuery( + "update EntityWithJsr310Fields set locTime = ?, locDate = ?, locDateTime = ? where id = ?" + ).setArgs(expEntity.getLocalTime(), expEntity.getLocalDate(), expEntity.getLocalDateTime(), entity.getId()); + + List> qryResults = cache.query(qry).getAll(); + + assertEquals(1, qryResults.size()); + assertEquals(1L, qryResults.get(0).get(0)); + assertEquals(expEntity, cache.get(expEntity.getId())); + } + + /** + * Tests deleting by all JSR-310 fields. + */ + public void testDeleteByAllJsr310Fields() { + SqlFieldsQuery qry = new SqlFieldsQuery( + "delete from EntityWithJsr310Fields where locTime = ? and locDate = ? and locDateTime = ?" + ).setArgs(entity.getLocalTime(), entity.getLocalDate(), entity.getLocalDateTime()); + + List> qryResults = cache.query(qry).getAll(); + + assertEquals(1, qryResults.size()); + assertEquals(1L, qryResults.get(0).get(0)); + assertEquals(0, cache.size()); + } +} diff --git a/modules/indexing/src/test/java8/org/apache/ignite/internal/processors/query/h2/CacheQueryJsr310Java8DateTimeApiBaseTest.java b/modules/indexing/src/test/java8/org/apache/ignite/internal/processors/query/h2/CacheQueryJsr310Java8DateTimeApiBaseTest.java new file mode 100644 index 0000000000000..a65d997e3532d --- /dev/null +++ b/modules/indexing/src/test/java8/org/apache/ignite/internal/processors/query/h2/CacheQueryJsr310Java8DateTimeApiBaseTest.java @@ -0,0 +1,71 @@ +/* + * 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.query.h2; + +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.CacheWriteSynchronizationMode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Base class for JSR-310 Java 8 Date and Time API queries tests. + */ +public abstract class CacheQueryJsr310Java8DateTimeApiBaseTest extends GridCommonAbstractTest { + /** {@link LocalTime} instance. */ + protected static final LocalTime LOCAL_TIME = LocalTime.now().minusHours(10); + + /** + * The number of days subtracted from the current time when constructing + * {@link LocalDate} and {@link LocalDateTime} + * instances. + */ + protected static final long DAYS_BEFORE_NOW = 10; + + /** {@link LocalDate} instance. */ + protected static final LocalDate LOCAL_DATE = LocalDate.now().minusDays(DAYS_BEFORE_NOW); + + /** {@link LocalDateTime} instance. */ + protected static final LocalDateTime LOCAL_DATE_TIME = LocalDateTime.of(LOCAL_DATE, LocalTime.MIDNIGHT); + + /** + * Creates a cache configuration with the specified cache name + * and indexed type key/value pairs. + * + * @param cacheName Cache name + * @param indexedTypes key/value pairs according to {@link CacheConfiguration#setIndexedTypes(Class[])}. + * @param Key type. + * @param Value type. + * @return Cache configuration. + */ + protected static CacheConfiguration createCacheConfig(String cacheName, Class... indexedTypes) { + return new CacheConfiguration(cacheName) + .setCacheMode(CacheMode.REPLICATED) + .setAtomicityMode(CacheAtomicityMode.ATOMIC) + .setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC) + .setIndexedTypes(indexedTypes); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } +} diff --git a/modules/indexing/src/test/java8/org/apache/ignite/internal/processors/query/h2/CacheQueryJsr310Java8DateTimeApiKeyAndValueTest.java b/modules/indexing/src/test/java8/org/apache/ignite/internal/processors/query/h2/CacheQueryJsr310Java8DateTimeApiKeyAndValueTest.java new file mode 100644 index 0000000000000..967598edf9425 --- /dev/null +++ b/modules/indexing/src/test/java8/org/apache/ignite/internal/processors/query/h2/CacheQueryJsr310Java8DateTimeApiKeyAndValueTest.java @@ -0,0 +1,301 @@ +/* + * 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.query.h2; + +import java.io.Serializable; +import java.sql.Date; +import java.sql.Time; +import java.sql.Timestamp; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.util.Collections; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.List; +import javax.cache.Cache; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheEntry; +import org.apache.ignite.cache.QueryEntity; +import org.apache.ignite.cache.query.ScanQuery; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.util.typedef.internal.A; + +/** + * Tests queries against JSR-310 Java 8 Date and Time API keys and values. + */ +public class CacheQueryJsr310Java8DateTimeApiKeyAndValueTest extends CacheQueryJsr310Java8DateTimeApiBaseTest { + /** Ignite instance. */ + private Ignite ignite; + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + ignite = startGridsMultiThreaded(1, true); + } + + /** + * Tests that the inserted key-value pair is stored as is. + * + * @param key Key. + * @param val Value. + * @param Key type. + * @param Value type + */ + private void doTestInsertedKeyValuePairStoredAsIs(K key, V val) { + A.notNull(key, "key"); + A.notNull(val, "val"); + + final Class keyCls = key.getClass(); + final Class valCls = val.getClass(); + + IgniteCache cache = + ignite.getOrCreateCache(createCacheConfig(valCls.getSimpleName() + "Cache", keyCls, valCls)); + + SqlFieldsQuery qry = + new SqlFieldsQuery("insert into " + valCls.getSimpleName() + "(_key, _val) values(?, ?)").setArgs(key, val); + + List> qryResults = cache.query(qry).getAll(); + + assertEquals(1, qryResults.size()); + assertEquals(1L, qryResults.get(0).get(0)); + assertEquals(1, cache.size()); + + CacheEntry entry = cache.getEntry(key); + List> all = cache.query(new ScanQuery<>()).getAll(); + + assertNotNull(entry); + assertEquals(key, entry.getKey()); + assertEquals(val, entry.getValue()); + assertEquals(keyCls, entry.getKey().getClass()); + assertEquals(valCls, entry.getValue().getClass()); + } + + /** + * Tests that selection by a key-value pair returns the expected key and value types. + * + * @param key Key. + * @param val Value. + * @param expKeyCls Expected key class. + * @param expValCls Expected value class. + * @param Key type. + * @param Value type. + */ + private void doTestSelectByKeyValuePairReturnsExpectedKeyAndValueTypes( + K key, V val, Class expKeyCls, Class expValCls + ) { + A.notNull(key, "key"); + A.notNull(val, "val"); + + final Class keyCls = key.getClass(); + final Class valCls = val.getClass(); + + IgniteCache cache = + ignite.getOrCreateCache(createCacheConfig(valCls.getSimpleName() + "Cache", keyCls, valCls)); + + cache.put(key, val); + + SqlFieldsQuery qry = + new SqlFieldsQuery("select _key, _val from " + valCls.getSimpleName() + " where _key=? and _val=?") + .setArgs(key, val); + + List> qryResults = cache.query(qry).getAll(); + + assertEquals(1, qryResults.size()); + assertEquals(expKeyCls, qryResults.get(0).get(0).getClass()); + assertEquals(expValCls, qryResults.get(0).get(1).getClass()); + } + + /** + * Tests updating a value by a key-value pair. + * + * @param key Key. + * @param val Value. + * @param newVal New value. + * @param Key type. + * @param Value type. + */ + private void doTestUpdateValueByKeyValuePair(K key, V val, V newVal) { + A.notNull(key, "key"); + A.notNull(val, "val"); + A.notNull(newVal, "newVal"); + + final Class keyCls = key.getClass(); + final Class valCls = val.getClass(); + + IgniteCache cache = + ignite.getOrCreateCache(createCacheConfig(valCls.getSimpleName() + "Cache", keyCls, valCls)); + + cache.put(key, val); + + SqlFieldsQuery qry = + new SqlFieldsQuery("update " + valCls.getSimpleName() + " set _val=? where _key=? and _val=?") + .setArgs(newVal, key, val); + + List> qryResults = cache.query(qry).getAll(); + + assertEquals(1, qryResults.size()); + assertEquals(1L, qryResults.get(0).get(0)); + assertEquals(newVal, cache.get(key)); + } + + /** + * Tests deleting by a key-value pair. + * + * @param key Key. + * @param val Value. + * @param Key type. + * @param Value type. + */ + private void doTestDeleteByKeyValuePair(K key, V val) { + A.notNull(key, "key"); + A.notNull(val, "val"); + + final Class keyCls = key.getClass(); + final Class valCls = val.getClass(); + + IgniteCache cache = + ignite.getOrCreateCache(createCacheConfig(valCls.getSimpleName() + "Cache", keyCls, valCls)); + + cache.put(key, val); + + SqlFieldsQuery qry =new SqlFieldsQuery("delete from " + valCls.getSimpleName() + " where _key=? and _val=?") + .setArgs(key, val); + + List> qryResults = cache.query(qry).getAll(); + + assertEquals(1, qryResults.size()); + assertEquals(1L, qryResults.get(0).get(0)); + assertEquals(0, cache.size()); + } + + /** + * Tests that the inserted {@link LocalTime} key-value pair is stored as is. + * + * @throws Exception If failed. + */ + public void testInsertedLocalTimeKeyValuePairStoredAsIs() throws Exception { + doTestInsertedKeyValuePairStoredAsIs(LOCAL_TIME, LOCAL_TIME); + } + + /** + * Tests that the inserted {@link LocalDate} key-value pair is stored as is. + * + * @throws Exception If failed. + */ + public void testInsertedLocalDateKeyValuePairStoredAsIs() throws Exception { + doTestInsertedKeyValuePairStoredAsIs(LOCAL_DATE, LOCAL_DATE); + } + + /** + * Tests that the inserted {@link LocalDateTime} key-value pair is stored as is. + * + * @throws Exception If failed. + */ + public void testInsertedLocalDateTimeKeyValuePairStoredAsIs() throws Exception { + doTestInsertedKeyValuePairStoredAsIs(LOCAL_DATE_TIME, LOCAL_DATE_TIME); + } + + /** + * Tests that selection by a {@link LocalTime} key-value pair returns {@link LocalTime} key and value types. + * + * @throws Exception If failed. + */ + public void testSelectByLocalTimeKeyValuePairReturnsLocalTimeKeyAndValue() throws Exception { + doTestSelectByKeyValuePairReturnsExpectedKeyAndValueTypes( + LOCAL_TIME, LOCAL_TIME, LocalTime.class, LocalTime.class + ); + } + + /** + * Tests that selection by a {@link LocalDate} key-value pair returns {@link LocalDate} key and value types. + * + * @throws Exception If failed. + */ + public void testSelectByLocalDateKeyValuePairReturnsLocalDateKeyAndValue() throws Exception { + doTestSelectByKeyValuePairReturnsExpectedKeyAndValueTypes( + LOCAL_DATE, LOCAL_DATE, LocalDate.class, LocalDate.class + ); + } + + /** + * Tests that selection by a {@link LocalDateTime} key-value pair returns {@link LocalDateTime} key and value types. + * + * @throws Exception If failed. + */ + public void testSelectByLocalDateTimeKeyValuePairReturnsLocalDateTimeKeyAndValue() throws Exception { + doTestSelectByKeyValuePairReturnsExpectedKeyAndValueTypes( + LOCAL_DATE_TIME, LOCAL_DATE_TIME, LocalDateTime.class, LocalDateTime.class + ); + } + + /** + * Tests updating a value by a {@link LocalTime} key-value pair. + * + * @throws Exception If failed. + */ + public void testUpdateValueByLocalTimeKeyValuePair() throws Exception { + doTestUpdateValueByKeyValuePair(LOCAL_TIME, LOCAL_TIME, LOCAL_TIME.plusHours(1)); + } + + /** + * Tests updating a value by a {@link LocalDate} key-value pair. + * + * @throws Exception If failed. + */ + public void testUpdateValueByLocalDateKeyValuePair() throws Exception { + doTestUpdateValueByKeyValuePair(LOCAL_DATE, LOCAL_DATE, LOCAL_DATE.plusDays(1)); + } + + /** + * Tests updating a value by a {@link LocalDateTime} key-value pair. + * + * @throws Exception If failed. + */ + public void testUpdateValueByLocalDateTimeKeyValuePair() throws Exception { + doTestUpdateValueByKeyValuePair(LOCAL_DATE_TIME, LOCAL_DATE_TIME, LOCAL_DATE_TIME.plusHours(1)); + } + + /** + * Test deleting by a {@link LocalTime} key-value pair. + * + * @throws Exception If failed. + */ + public void testDeleteByLocalTimeKeyValuePair() throws Exception { + doTestDeleteByKeyValuePair(LOCAL_TIME, LOCAL_TIME); + } + + /** + * Test deleting by a {@link LocalDate} key-value pair. + * + * @throws Exception If failed. + */ + public void testDeleteByLocalDateKeyValuePair() throws Exception { + doTestDeleteByKeyValuePair(LOCAL_DATE, LOCAL_DATE); + } + + /** + * Test deleting by a {@link LocalDateTime} key-value pair. + * + * @throws Exception If failed. + */ + public void testDeleteByLocalDateTimeKeyValuePair() throws Exception { + doTestDeleteByKeyValuePair(LOCAL_DATE_TIME, LOCAL_DATE_TIME); + } +} diff --git a/modules/indexing/src/test/java8/org/apache/ignite/testsuites/CacheQueryJsr310Java8DateTimeApiSupportTestSuite.java b/modules/indexing/src/test/java8/org/apache/ignite/testsuites/CacheQueryJsr310Java8DateTimeApiSupportTestSuite.java new file mode 100644 index 0000000000000..4c018368e4aa9 --- /dev/null +++ b/modules/indexing/src/test/java8/org/apache/ignite/testsuites/CacheQueryJsr310Java8DateTimeApiSupportTestSuite.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.testsuites; + +import junit.framework.TestSuite; +import org.apache.ignite.internal.processors.query.h2.CacheQueryEntityWithJsr310Java8DateTimeApiFieldsTest; +import org.apache.ignite.internal.processors.query.h2.CacheQueryJsr310Java8DateTimeApiKeyAndValueTest; + +/** + * Test suite for JSR-310 Java 8 Date and Time API queries. + */ +public class CacheQueryJsr310Java8DateTimeApiSupportTestSuite extends TestSuite { + /** + * @return Test suite. + * @throws Exception If failed. + */ + public static TestSuite suite() throws Exception { + TestSuite suite = new TestSuite("JSR-310 Java 8 Date and Time API Cache Queries Test Suite"); + + suite.addTestSuite(CacheQueryEntityWithJsr310Java8DateTimeApiFieldsTest.class); + suite.addTestSuite(CacheQueryJsr310Java8DateTimeApiKeyAndValueTest.class); + + return suite; + } +} diff --git a/parent/pom.xml b/parent/pom.xml index 59afbf20ad771..391415a957805 100644 --- a/parent/pom.xml +++ b/parent/pom.xml @@ -116,6 +116,10 @@ false false + + + ${project.build.sourceDirectory} + ${project.build.testSourceDirectory} org.apache.ignite @@ -546,6 +550,11 @@ + + org.codehaus.mojo + build-helper-maven-plugin + 3.0.0 + @@ -559,6 +568,37 @@ + + org.codehaus.mojo + build-helper-maven-plugin + + + add-sources + generate-sources + + add-source + + + + ${java8.folder} + + + + + add-tests + generate-test-sources + + add-test-source + + + + ${java8.test.folder} + + + + + + org.apache.maven.plugins maven-surefire-plugin @@ -938,6 +978,8 @@ -Xdoclint:none + ${project.build.sourceDirectory}/../java8 + ${project.build.testSourceDirectory}/../java8 From 4ee9e7da3187245fb34d87e2475b69bb5b474b0c Mon Sep 17 00:00:00 2001 From: Andrey Gura Date: Wed, 16 Aug 2017 18:00:31 +0300 Subject: [PATCH 234/357] 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 7acb5b429e693..fe146ea7ad191 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(); @@ -386,7 +413,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 f9c5bb0a00877df508c407f8daf0c350fd1bc99f Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Wed, 9 Aug 2017 18:58:02 +0700 Subject: [PATCH 235/357] IGNITE-5987 Added -nq (visor will not quit in batch mode) option for Visor Cmd. (cherry picked from commit 9b9eabd) --- .../apache/ignite/visor/commands/VisorConsole.scala | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/VisorConsole.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/VisorConsole.scala index ad8c2edaaf8dd..31b4ff06b00dc 100644 --- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/VisorConsole.scala +++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/VisorConsole.scala @@ -93,6 +93,7 @@ class VisorConsole { println(" -cfg= - connect with specified configuration.") println(" -b= - batch mode with file.") println(" -e=cmd1;cmd2;... - batch mode with commands.") + println(" -nq - batch mode will not quit after execution (useful for alerts monitoring).") visor.quit() } @@ -104,6 +105,10 @@ class VisorConsole { val cfgFile = argValue("cfg", argLst) val batchFile = argValue("b", argLst) val batchCommand = argValue("e", argLst) + val noBatchQuit = hasArgName("nq", argLst) + + if (noBatchQuit && batchFile.isEmpty && batchCommand.isEmpty) + visor.warn("Option \"-nq\" will be ignored because batch mode options \"-b\" or \"-e\" were not specified.") cfgFile.foreach(cfg => { if (cfg.trim.isEmpty) { @@ -150,7 +155,10 @@ class VisorConsole { case Some(cmd) => visor.batchMode = true - new ByteArrayInputStream((cmd + "\nquit\n").getBytes("UTF-8")) + val script = if (noBatchQuit) cmd else cmd + "\nquit\n" + + new ByteArrayInputStream(script.getBytes("UTF-8")) + case None => new FileInputStream(FileDescriptor.in) } @@ -160,7 +168,7 @@ class VisorConsole { new TerminalSupport(false) {} } catch { - case ignored: ClassNotFoundException => null + case _: ClassNotFoundException => null } val reader = new ConsoleReader(inputStream, System.out, term) From a76932f6f3d253a292803a0d1e954d42589bdbc6 Mon Sep 17 00:00:00 2001 From: Andrey Gura Date: Wed, 16 Aug 2017 18:00:31 +0300 Subject: [PATCH 236/357] 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 3e7849fbdfc60bfe6d24ab734672924293394f8b Mon Sep 17 00:00:00 2001 From: Andrey Gura Date: Wed, 16 Aug 2017 18:00:31 +0300 Subject: [PATCH 237/357] 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 7acb5b429e693..fe146ea7ad191 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(); @@ -386,7 +413,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 a1e51e652665d0c1d368bf73976b2b62bd3cb23d Mon Sep 17 00:00:00 2001 From: Evgenii Zhuravlev Date: Thu, 17 Aug 2017 16:15:31 +0300 Subject: [PATCH 238/357] IGNITE-6088 Socket#shutdownOutput in ServerImpl leads to UnsupportedOperationException on SSLSocket. Signed-off-by: nikolay_tikhonov --- .../java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) 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 4574b3816af8f..22cb618623644 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 @@ -57,6 +57,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import javax.net.ssl.SSLException; +import javax.net.ssl.SSLSocket; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; @@ -5804,7 +5805,8 @@ private class SocketReader extends IgniteSpiThread { spi.writeToSocket(sock, res, timeoutHelper.nextTimeoutChunk(spi.getSocketTimeout())); - sock.shutdownOutput(); + if (!(sock instanceof SSLSocket)) + sock.shutdownOutput(); if (log.isInfoEnabled()) log.info("Finished writing ping response " + "[rmtNodeId=" + msg.creatorNodeId() + From b1e0bdb5186dee6ebf65054b30674e5d33247fc1 Mon Sep 17 00:00:00 2001 From: mcherkasov Date: Wed, 16 Aug 2017 00:24:07 +0300 Subject: [PATCH 239/357] IgniteCacheNearRestartRollbackSelfTest#testRestarts is muted. (cherry picked from commit 60e2de7) Signed-off-by: nikolay_tikhonov --- .../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 d5c8e5f0ade84dafbd81d8592bbab08c099782b5 Mon Sep 17 00:00:00 2001 From: nikolay_tikhonov Date: Wed, 16 Aug 2017 14:06:37 +0300 Subject: [PATCH 240/357] Fixed flaky test "IgniteCacheEntryListener*" (cherry picked from commit 928d445) Signed-off-by: nikolay_tikhonov --- .../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 299a95f720e28..0dbf1f402240d 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 @@ -481,8 +481,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 15de7d4fda1e8ccd4102c82851973d3ed3e47a2e Mon Sep 17 00:00:00 2001 From: Nikolay Izhikov Date: Wed, 16 Aug 2017 15:45:16 +0300 Subject: [PATCH 241/357] IGNITE-5897 Fix session init/end logic. This fixes tests. Signed-off-by: nikolay_tikhonov --- .../cache/store/CacheStoreManager.java | 4 +++- .../store/GridCacheStoreManagerAdapter.java | 20 +++++++++---------- .../cache/transactions/IgniteTxAdapter.java | 9 +++++++-- 3 files changed, 19 insertions(+), 14 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java index 8d6b63dce3c8a..b096edf6a8b95 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java @@ -165,9 +165,11 @@ public boolean removeAll(@Nullable IgniteInternalTx tx, Collection keys) /** * @param tx Transaction. * @param commit Commit. + * @param last {@code True} if this is last store in transaction. + * @param storeSessionEnded {@code True} if session for underlying store already ended. * @throws IgniteCheckedException If failed. */ - public void sessionEnd(IgniteInternalTx tx, boolean commit, boolean last) throws IgniteCheckedException; + public void sessionEnd(IgniteInternalTx tx, boolean commit, boolean last, boolean storeSessionEnded) throws IgniteCheckedException; /** * End session initiated by write-behind store. 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 cebc5f1f62077..0753af4f1d2a2 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 @@ -787,7 +787,8 @@ private void loadAllFromStore(@Nullable IgniteInternalTx tx, } /** {@inheritDoc} */ - @Override public final void sessionEnd(IgniteInternalTx tx, boolean commit, boolean last) throws IgniteCheckedException { + @Override public final void sessionEnd(IgniteInternalTx tx, boolean commit, boolean last, + boolean storeSessionEnded) throws IgniteCheckedException { assert store != null; sessionInit0(tx); @@ -798,7 +799,7 @@ private void loadAllFromStore(@Nullable IgniteInternalTx tx, lsnr.onSessionEnd(locSes, commit); } - if (!sesHolder.get().ended(store)) + if (!sesHolder.get().ended(store) && !storeSessionEnded) store.sessionEnd(commit); } catch (Throwable e) { @@ -867,7 +868,7 @@ private void sessionInit0(@Nullable IgniteInternalTx tx) throws IgniteCheckedExc sesHolder.set(ses); try { - if (sesLsnrs != null && !ses.started(this)) { + if (!ses.started(store) && sesLsnrs != null) { for (CacheStoreSessionListener lsnr : sesLsnrs) lsnr.onSessionStart(locSes); } @@ -930,11 +931,8 @@ private static class SessionData { private Object attachment; /** */ - private final Set started = - new GridSetWrapper<>(new IdentityHashMap()); - - /** */ - private final Set ended = new GridSetWrapper<>(new IdentityHashMap()); + private final Set started = + new GridSetWrapper<>(new IdentityHashMap()); /** * @param tx Current transaction. @@ -997,8 +995,8 @@ private void cacheName(String cacheName) { /** * @return If session is started. */ - private boolean started(CacheStoreManager mgr) { - return !started.add(mgr); + private boolean started(CacheStore store) { + return !started.add(store); } /** @@ -1006,7 +1004,7 @@ private boolean started(CacheStoreManager mgr) { * @return Whether session already ended on this store instance. */ private boolean ended(CacheStore store) { - return !ended.add(store); + return !started.remove(store); } /** {@inheritDoc} */ 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 b07a1175c436d..266c5a83cd8c9 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 @@ -27,6 +27,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.IdentityHashMap; import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; @@ -40,6 +41,7 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.cache.CacheWriteSynchronizationMode; +import org.apache.ignite.cache.store.CacheStore; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; @@ -61,6 +63,7 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheVersionConflictContext; import org.apache.ignite.internal.processors.cache.version.GridCacheVersionedEntryEx; import org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException; +import org.apache.ignite.internal.util.GridSetWrapper; import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.lang.GridMetadataAwareAdapter; import org.apache.ignite.internal.util.lang.GridTuple; @@ -1238,13 +1241,15 @@ protected boolean isWriteToStoreFromDhtValid(Collection store * @param commit Commit flag. * @throws IgniteCheckedException In case of error. */ - protected void sessionEnd(Collection stores, boolean commit) throws IgniteCheckedException { + protected void sessionEnd(final Collection stores, boolean commit) throws IgniteCheckedException { Iterator it = stores.iterator(); + Set visited = new GridSetWrapper<>(new IdentityHashMap()); + while (it.hasNext()) { CacheStoreManager store = it.next(); - store.sessionEnd(this, commit, !it.hasNext()); + store.sessionEnd(this, commit, !it.hasNext(), !visited.add(store.store())); } } From 2627c1eb8bb91b90b2f0899c30a28ea98ac562b9 Mon Sep 17 00:00:00 2001 From: Alexander Fedotov Date: Fri, 18 Aug 2017 16:00:30 +0300 Subject: [PATCH 242/357] IGNITE-GG-12646 Fix tests for JSR-310 Java 8 Date and Time API introduced with IGNITE-GG-12549 --- ...CacheQueryJsr310Java8DateTimeApiBaseTest.java | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/modules/indexing/src/test/java8/org/apache/ignite/internal/processors/query/h2/CacheQueryJsr310Java8DateTimeApiBaseTest.java b/modules/indexing/src/test/java8/org/apache/ignite/internal/processors/query/h2/CacheQueryJsr310Java8DateTimeApiBaseTest.java index a65d997e3532d..4e86c6d2c46bf 100644 --- a/modules/indexing/src/test/java8/org/apache/ignite/internal/processors/query/h2/CacheQueryJsr310Java8DateTimeApiBaseTest.java +++ b/modules/indexing/src/test/java8/org/apache/ignite/internal/processors/query/h2/CacheQueryJsr310Java8DateTimeApiBaseTest.java @@ -24,12 +24,19 @@ import org.apache.ignite.cache.CacheMode; import org.apache.ignite.cache.CacheWriteSynchronizationMode; import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +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.common.GridCommonAbstractTest; /** * Base class for JSR-310 Java 8 Date and Time API queries tests. */ public abstract class CacheQueryJsr310Java8DateTimeApiBaseTest extends GridCommonAbstractTest { + /** IP finder. */ + private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + /** {@link LocalTime} instance. */ protected static final LocalTime LOCAL_TIME = LocalTime.now().minusHours(10); @@ -46,6 +53,15 @@ public abstract class CacheQueryJsr310Java8DateTimeApiBaseTest extends GridCommo /** {@link LocalDateTime} instance. */ protected static final LocalDateTime LOCAL_DATE_TIME = LocalDateTime.of(LOCAL_DATE, LocalTime.MIDNIGHT); + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + TcpDiscoverySpi discoverySpi = (TcpDiscoverySpi)cfg.getDiscoverySpi(); + + discoverySpi.setIpFinder(IP_FINDER); + + return cfg; + } + /** * Creates a cache configuration with the specified cache name * and indexed type key/value pairs. From 533128821357c0909710069ea589894d99908474 Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Fri, 10 Feb 2017 16:51:37 +0300 Subject: [PATCH 243/357] 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 558d755a2f5a8499f0dd690834fd97137245fc63 Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Fri, 10 Feb 2017 16:51:37 +0300 Subject: [PATCH 244/357] 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 1153d155c0400..6831ef9bb0d66 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 adaacdda40676..3289780e3fd0e 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 @@ -403,18 +403,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 cd8a487d6d876..31d6f31eef961 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 22191eb8b045e..2c8f9f3a3cdab 100644 --- a/modules/platforms/cpp/odbc/src/utility.cpp +++ b/modules/platforms/cpp/odbc/src/utility.cpp @@ -87,8 +87,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); @@ -101,14 +107,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 71a4718130475..3a3100a6b3131 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs @@ -817,16 +817,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 5efefcbf3bb4573cf85e22869109195158cbe81f Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Fri, 10 Feb 2017 16:51:37 +0300 Subject: [PATCH 245/357] 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 1153d155c0400..6831ef9bb0d66 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 adaacdda40676..3289780e3fd0e 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 @@ -403,18 +403,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 cd8a487d6d876..31d6f31eef961 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 63454dcdb188e..3222a38386262 100644 --- a/modules/platforms/cpp/odbc/src/utility.cpp +++ b/modules/platforms/cpp/odbc/src/utility.cpp @@ -83,8 +83,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); @@ -97,14 +103,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 71a4718130475..3a3100a6b3131 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs @@ -817,16 +817,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 e6a3716b934f2..09c3ad46e08d0 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 e4851162baecfab85263961d3186e070ff296361 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Fri, 18 Aug 2017 19:34:01 +0300 Subject: [PATCH 246/357] GG-12620 - Back port EVT_CACHE_REBALANCE_PART_DATA_LOST changes from 2.x --- .../GridCachePartitionExchangeManager.java | 2 +- .../dht/GridClientPartitionTopology.java | 6 + .../dht/GridDhtPartitionTopology.java | 11 + .../dht/GridDhtPartitionTopologyImpl.java | 51 +++++ .../GridDhtPartitionsExchangeFuture.java | 23 +- .../GridLostPartitionRebalanceTest.java | 198 ++++++++++++++++++ .../testsuites/IgniteCacheTestSuite3.java | 2 + 7 files changed, 291 insertions(+), 2 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridLostPartitionRebalanceTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java index c62ffd2aec79c..e623a2e26368f 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,7 +43,6 @@ 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; @@ -53,6 +52,7 @@ import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.IgniteNeedReconnectException; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.events.DiscoveryCustomEvent; import org.apache.ignite.internal.managers.discovery.DiscoCache; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java index ce8e33ffe5776..3cfc2ada747f9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java @@ -30,6 +30,7 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.events.DiscoveryEvent; import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.managers.discovery.DiscoCache; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; @@ -753,6 +754,11 @@ public long lastUpdateSequence() { } } + /** {@inheritDoc} */ + @Override public void detectLostPartitions(DiscoveryEvent discoEvt) { + assert false : "detectLostPartitions should never be called on client topology"; + } + /** {@inheritDoc} */ @Override public void checkEvictions() { // No-op. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java index bdd84b0329ea1..6275378f38b96 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java @@ -22,6 +22,7 @@ import java.util.UUID; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.events.DiscoveryEvent; import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionExchangeId; @@ -225,6 +226,16 @@ public boolean update(@Nullable GridDhtPartitionExchangeId exchId, @Nullable Map cntrMap, boolean checkEvictions); + /** + * Checks if there is at least one owner for each partition in the cache topology. + * If not, marks such a partition as LOST. + *

    + * This method should be called on topology coordinator after all partition messages are received. + * + * @param discoEvt Discovery event for which we detect lost partitions. + */ + public void detectLostPartitions(DiscoveryEvent discoEvt); + /** * */ 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 0cea80de02aac..eb6658bd2a01b 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 @@ -34,6 +34,7 @@ import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.events.DiscoveryEvent; +import org.apache.ignite.events.EventType; import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException; import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.managers.discovery.DiscoCache; @@ -1234,6 +1235,56 @@ private List ownersAndMoving(int p, AffinityTopologyVersion topVer) } } + /** {@inheritDoc} */ + @Override public void detectLostPartitions(DiscoveryEvent discoEvt) { + lock.writeLock().lock(); + + try { + int parts = cctx.affinity().partitions(); + + Collection lost = null; + + for (int p = 0; p < parts; p++) { + boolean foundOwner = false; + + Set nodeIds = part2node.get(p); + + if (nodeIds != null) { + for (UUID nodeId : nodeIds) { + GridDhtPartitionMap2 partMap = node2part.get(nodeId); + + GridDhtPartitionState state = partMap.get(p); + + if (state == OWNING) { + foundOwner = true; + + break; + } + } + } + + if (!foundOwner) { + if (lost == null) + lost = new HashSet<>(parts - p, 1.0f); + + lost.add(p); + } + } + + if (lost != null) { + // Update partition state on all nodes. + for (Integer part : lost) { + if (cctx.events().isRecordable(EventType.EVT_CACHE_REBALANCE_PART_DATA_LOST)) + cctx.events().addPreloadEvent(part, EVT_CACHE_REBALANCE_PART_DATA_LOST, + discoEvt.eventNode(), discoEvt.type(), discoEvt.timestamp()); + } + } + } + finally { + lock.writeLock().unlock(); + } + } + /** * @param updateSeq Update sequence. * @return {@code True} if state changed. 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 3a12198a62340..d7737308075ec 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 @@ -33,7 +33,6 @@ 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; @@ -45,6 +44,7 @@ import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.IgniteNeedReconnectException; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.events.DiscoveryCustomEvent; import org.apache.ignite.internal.managers.discovery.DiscoCache; @@ -1066,6 +1066,11 @@ private void sendPartitions(ClusterNode oldestNode) { } } + if (discoEvt.type() == EVT_NODE_LEFT || + discoEvt.type() == EVT_NODE_FAILED || + discoEvt.type() == EVT_NODE_JOINED) + detectLostPartitions(); + Map m = null; for (GridCacheContext cacheCtx : cctx.cacheContexts()) { @@ -1278,6 +1283,19 @@ private void onAffinityInitialized(IgniteInternalFuture, int[]> listeners = new HashMap<>(); + + listeners.put(new Listener(), new int[]{EVT_CACHE_REBALANCE_PART_DATA_LOST}); + + cfg.setLocalEventListeners(listeners); + + cfg.setClientMode(gridName.contains("client")); + + final Map attrs = new HashMap<>(); + + attrs.put("node.name", gridName); + + cfg.setUserAttributes(attrs); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** Filter. */ + private static final IgnitePredicate NODE_FILTER = new IgnitePredicate() { + /** */ + private static final long serialVersionUID = 0L; + + @Override public boolean apply(ClusterNode node) { + return !"basenode".equals(node.attribute("node.name")); + } + }; + + /** + * @throws Exception If failed. + */ + public void testPartDataLostEvent() throws Exception { + List srvrs = new ArrayList<>(); + + // Client router. It always up, so client is guaranteed to get + // event. + srvrs.add(startGrid("basenode")); + + Ignite client = startGrid("client"); + + srvrs.add(startGrid("server-1")); + srvrs.add(startGrid("server-2")); + srvrs.add(startGrid("server-3")); + + awaitPartitionMapExchange(); + + IgniteCache cache = client.cache(CACHE_NAME); + + for (int i = 0; i < 10_000; i++) + cache.put(i, i); + + // Stop node with 0 partition. + ClusterNode node = client.affinity(CACHE_NAME).mapPartitionToNode(0); + + for (Ignite srv : srvrs) { + if (node.equals(srv.cluster().localNode())) { + srv.close(); + + System.out.println(">> Stopped " + srv.name()); + + break; + } + } + + // Check that all nodes (and clients) got notified. + assert latch.await(15, TimeUnit.SECONDS) : latch.getCount(); + + // Check that exchange was not finished when event received. + assertFalse("Exchange was finished when event received.", failed); + } + + /** + * + */ + private static class Listener implements IgnitePredicate { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + + /** Ignite. */ + @SuppressWarnings("unused") + @IgniteInstanceResource + private Ignite ignite; + + /** Got. */ + private final AtomicBoolean got = new AtomicBoolean(false); + + /** {@inheritDoc} */ + @Override public boolean apply(CacheRebalancingEvent evt) { + int part = evt.partition(); + + // AtomicBoolean because new owner will produce two events. + if (part == 0 && CACHE_NAME.equals(evt.cacheName()) && got.compareAndSet(false, true)) { + System.out.println(">> Received event for 0 partition. [node=" + ignite.name() + ", evt=" + evt + + ", thread=" + Thread.currentThread().getName() + ']'); + + latch.countDown(); + + if (exchangeCompleted(ignite)) + failed = true; + + return false; + } + + return true; + } + } + + /** + * @param ignite Ignite. + * @return {@code True} if exchange finished. + */ + private static boolean exchangeCompleted(Ignite ignite) { + GridKernalContext ctx = ((IgniteKernal)ignite).context(); + + List futs = ctx.cache().context().exchange().exchangeFutures(); + + for (GridDhtPartitionsExchangeFuture fut : futs) { + if (!fut.isDone()) + return false; + } + + return true; + } +} 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..6101aa3f20a0a 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 @@ -58,6 +58,7 @@ import org.apache.ignite.internal.processors.cache.distributed.rebalancing.GridCacheRebalancingSyncCheckDataTest; import org.apache.ignite.internal.processors.cache.distributed.rebalancing.GridCacheRebalancingSyncSelfTest; import org.apache.ignite.internal.processors.cache.distributed.rebalancing.GridCacheRebalancingUnmarshallingFailedSelfTest; +import org.apache.ignite.internal.processors.cache.distributed.rebalancing.GridLostPartitionRebalanceTest; import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCacheDaemonNodeReplicatedSelfTest; import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCacheReplicatedAtomicGetAndTransformStoreSelfTest; import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCacheReplicatedBasicApiTest; @@ -152,6 +153,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(GridCacheRebalancingUnmarshallingFailedSelfTest.class); suite.addTestSuite(GridCacheRebalancingAsyncSelfTest.class); suite.addTestSuite(GridCacheRabalancingDelayedPartitionMapExchangeSelfTest.class); + suite.addTestSuite(GridLostPartitionRebalanceTest.class); // Test for byte array value special case. suite.addTestSuite(GridCacheLocalByteArrayValuesSelfTest.class); From 74484b2017281595ec964433e0e2f7e76805b01b Mon Sep 17 00:00:00 2001 From: Denis Mekhanikov Date: Sat, 5 Aug 2017 15:54:00 +0300 Subject: [PATCH 247/357] IGNITE-5860 make client reconnect on router's suspend --- .../ignite/spi/discovery/tcp/ClientImpl.java | 50 ++-- .../ignite/spi/discovery/tcp/ServerImpl.java | 250 +++++++++--------- .../spi/discovery/tcp/TcpDiscoveryImpl.java | 2 +- .../spi/discovery/tcp/TcpDiscoverySpi.java | 2 +- .../tcp/TcpClientDiscoverySpiSelfTest.java | 229 +++++++++++++++- .../IgniteSpiDiscoverySelfTestSuite.java | 3 + 6 files changed, 387 insertions(+), 149 deletions(-) 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 2f00f95938ca0..24e7457b75366 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 @@ -26,6 +26,7 @@ import java.net.SocketTimeoutException; import java.util.ArrayDeque; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -467,7 +468,8 @@ else if (state == DISCONNECTED) { } /** - * @param recon {@code True} if reconnects. + * @param prevAddr If reconnect is in progress, then previous address of the router the client was connected to + * and {@code null} otherwise. * @param timeout Timeout. * @return Opened socket or {@code null} if timeout. * @throws InterruptedException If interrupted. @@ -475,9 +477,9 @@ else if (state == DISCONNECTED) { * @see TcpDiscoverySpi#joinTimeout */ @SuppressWarnings("BusyWait") - @Nullable private T2 joinTopology(boolean recon, long timeout) + @Nullable private T2 joinTopology(InetSocketAddress prevAddr, long timeout) throws IgniteSpiException, InterruptedException { - Collection addrs = null; + List addrs = null; long startTime = U.currentTimeMillis(); @@ -506,17 +508,25 @@ else if (state == DISCONNECTED) { } } - Collection addrs0 = new ArrayList<>(addrs); + // process failed node last + if (prevAddr != null) { + int idx = addrs.indexOf(prevAddr); - Iterator it = addrs.iterator(); + if (idx != -1) + Collections.swap(addrs, idx, 0); + } + + Collection addrs0 = new ArrayList<>(addrs); boolean wait = false; - while (it.hasNext()) { + for (int i = addrs.size() - 1; i >= 0; i--) { if (Thread.currentThread().isInterrupted()) throw new InterruptedException(); - InetSocketAddress addr = it.next(); + InetSocketAddress addr = addrs.get(i); + + boolean recon = prevAddr != null; T3 sockAndRes; @@ -530,7 +540,7 @@ else if (state == DISCONNECTED) { } if (sockAndRes == null) { - it.remove(); + addrs.remove(i); continue; } @@ -852,8 +862,8 @@ private NavigableSet allVisibleNodes() { } /** {@inheritDoc} */ - @Override protected IgniteSpiThread workerThread() { - return msgWorker; + @Override protected Collection threads() { + return Arrays.asList(sockWriter, msgWorker); } /** @@ -1339,15 +1349,20 @@ private class Reconnector extends IgniteSpiThread { private boolean clientAck; /** */ - private boolean join; + private final boolean join; + + /** */ + private final InetSocketAddress prevAddr; /** * @param join {@code True} if reconnects during join. + * @param prevAddr Address of the node, that this client was previously connected to. */ - protected Reconnector(boolean join) { + protected Reconnector(boolean join, InetSocketAddress prevAddr) { super(spi.ignite().name(), "tcp-client-disco-reconnector", log); this.join = join; + this.prevAddr = prevAddr; } /** @@ -1377,7 +1392,7 @@ public void cancel() { try { while (true) { - T2 joinRes = joinTopology(true, timeout); + T2 joinRes = joinTopology(prevAddr, timeout); if (joinRes == null) { if (join) { @@ -1612,6 +1627,10 @@ else if (msg instanceof TcpDiscoveryNodeFailedMessage && } else if (msg instanceof SocketClosedMessage) { if (((SocketClosedMessage)msg).sock == currSock) { + Socket sock = currSock.sock; + + InetSocketAddress prevAddr = new InetSocketAddress(sock.getInetAddress(), sock.getPort()); + currSock = null; boolean join = joinLatch.getCount() > 0; @@ -1640,8 +1659,7 @@ else if (msg instanceof SocketClosedMessage) { assert reconnector == null; - final Reconnector reconnector = new Reconnector(join); - this.reconnector = reconnector; + reconnector = new Reconnector(join, prevAddr); reconnector.start(); } } @@ -1811,7 +1829,7 @@ private void tryJoin() throws InterruptedException { T2 joinRes; try { - joinRes = joinTopology(false, spi.joinTimeout); + joinRes = joinTopology(null, spi.joinTimeout); } catch (IgniteSpiException e) { joinError(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 22cb618623644..84343e46a9521 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 @@ -222,6 +222,9 @@ class ServerImpl extends TcpDiscoveryImpl { /** Pending custom messages that should not be sent between NodeAdded and NodeAddFinished messages. */ private Queue pendingCustomMsgs = new ArrayDeque<>(); + /** Messages history used for client reconnect. */ + private final EnsuredMessageHistory msgHist = new EnsuredMessageHistory(); + /** If non-shared IP finder is used this flag shows whether IP finder contains local address. */ private boolean ipFinderHasLocAddr; @@ -1638,8 +1641,23 @@ private void clearNodeAddedMessage(TcpDiscoveryAbstractMessage msg) { } /** {@inheritDoc} */ - @Override protected IgniteSpiThread workerThread() { - return msgWorker; + @Override protected Collection threads() { + Collection threads; + + synchronized (mux) { + threads = new ArrayList<>(readers.size() + clientMsgWorkers.size() + 4); + threads.addAll(readers); + } + + threads.addAll(clientMsgWorkers.values()); + threads.add(tcpSrvr); + threads.add(ipFinderCleaner); + threads.add(msgWorker); + threads.add(statsPrinter); + + threads.removeAll(Collections.singleton(null)); + + return threads; } /** @@ -2069,7 +2087,9 @@ else if (msg instanceof TcpDiscoveryNodeLeftMessage) else if (msg instanceof TcpDiscoveryNodeFailedMessage) clearClientAddFinished(((TcpDiscoveryNodeFailedMessage)msg).failedNodeId()); - msgs.add(msg); + synchronized (msgs) { + msgs.add(msg); + } } /** @@ -2131,14 +2151,16 @@ private boolean mapsEqual(Map m1, Map m2) { // Client connection failed before it received TcpDiscoveryNodeAddedMessage. List res = null; - for (TcpDiscoveryAbstractMessage msg : msgs) { - if (msg instanceof TcpDiscoveryNodeAddedMessage) { - if (node.id().equals(((TcpDiscoveryNodeAddedMessage)msg).node().id())) - res = new ArrayList<>(msgs.size()); - } + synchronized (msgs) { + for (TcpDiscoveryAbstractMessage msg : msgs) { + if (msg instanceof TcpDiscoveryNodeAddedMessage) { + if (node.id().equals(((TcpDiscoveryNodeAddedMessage)msg).node().id())) + res = new ArrayList<>(msgs.size()); + } - if (res != null) - res.add(prepare(msg, node.id())); + if (res != null) + res.add(prepare(msg, node.id())); + } } if (log.isDebugEnabled()) { @@ -2151,20 +2173,26 @@ private boolean mapsEqual(Map m1, Map m2) { return res; } else { - if (msgs.isEmpty()) - return Collections.emptyList(); + Collection cp; - Collection cp = new ArrayList<>(msgs.size()); + boolean skip; - boolean skip = true; + synchronized (msgs) { + if (msgs.isEmpty()) + return Collections.emptyList(); - for (TcpDiscoveryAbstractMessage msg : msgs) { - if (skip) { - if (msg.id().equals(lastMsgId)) - skip = false; + cp = new ArrayList<>(msgs.size()); + + skip = true; + + for (TcpDiscoveryAbstractMessage msg : msgs) { + if (skip) { + if (msg.id().equals(lastMsgId)) + skip = false; + } + else + cp.add(prepare(msg, node.id())); } - else - cp.add(prepare(msg, node.id())); } cp = !skip ? cp : null; @@ -2453,9 +2481,6 @@ private class RingMessageWorker extends MessageWorkerAdapter pending = msgHist.messages(msg.lastMessageId(), node); - - if (pending != null) { - msg.pendingMessages(pending); - msg.success(true); - - if (log.isDebugEnabled()) - log.debug("Accept client reconnect, restored pending messages " + - "[locNodeId=" + locNodeId + ", clientNodeId=" + nodeId + ']'); - } - else { - if (log.isDebugEnabled()) - log.debug("Failing reconnecting client node because failed to restore pending " + - "messages [locNodeId=" + locNodeId + ", clientNodeId=" + nodeId + ']'); - - TcpDiscoveryNodeFailedMessage nodeFailedMsg = new TcpDiscoveryNodeFailedMessage(locNodeId, - node.id(), node.internalOrder()); - - processNodeFailedMessage(nodeFailedMsg); - - if (nodeFailedMsg.verified()) - msgHist.add(nodeFailedMsg); - } - } - else if (log.isDebugEnabled()) - log.debug("Reconnecting client node is already failed [nodeId=" + nodeId + ']'); + TcpDiscoveryNode node = ring.node(nodeId); - if (isLocNodeRouter) { - ClientMessageWorker wrk = clientMsgWorkers.get(nodeId); + assert node == null || node.isClient(); - if (wrk != null) - wrk.addMessage(msg); - else if (log.isDebugEnabled()) - log.debug("Failed to reconnect client node (disconnected during the process) [locNodeId=" + - locNodeId + ", clientNodeId=" + nodeId + ']'); - } - else { - if (sendMessageToRemotes(msg)) - sendMessageAcrossRing(msg); - } - } - else { - if (sendMessageToRemotes(msg)) - sendMessageAcrossRing(msg); - } + if (node != null) { + node.clientRouterNodeId(msg.routerNodeId()); + node.aliveCheck(spi.maxMissedClientHbs); } - else { - if (isLocalNodeCoordinator()) - addMessage(new TcpDiscoveryDiscardMessage(locNodeId, msg.id(), false)); - if (isLocNodeRouter) { - ClientMessageWorker wrk = clientMsgWorkers.get(nodeId); + if (!isLocalNodeCoordinator() && sendMessageToRemotes(msg)) + sendMessageAcrossRing(msg); - if (wrk != null) - wrk.addMessage(msg); - else if (log.isDebugEnabled()) - log.debug("Failed to reconnect client node (disconnected during the process) [locNodeId=" + - locNodeId + ", clientNodeId=" + nodeId + ']'); - } - else { - if (ring.hasRemoteNodes() && !isLocalNodeCoordinator()) - sendMessageAcrossRing(msg); - } + if (msg.verified() && msg.routerNodeId().equals(getLocalNodeId())) { + ClientMessageWorker wrk = clientMsgWorkers.get(nodeId); + + if (wrk != null) + wrk.addMessage(msg); + else if (log.isDebugEnabled()) + log.debug("Failed to reconnect client node (disconnected during the process) [locNodeId=" + + locNodeId + ", clientNodeId=" + nodeId + ']'); } } @@ -4071,9 +4033,6 @@ private void processNodeAddedMessage(TcpDiscoveryNodeAddedMessage msg) { processNodeAddFinishedMessage(addFinishMsg); - if (addFinishMsg.verified()) - msgHist.add(addFinishMsg); - addMessage(new TcpDiscoveryDiscardMessage(locNodeId, msg.id(), false)); return; @@ -5134,9 +5093,6 @@ private void processHeartbeatMessage(TcpDiscoveryHeartbeatMessage msg) { locNodeId, clientNode.id(), clientNode.internalOrder()); processNodeFailedMessage(nodeFailedMsg); - - if (nodeFailedMsg.verified()) - msgHist.add(nodeFailedMsg); } } } @@ -5334,9 +5290,6 @@ private void processCustomMessage(TcpDiscoveryCustomEventMessage msg) { ackMsg.topologyVersion(msg.topologyVersion()); processCustomMessage(ackMsg); - - if (ackMsg.verified()) - msgHist.add(ackMsg); } catch (IgniteCheckedException e) { U.error(log, "Failed to marshal discovery custom message.", e); @@ -5438,12 +5391,8 @@ private void checkPendingCustomMessages() { if (joiningEmpty && isLocalNodeCoordinator()) { TcpDiscoveryCustomEventMessage msg; - while ((msg = pollPendingCustomeMessage()) != null) { + while ((msg = pollPendingCustomeMessage()) != null) processCustomMessage(msg); - - if (msg.verified()) - msgHist.add(msg); - } } } @@ -5997,7 +5946,7 @@ else if (msg instanceof TcpDiscoveryClientReconnectMessage) { if (clientMsgWrk.getState() == State.NEW) clientMsgWrk.start(); - msgWorker.addMessage(msg); + processClientReconnectMessage((TcpDiscoveryClientReconnectMessage)msg); continue; } @@ -6234,6 +6183,67 @@ else if (msg instanceof TcpDiscoveryRingLatencyCheckMessage) { } } + /** + * Processes client reconnect message. + * + * @param msg Client reconnect message. + */ + private void processClientReconnectMessage(TcpDiscoveryClientReconnectMessage msg) { + UUID nodeId = msg.creatorNodeId(); + UUID locNodeId = getLocalNodeId(); + + boolean isLocNodeRouter = locNodeId.equals(msg.routerNodeId()); + + if (isLocNodeRouter) { + TcpDiscoveryNode node = ring.node(nodeId); + ClientMessageWorker wrk = clientMsgWorkers.get(nodeId); + + if (wrk != null && node != null) { + if (!msg.verified()) { + msg.verify(getLocalNodeId()); + + Collection pending = msgHist.messages(msg.lastMessageId(), node); + + if (pending != null) { + msg.success(true); + msg.pendingMessages(pending); + + TcpDiscoveryClientReconnectMessage msgCp = new TcpDiscoveryClientReconnectMessage( + msg.creatorNodeId(), msg.routerNodeId(), msg.lastMessageId()); + msgCp.client(msg.client()); + + msgWorker.addMessage(msgCp); + + if (log.isDebugEnabled()) { + log.debug("Accept client reconnect, restored pending messages " + + "[locNodeId=" + locNodeId + ", clientNodeId=" + nodeId + ']'); + } + } + else { + if (log.isDebugEnabled()) + log.debug("Failing reconnecting client node because failed to restore pending " + + "messages [locNodeId=" + locNodeId + ", clientNodeId=" + nodeId + ']'); + + TcpDiscoveryNodeFailedMessage nodeFailedMsg = new TcpDiscoveryNodeFailedMessage(locNodeId, + node.id(), node.internalOrder()); + + msgWorker.addMessage(nodeFailedMsg); + } + } + else + wrk.addMessage(msg); + } + else if (log.isDebugEnabled()) + log.debug("Failed to reconnect client node (disconnected during the process) [locNodeId=" + + locNodeId + ", clientNodeId=" + nodeId + ']'); + + if (wrk != null) + wrk.addMessage(msg); + } + else + msgWorker.addMessage(msg); + } + /** * Processes client heartbeat message. * 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 cb85dc1c44b86..1fa7139e7e956 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 @@ -292,7 +292,7 @@ protected static String threadStatus(Thread t) { * * @return Worker thread. */ - protected abstract IgniteSpiThread workerThread(); + protected abstract Collection threads(); /** * @throws IgniteSpiException If failed. 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 8c7ef99afcbff..a664ae4f91ab9 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 @@ -1563,7 +1563,7 @@ protected int readReceipt(Socket sock, long timeout) throws IOException { * empty but never null). * @throws org.apache.ignite.spi.IgniteSpiException If an error occurs. */ - protected Collection resolvedAddresses() throws IgniteSpiException { + protected List resolvedAddresses() throws IgniteSpiException { List res = new ArrayList<>(); Collection addrs; 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 419497753bbc2..819108148cf16 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 @@ -62,8 +62,8 @@ import org.apache.ignite.spi.IgniteSpiException; import org.apache.ignite.spi.IgniteSpiOperationTimeoutException; import org.apache.ignite.spi.IgniteSpiOperationTimeoutHelper; +import org.apache.ignite.spi.IgniteSpiThread; 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.TcpDiscoveryClientReconnectMessage; @@ -82,13 +82,14 @@ 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.events.EventType.EVT_NODE_SEGMENTED; +import static org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi.DFLT_HEARTBEAT_FREQ; /** * Client-based discovery tests. */ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest { /** */ - private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + private static final TcpDiscoveryVmIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); /** */ protected static final AtomicInteger srvIdx = new AtomicInteger(); @@ -123,6 +124,9 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest { /** */ private static CountDownLatch clientFailedLatch; + /** */ + private static CountDownLatch clientReconnectedLatch; + /** */ private static CountDownLatch msgLatch; @@ -138,6 +142,9 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest { /** */ protected long netTimeout = TcpDiscoverySpi.DFLT_NETWORK_TIMEOUT; + /** */ + protected Integer reconnectCnt; + /** */ private boolean longSockTimeouts; @@ -208,6 +215,9 @@ else if (gridName.startsWith("client")) { disco.setJoinTimeout(joinTimeout); disco.setNetworkTimeout(netTimeout); + if (reconnectCnt != null) + disco.setReconnectCount(reconnectCnt); + disco.setClientReconnectDisabled(reconnectDisabled); if (disco instanceof TestTcpDiscoverySpi) @@ -254,6 +264,8 @@ protected TcpDiscoverySpi getDiscoverySpi() { clientIpFinder = null; joinTimeout = TcpDiscoverySpi.DFLT_JOIN_TIMEOUT; netTimeout = TcpDiscoverySpi.DFLT_NETWORK_TIMEOUT; + maxMissedClientHbs = TcpDiscoverySpi.DFLT_MAX_MISSED_CLIENT_HEARTBEATS; + reconnectCnt = null; longSockTimeouts = false; assert G.allGrids().isEmpty(); @@ -534,6 +546,176 @@ public void testClientReconnectOnRouterFail() throws Exception { checkNodes(2, 3); } + /** + * Client should reconnect to available server without EVT_CLIENT_NODE_RECONNECTED event. + * + * @throws Exception If failed. + */ + public void testClientReconnectOnRouterSuspend() throws Exception { + reconnectAfterSuspend(false); + } + + /** + * Client should receive all topology updates after reconnect. + * + * @throws Exception If failed. + */ + public void testClientReconnectOnRouterSuspendTopologyChange() throws Exception { + reconnectAfterSuspend(true); + } + + /** + * @param changeTop If {@code true} topology is changed after client disconnects + * @throws Exception if failed. + */ + private void reconnectAfterSuspend(boolean changeTop) throws Exception { + reconnectCnt = 2; + + startServerNodes(2); + + Ignite srv0 = G.ignite("server-0"); + TcpDiscoveryNode srv0Node = (TcpDiscoveryNode)srv0.cluster().localNode(); + + TcpDiscoveryNode srv1Node = (TcpDiscoveryNode)G.ignite("server-1").cluster().localNode(); + + clientIpFinder = new TcpDiscoveryVmIpFinder(); + + clientIpFinder.setAddresses( + Collections.singleton("localhost:" + srv0Node.discoveryPort())); + + startClientNodes(1); + + Ignite client = G.ignite("client-0"); + TcpDiscoveryNode clientNode = (TcpDiscoveryNode)client.cluster().localNode(); + TestTcpDiscoverySpi clientSpi = (TestTcpDiscoverySpi)client.configuration().getDiscoverySpi(); + + UUID clientNodeId = clientNode.id(); + + checkNodes(2, 1); + + clientIpFinder.setAddresses(Collections.singleton("localhost:" + srv1Node.discoveryPort())); + + srvFailedLatch = new CountDownLatch(1); + + attachListeners(2, 1); + + log.info("Pausing router"); + + TestTcpDiscoverySpi srvSpi = (TestTcpDiscoverySpi)srv0.configuration().getDiscoverySpi(); + + int joinedNodesNum = 3; + final CountDownLatch srvJoinedLatch = new CountDownLatch(joinedNodesNum); + + if (changeTop) { + client.events().localListen(new IgnitePredicate() { + @Override public boolean apply(Event e) { + srvJoinedLatch.countDown(); + + return true; + } + }, EVT_NODE_JOINED); + } + + srvSpi.pauseAll(true); + + if (changeTop) + startServerNodes(joinedNodesNum); + + try { + await(srvFailedLatch, 60_000); + + if (changeTop) + await(srvJoinedLatch, 5000); + + assertEquals("connected", clientSpi.getSpiState()); + assertEquals(clientNodeId, clientNode.id()); + assertEquals(srv1Node.id(), clientNode.clientRouterNodeId()); + } + finally { + srvSpi.resumeAll(); + } + } + + /** + * + */ + public void testReconnectAfterPause() throws Exception { + maxMissedClientHbs = 2; + + startServerNodes(2); + startClientNodes(1); + + Ignite client = G.ignite("client-0"); + TestTcpDiscoverySpi clientSpi = (TestTcpDiscoverySpi)client.configuration().getDiscoverySpi(); + + clientReconnectedLatch = new CountDownLatch(1); + + attachListeners(0, 1); + + clientSpi.pauseAll(false); + + try { + clientSpi.brakeConnection(); + + Thread.sleep(maxMissedClientHbs * DFLT_HEARTBEAT_FREQ * 2); + } + finally { + clientSpi.resumeAll(); + } + + await(clientReconnectedLatch); + } + + /** + * @throws Exception if failed. + */ + public void testReconnectAfterMassiveTopologyChange() throws Exception { + clientIpFinder = IP_FINDER; + + maxMissedClientHbs = 100; + netTimeout = 100000; + + int initSrvsNum = 5; + int killNum = 2; + int iterations = 3; + + startServerNodes(initSrvsNum); + startClientNodes(1); + + Ignite client = G.ignite("client-0"); + TcpDiscoveryNode clientNode = (TcpDiscoveryNode)client.cluster().localNode(); + TestTcpDiscoverySpi clientSpi = (TestTcpDiscoverySpi)client.configuration().getDiscoverySpi(); + final UUID clientNodeId = clientNode.id(); + + final CountDownLatch srvJoinedLatch = new CountDownLatch(iterations * killNum); + + client.events().localListen(new IgnitePredicate() { + @Override public boolean apply(Event e) { + srvJoinedLatch.countDown(); + + return true; + } + }, EVT_NODE_JOINED); + + int minAliveSrvId = 0; + + for (int i = 0; i < iterations; i++) { + startServerNodes(killNum); + + for (int j = 0; j < killNum; j++) { + failServer(minAliveSrvId); + + minAliveSrvId++; + } + + Thread.sleep(500); + } + + await(srvJoinedLatch); + assertEquals("connected", clientSpi.getSpiState()); + assertEquals(clientNodeId, clientNode.id()); + } + /** * @throws Exception If failed. */ @@ -1387,17 +1569,16 @@ else if (evt.type() == EVT_CLIENT_NODE_RECONNECTED) { srvSpi.failNode(client.cluster().localNode().id(), null); - if (changeTop) { - Ignite g = startGrid("server-" + srvIdx.getAndIncrement()); + assertTrue(disconnectLatch.await(5000, MILLISECONDS)); + assertTrue(failLatch.await(5000, MILLISECONDS)); - srvNodeIds.add(g.cluster().localNode().id()); + if (changeTop) { + startServerNodes(1); clientSpi.resumeAll(); } - assertTrue(disconnectLatch.await(5000, MILLISECONDS)); assertTrue(reconnectLatch.await(5000, MILLISECONDS)); - assertTrue(failLatch.await(5000, MILLISECONDS)); assertTrue(joinLatch.await(5000, MILLISECONDS)); long topVer = changeTop ? 5L : 4L; @@ -2003,6 +2184,20 @@ private void attachListeners(int srvCnt, int clientCnt) throws Exception { }, EVT_NODE_FAILED); } } + + if (clientReconnectedLatch != null) { + for (int i = 0; i < clientCnt; i++) { + G.ignite("client-" + i).events().localListen(new IgnitePredicate() { + @Override public boolean apply(Event evt) { + info("Reconnected event fired on client: " + evt); + + clientReconnectedLatch.countDown(); + + return true; + } + }, EVT_CLIENT_NODE_RECONNECTED); + } + } } /** @@ -2072,7 +2267,16 @@ else if (srvNodeIds.contains(id)) * @throws InterruptedException If interrupted. */ protected void await(CountDownLatch latch) throws InterruptedException { - assertTrue("Latch count: " + latch.getCount(), latch.await(awaitTime(), MILLISECONDS)); + await(latch, awaitTime()); + } + + /** + * @param latch Latch. + * @param timeout Timeout. + * @throws InterruptedException If interrupted. + */ + protected void await(CountDownLatch latch, long timeout) throws InterruptedException { + assertTrue("Latch count: " + latch.getCount(), latch.await(timeout, MILLISECONDS)); } /** @@ -2283,8 +2487,10 @@ public void pauseSocketWrite() { public void pauseAll(boolean suspend) { pauseResumeOperation(true, openSockLock, writeLock); - if (suspend) - impl.workerThread().suspend(); + if (suspend) { + for (Thread t : impl.threads()) + t.suspend(); + } } /** @@ -2293,7 +2499,8 @@ public void pauseAll(boolean suspend) { public void resumeAll() { pauseResumeOperation(false, openSockLock, writeLock); - impl.workerThread().resume(); + for (IgniteSpiThread t : impl.threads()) + t.resume(); } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java index 12871492d0979..fa470e911790a 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java @@ -20,6 +20,7 @@ import junit.framework.TestSuite; import org.apache.ignite.spi.GridTcpSpiForwardingSelfTest; import org.apache.ignite.spi.discovery.AuthenticationRestartTest; +import org.apache.ignite.spi.discovery.tcp.IgniteClientReconnectMassiveShutdownTest; import org.apache.ignite.spi.discovery.tcp.TcpClientDiscoveryMarshallerCheckSelfTest; import org.apache.ignite.spi.discovery.tcp.TcpClientDiscoverySpiFailureTimeoutSelfTest; import org.apache.ignite.spi.discovery.tcp.TcpClientDiscoverySpiMulticastTest; @@ -90,6 +91,8 @@ public static TestSuite suite() throws Exception { suite.addTest(new TestSuite(TcpDiscoveryNodeAttributesUpdateOnReconnectTest.class)); suite.addTest(new TestSuite(AuthenticationRestartTest.class)); + suite.addTest(new TestSuite(IgniteClientReconnectMassiveShutdownTest.class)); + // SSL. suite.addTest(new TestSuite(TcpDiscoverySslSelfTest.class)); suite.addTest(new TestSuite(TcpDiscoverySslSecuredUnsecuredTest.class)); From 5b89e7acf86cfa95a963c587e039efd29638c772 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Mon, 21 Aug 2017 17:23:36 +0300 Subject: [PATCH 248/357] IGNITE-5860 - Api compliance fix. --- .../java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java | 2 +- .../org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) 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 24e7457b75366..6c12ceeb364bc 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 @@ -488,7 +488,7 @@ else if (state == DISCONNECTED) { throw new InterruptedException(); while (addrs == null || addrs.isEmpty()) { - addrs = spi.resolvedAddresses(); + addrs = new ArrayList<>(spi.resolvedAddresses()); if (!F.isEmpty(addrs)) { if (log.isDebugEnabled()) 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 a664ae4f91ab9..8c7ef99afcbff 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 @@ -1563,7 +1563,7 @@ protected int readReceipt(Socket sock, long timeout) throws IOException { * empty but never null). * @throws org.apache.ignite.spi.IgniteSpiException If an error occurs. */ - protected List resolvedAddresses() throws IgniteSpiException { + protected Collection resolvedAddresses() throws IgniteSpiException { List res = new ArrayList<>(); Collection addrs; From ded6760b116981a4ad5d58571b4cd140eca72966 Mon Sep 17 00:00:00 2001 From: Alexander Fedotov Date: Tue, 22 Aug 2017 17:07:09 +0300 Subject: [PATCH 249/357] IGNITE-6150 ignite-osgi-karaf-licenses.txt is absent in a build --- modules/osgi-karaf/pom.xml | 26 +++++++++++++------------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/modules/osgi-karaf/pom.xml b/modules/osgi-karaf/pom.xml index 97a9048974c33..9b99823b8e571 100644 --- a/modules/osgi-karaf/pom.xml +++ b/modules/osgi-karaf/pom.xml @@ -42,19 +42,6 @@ - - org.apache.maven.plugins - maven-resources-plugin - - - filter - generate-resources - - resources - - - - org.codehaus.mojo build-helper-maven-plugin @@ -78,6 +65,19 @@ + + org.apache.maven.plugins + maven-resources-plugin + + + filter + generate-resources + + resources + + + + From 0700f8f473fd1e5507e3782f4c4bb9e74ec17947 Mon Sep 17 00:00:00 2001 From: Ilya Kasnacheev Date: Mon, 21 Aug 2017 17:54:12 +0300 Subject: [PATCH 250/357] GG-12588 Backport IGNITE-5943 Communication. Server node may reject client connection during massive clients join. --- .../tcp/TcpCommunicationSpi.java | 61 ++++++- .../ignite/spi/discovery/tcp/ServerImpl.java | 24 ++- .../spi/discovery/tcp/TcpDiscoverySpi.java | 10 + .../tcp/IgniteClientConnectTest.java | 172 ++++++++++++++++++ .../IgniteSpiDiscoverySelfTestSuite.java | 6 + 5 files changed, 266 insertions(+), 7 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/IgniteClientConnectTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java index 7a5f7c1f16f3a..0743ea47c3e67 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 @@ -65,6 +65,7 @@ import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener; +import org.apache.ignite.internal.util.typedef.CI1; import org.apache.ignite.internal.util.GridConcurrentFactory; import org.apache.ignite.internal.util.GridSpinReadWriteLock; import org.apache.ignite.internal.util.future.GridFutureAdapter; @@ -129,6 +130,8 @@ import org.apache.ignite.spi.IgniteSpiTimeoutObject; import org.apache.ignite.spi.communication.CommunicationListener; import org.apache.ignite.spi.communication.CommunicationSpi; +import org.apache.ignite.spi.discovery.DiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.thread.IgniteThread; import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentLinkedDeque8; @@ -137,6 +140,7 @@ import static org.apache.ignite.events.EventType.EVT_NODE_FAILED; import static org.apache.ignite.events.EventType.EVT_NODE_LEFT; import static org.apache.ignite.internal.util.nio.GridNioSessionMetaKey.SSL_META; +import static org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi.RecoveryLastReceivedMessage.NEED_WAIT; /** * TcpCommunicationSpi is default communication SPI which uses @@ -301,6 +305,13 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter /** Connection index meta for session. */ private static final int CONN_IDX_META = GridNioSessionMetaKey.nextUniqueKey(); + /** + * Version when client is ready to wait to connect to server (could be needed when client tries to open connection + * before it starts being visible for server) + */ + private static final IgniteProductVersion VERSION_SINCE_CLIENT_COULD_WAIT_TO_CONNECT = + IgniteProductVersion.fromString("1.8.10"); + /** Message tracker meta for session. */ private static final int TRACKER_META = GridNioSessionMetaKey.nextUniqueKey(); @@ -435,7 +446,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter * @param ses Session. * @param msg Message. */ - private void onFirstMessage(GridNioSession ses, Message msg) { + private void onFirstMessage(final GridNioSession ses, Message msg) { UUID sndId; ConnectionKey connKey; @@ -459,10 +470,35 @@ private void onFirstMessage(GridNioSession ses, Message msg) { final ClusterNode rmtNode = getSpiContext().node(sndId); if (rmtNode == null) { - U.warn(log, "Close incoming connection, unknown node [nodeId=" + sndId + - ", ses=" + ses + ']'); + DiscoverySpi discoverySpi = ignite.configuration().getDiscoverySpi(); + + assert discoverySpi instanceof TcpDiscoverySpi; + + TcpDiscoverySpi tcpDiscoverySpi = (TcpDiscoverySpi) discoverySpi; + + ClusterNode node0 = tcpDiscoverySpi.getNode0(sndId); + + boolean unknownNode = true; - ses.close(); + if (node0 != null) { + assert node0.isClient() : node0; + + if (node0.version().compareTo(VERSION_SINCE_CLIENT_COULD_WAIT_TO_CONNECT) >= 0) + unknownNode = false; + } + + if (unknownNode) { + U.warn(log, "Close incoming connection, unknown node [nodeId=" + sndId + ", ses=" + ses + ']'); + + ses.close(); + } + else { + ses.send(new RecoveryLastReceivedMessage(NEED_WAIT)).listen(new CI1>() { + @Override public void apply(IgniteInternalFuture fut) { + ses.close(); + } + }); + } return; } @@ -2858,6 +2894,8 @@ protected GridCommunicationClient createTcpClient(ClusterNode node, int connIdx) IgniteSpiOperationTimeoutHelper timeoutHelper = new IgniteSpiOperationTimeoutHelper(this); + int lastWaitingTimeout = 1; + while (!conn) { // Reconnection on handshake timeout. try { SocketChannel ch = SocketChannel.open(); @@ -2920,6 +2958,18 @@ protected GridCommunicationClient createTcpClient(ClusterNode node, int connIdx) if (rcvCnt == -1) return null; + else if (rcvCnt == NEED_WAIT) { + recoveryDesc.release(); + + U.closeQuiet(ch); + + if (lastWaitingTimeout < 60000) + lastWaitingTimeout *= 2; + + U.sleep(lastWaitingTimeout); + + continue; + } } finally { if (recoveryDesc != null && rcvCnt == -1) @@ -4294,6 +4344,9 @@ public static class RecoveryLastReceivedMessage implements Message { /** */ private static final long serialVersionUID = 0L; + /** Need wait. */ + static final long NEED_WAIT = -3; + /** */ private long rcvCnt; 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 1c8988acb2ba1..cfa36f1599590 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 @@ -1785,9 +1785,27 @@ private static void removeMetrics(TcpDiscoveryHeartbeatMessage msg, UUID nodeId) } /** - * Thread that cleans IP finder and keeps it in the correct state, unregistering addresses of the nodes that has - * left the topology.

    This thread should run only on coordinator node and will clean IP finder if and only if - * {@link org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder#isShared()} is {@code true}. + * Trying get node in any state (visible or not) + * @param nodeId Node id. + */ + ClusterNode getNode0(UUID nodeId) { + assert nodeId != null; + + UUID locNodeId0 = getLocalNodeId(); + + if (locNodeId0 != null && locNodeId0.equals(nodeId)) + // Return local node directly. + return locNode; + + return ring.node(nodeId); + } + + /** + * Thread that cleans IP finder and keeps it in the correct state, unregistering + * addresses of the nodes that has left the topology. + *

    + * This thread should run only on coordinator node and will clean IP finder + * if and only if {@link org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder#isShared()} is {@code true}. */ private class IpFinderCleaner extends IgniteSpiThread { /** 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 3cc4ee53242b2..b505ae373f210 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 @@ -437,6 +437,16 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T return impl.getNode(nodeId); } + /** + * @param id Id. + */ + public ClusterNode getNode0(UUID id) { + if (impl instanceof ServerImpl) + return ((ServerImpl)impl).getNode0(id); + + return getNode(id); + } + /** {@inheritDoc} */ @Override public boolean pingNode(UUID nodeId) { return impl.pingNode(nodeId); diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/IgniteClientConnectTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/IgniteClientConnectTest.java new file mode 100644 index 0000000000000..b1c4dbf25434d --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/IgniteClientConnectTest.java @@ -0,0 +1,172 @@ +/* + * 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.io.IOException; +import java.io.OutputStream; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.net.Socket; +import java.util.*; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteEx; +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.TcpDiscoveryNodeAddFinishedMessage; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + + +/** + * We emulate that client receive message about joining to topology earlier than some server nodes in topology. + * And make this client connect to such servers. + * To emulate this we connect client to second node in topology and pause sending message about joining finishing to + * third node. + */ +public class IgniteClientConnectTest extends GridCommonAbstractTest { + + /** Custom cache name. */ + private static final String DEFAULT_CACHE_NAME = "default-cache"; + + /** */ + private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); + + /** Latch to stop message sending. */ + private final CountDownLatch latch = new CountDownLatch(1); + + /** Start client flag. */ + private final AtomicBoolean clientJustStarted = new AtomicBoolean(false); + + /** Instance name. */ + private String igniteInstanceName; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + this.igniteInstanceName = igniteInstanceName; + + TestTcpDiscoverySpi disco = new TestTcpDiscoverySpi(); + + if (igniteInstanceName.equals("client")) { + TcpDiscoveryVmIpFinder ipFinder = new TcpDiscoveryVmIpFinder(); + + ipFinder.registerAddresses(Collections.singleton(new InetSocketAddress(InetAddress.getLoopbackAddress(), 47501))); + + disco.setIpFinder(ipFinder); + } + else + disco.setIpFinder(ipFinder); + + disco.setJoinTimeout(2 * 60_000); + disco.setSocketTimeout(1000); + disco.setNetworkTimeout(2000); + + cfg.setDiscoverySpi(disco); + + CacheConfiguration cacheConfiguration = new CacheConfiguration() + .setName(DEFAULT_CACHE_NAME) + .setCacheMode(CacheMode.PARTITIONED) + .setAffinity(new RendezvousAffinityFunction(false, 8)) + .setBackups(0); + + cfg.setCacheConfiguration(cacheConfiguration); + + return cfg; + } + + /** + * + * @throws Exception If failed. + */ + public void testClientConnectToBigTopology() throws Exception { + Ignite ignite = startGrids(3); + + IgniteCache cache = ignite.cache(DEFAULT_CACHE_NAME); + + Set keys = new HashSet<>(); + + for (int i = 0; i < 80; i++) { + cache.put(i, i); + + keys.add(i); + } + + TcpDiscoveryImpl discovery = ((TestTcpDiscoverySpi) ignite.configuration().getDiscoverySpi()).discovery(); + + assertTrue(discovery instanceof ServerImpl); + + IgniteConfiguration clientCfg = getConfiguration("client"); + + clientCfg.setClientMode(true); + + clientJustStarted.set(true); + + Ignite client = startGrid(igniteInstanceName, clientCfg); + + latch.countDown(); + + System.err.println("GET ALL"); + client.cache(DEFAULT_CACHE_NAME).getAll(keys); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** + * + */ + class TestTcpDiscoverySpi extends TcpDiscoverySpi { + /** {@inheritDoc} */ + protected void writeToSocket(Socket sock, OutputStream out, TcpDiscoveryAbstractMessage msg, long timeout) throws IOException, + IgniteCheckedException { + if (msg instanceof TcpDiscoveryNodeAddFinishedMessage) { + if (msg.senderNodeId() != null && clientJustStarted.get()) + try { + latch.await(); + + Thread.sleep(3000); + } catch (InterruptedException e) { + e.printStackTrace(); + } + + super.writeToSocket(sock, out, msg, timeout); + } + else + super.writeToSocket(sock, out, msg, timeout); + } + + /** + * + */ + TcpDiscoveryImpl discovery() { + return impl; + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java index e6b39f740a0dd..c2913ba6c6a5b 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java @@ -19,6 +19,8 @@ import junit.framework.TestSuite; import org.apache.ignite.spi.GridTcpSpiForwardingSelfTest; +import org.apache.ignite.spi.discovery.tcp.IgniteClientConnectTest; +import org.apache.ignite.spi.discovery.tcp.IgniteClientReconnectMassiveShutdownTest; import org.apache.ignite.spi.discovery.tcp.TcpClientDiscoveryMarshallerCheckSelfTest; import org.apache.ignite.spi.discovery.tcp.TcpClientDiscoverySpiFailureTimeoutSelfTest; import org.apache.ignite.spi.discovery.tcp.TcpClientDiscoverySpiMulticastTest; @@ -88,6 +90,10 @@ public static TestSuite suite() throws Exception { suite.addTest(new TestSuite(TcpDiscoveryNodeAttributesUpdateOnReconnectTest.class)); + //Client connect + suite.addTest(new TestSuite(IgniteClientConnectTest.class)); + suite.addTest(new TestSuite(IgniteClientReconnectMassiveShutdownTest.class)); + // SSL. suite.addTest(new TestSuite(TcpDiscoverySslSelfTest.class)); suite.addTest(new TestSuite(TcpDiscoverySslSecuredUnsecuredTest.class)); From 890dcd128b3ae0829dd1fb96e3b1ab913f70d541 Mon Sep 17 00:00:00 2001 From: Slava Koptilin Date: Wed, 16 Aug 2017 18:06:36 +0300 Subject: [PATCH 251/357] IGNITE-1094: wip --- .../apache/ignite/internal/IgniteKernal.java | 4 + .../ignite/internal/IgniteNodeAttributes.java | 3 + .../discovery/GridDiscoveryManager.java | 39 +- .../cache/CacheAffinitySharedManager.java | 71 +++ .../DynamicCacheChangeFailureMessage.java | 129 ++++++ .../GridCachePartitionExchangeManager.java | 13 +- .../processors/cache/GridCacheProcessor.java | 42 +- .../GridDhtPartitionsExchangeFuture.java | 189 +++++++- .../GridDhtPartitionsSingleMessage.java | 59 ++- .../IgniteDynamicCacheStartFailSelfTest.java | 421 ++++++++++++++++++ 10 files changed, 938 insertions(+), 32 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeFailureMessage.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartFailSelfTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java index 28ac0436eab9d..e58811baa6fac 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 @@ -201,6 +201,7 @@ 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_DEPLOYMENT_MODE; +import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_EXCHANGE_ROLLBACK_SUPPORTED; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_GRID_NAME; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_IPS; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_JIT_NAME; @@ -1493,6 +1494,9 @@ private void fillNodeAttributes(boolean notifyEnabled) throws IgniteCheckedExcep if (cfg.getConnectorConfiguration() != null) add(ATTR_REST_PORT_RANGE, cfg.getConnectorConfiguration().getPortRange()); + // Exchange rollback is supported + add(ATTR_EXCHANGE_ROLLBACK_SUPPORTED, Boolean.TRUE); + // Stick in SPI versions and classes attributes. addSpiAttributes(cfg.getCollisionSpi()); addSpiAttributes(cfg.getSwapSpaceSpi()); 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 436792459af4a..5002c4c31867c 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 @@ -166,6 +166,9 @@ public final class IgniteNodeAttributes { /** Ignite security compatibility mode. */ public static final String ATTR_SECURITY_COMPATIBILITY_MODE = ATTR_PREFIX + ".security.compatibility.enabled"; + /** Internal attribute name constant. */ + public static final String ATTR_EXCHANGE_ROLLBACK_SUPPORTED = ATTR_PREFIX + ".exchange.rollback.supported"; + /** * 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 71b7217bd6afc..1d67869bb478e 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 @@ -254,7 +254,7 @@ public class GridDiscoveryManager extends GridManagerAdapter { new ConcurrentHashMap8<>(); /** Map of dynamic cache filters. */ - private Map registeredCaches = new HashMap<>(); + private Map registeredCachesPreds = new HashMap<>(); /** */ private final GridSpinBusyLock busyLock = new GridSpinBusyLock(); @@ -312,19 +312,20 @@ public void setCacheFilter( boolean nearEnabled, CacheMode cacheMode ) { - if (!registeredCaches.containsKey(cacheName)) - registeredCaches.put(cacheName, new CachePredicate(filter, nearEnabled, cacheMode)); + if (!registeredCachesPreds.containsKey(cacheName)) + registeredCachesPreds.put(cacheName, new CachePredicate(filter, nearEnabled, cacheMode)); } /** * Removes dynamic cache filter. * * @param cacheName Cache name. + * @param force Removes dynamic cache filter without check that it was previously created. */ - public void removeCacheFilter(String cacheName) { - CachePredicate p = registeredCaches.remove(cacheName); + public void removeCacheFilter(String cacheName, boolean force) { + CachePredicate p = registeredCachesPreds.remove(cacheName); - assert p != null : cacheName; + assert force || p != null : cacheName; } /** @@ -336,7 +337,7 @@ public void removeCacheFilter(String cacheName) { * @return {@code True} if new node ID was added. */ public boolean addClientNode(String cacheName, UUID clientNodeId, boolean nearEnabled) { - CachePredicate p = registeredCaches.get(cacheName); + CachePredicate p = registeredCachesPreds.get(cacheName); assert p != null : cacheName; @@ -351,7 +352,7 @@ public boolean addClientNode(String cacheName, UUID clientNodeId, boolean nearEn * @return {@code True} if existing node ID was removed. */ public boolean onClientCacheClose(String cacheName, UUID clientNodeId) { - CachePredicate p = registeredCaches.get(cacheName); + CachePredicate p = registeredCachesPreds.get(cacheName); assert p != null : cacheName; @@ -364,12 +365,12 @@ public boolean onClientCacheClose(String cacheName, UUID clientNodeId) { public Map> clientNodesMap() { Map> res = null; - for (Map.Entry entry : registeredCaches.entrySet()) { + for (Map.Entry entry : registeredCachesPreds.entrySet()) { CachePredicate pred = entry.getValue(); if (!F.isEmpty(pred.clientNodes)) { if (res == null) - res = U.newHashMap(registeredCaches.size()); + res = U.newHashMap(registeredCachesPreds.size()); res.put(entry.getKey(), new HashMap<>(pred.clientNodes)); } @@ -382,7 +383,7 @@ public Map> clientNodesMap() { * @param leftNodeId Left node ID. */ private void updateClientNodes(UUID leftNodeId) { - for (Map.Entry entry : registeredCaches.entrySet()) { + for (Map.Entry entry : registeredCachesPreds.entrySet()) { CachePredicate pred = entry.getValue(); pred.onNodeLeft(leftNodeId); @@ -602,7 +603,7 @@ else if (type == EVT_CLIENT_NODE_DISCONNECTED) { locJoin = new GridFutureAdapter<>(); - registeredCaches.clear(); + registeredCachesPreds.clear(); for (AffinityTopologyVersion histVer : discoCacheHist.keySet()) { Object rmvd = discoCacheHist.remove(histVer); @@ -1718,7 +1719,7 @@ public Collection cacheAffinityNodes(int cacheId, AffinityTopologyV * @return {@code True} if node is a cache data node. */ public boolean cacheAffinityNode(ClusterNode node, String cacheName) { - CachePredicate pred = registeredCaches.get(cacheName); + CachePredicate pred = registeredCachesPreds.get(cacheName); return pred != null && pred.dataNode(node); } @@ -1729,7 +1730,7 @@ public boolean cacheAffinityNode(ClusterNode node, String cacheName) { * @return {@code True} if node has near cache enabled. */ public boolean cacheNearNode(ClusterNode node, String cacheName) { - CachePredicate pred = registeredCaches.get(cacheName); + CachePredicate pred = registeredCachesPreds.get(cacheName); return pred != null && pred.nearNode(node); } @@ -1740,7 +1741,7 @@ public boolean cacheNearNode(ClusterNode node, String cacheName) { * @return {@code True} if node has client cache (without near cache). */ public boolean cacheClientNode(ClusterNode node, String cacheName) { - CachePredicate pred = registeredCaches.get(cacheName); + CachePredicate pred = registeredCachesPreds.get(cacheName); return pred != null && pred.clientNode(node); } @@ -1751,7 +1752,7 @@ public boolean cacheClientNode(ClusterNode node, String cacheName) { * @return If cache with the given name is accessible on the given node. */ public boolean cacheNode(ClusterNode node, String cacheName) { - CachePredicate pred = registeredCaches.get(cacheName); + CachePredicate pred = registeredCachesPreds.get(cacheName); return pred != null && pred.cacheNode(node); } @@ -1761,9 +1762,9 @@ public boolean cacheNode(ClusterNode node, String cacheName) { * @return Public cache names accessible on the given node. */ public Map nodeCaches(ClusterNode node) { - Map caches = U.newHashMap(registeredCaches.size()); + Map caches = U.newHashMap(registeredCachesPreds.size()); - for (Map.Entry entry : registeredCaches.entrySet()) { + for (Map.Entry entry : registeredCachesPreds.entrySet()) { String cacheName = entry.getKey(); CachePredicate pred = entry.getValue(); @@ -2012,7 +2013,7 @@ public void reconnect() { assert node.order() != 0 : "Invalid node order [locNode=" + loc + ", node=" + node + ']'; assert !node.isDaemon(); - for (Map.Entry entry : registeredCaches.entrySet()) { + for (Map.Entry entry : registeredCachesPreds.entrySet()) { String cacheName = entry.getKey(); CachePredicate filter = entry.getValue(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java index 84372febb2c91..fc9e02ce294ca 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java @@ -322,6 +322,77 @@ public void onCacheCreated(GridCacheContext cctx) { } } + /** + * Called during exchange rollback in order to stop the given cache(s) + * even if it's not fully initialized (e.g. fail on cache init stage). + * + * @param fut Exchange future. + * @param crd Coordinator flag. + * @param reqs Cache change requests. + */ + public void forceCloseCache(final GridDhtPartitionsExchangeFuture fut, boolean crd, + Collection reqs) { + + assert !F.isEmpty(reqs) : fut; + + for (DynamicCacheChangeRequest req : reqs) { + assert req.stop() : req; + + Integer cacheId = CU.cacheId(req.cacheName()); + + registeredCaches.remove(cacheId); + } + + Set stoppedCaches = null; + + for (DynamicCacheChangeRequest req : reqs) { + Integer cacheId = CU.cacheId(req.cacheName()); + + cctx.cache().blockGateway(req); + + if (crd) { + CacheHolder cache = caches.remove(cacheId); + + if (cache != null) { + if (stoppedCaches == null) + stoppedCaches = new HashSet<>(); + + stoppedCaches.add(cache.cacheId()); + + cctx.io().removeHandler(cacheId, GridDhtAffinityAssignmentResponse.class); + } + } + } + + if (stoppedCaches != null) { + boolean notify = false; + + synchronized (mux) { + if (waitInfo != null) { + for (Integer cacheId : stoppedCaches) { + boolean rmv = waitInfo.waitCaches.remove(cacheId) != null; + + if (rmv) { + notify = true; + + waitInfo.assignments.remove(cacheId); + } + } + } + } + + if (notify) { + final AffinityTopologyVersion topVer = affCalcVer; + + cctx.kernalContext().closure().runLocalSafe(new Runnable() { + @Override public void run() { + onCacheStopped(topVer); + } + }); + } + } + } + /** * Called on exchange initiated for cache start/stop request. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeFailureMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeFailureMessage.java new file mode 100644 index 0000000000000..162c67351cb32 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeFailureMessage.java @@ -0,0 +1,129 @@ +/* + * 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.util.Collection; +import java.util.Map; +import java.util.UUID; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionExchangeId; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.lang.IgniteUuid; +import org.jetbrains.annotations.Nullable; + +/** + * This class represents discovery message that is used to provide information about dynamic cache start failure. + */ +public class DynamicCacheChangeFailureMessage implements DiscoveryCustomMessage { + /** */ + private static final long serialVersionUID = 0L; + + /** Change requests. */ + @GridToStringInclude + private Collection reqs; + + /** Custom message ID. */ + private IgniteUuid id; + + /** */ + private GridDhtPartitionExchangeId exchId; + + /** */ + private GridCacheVersion lastVer; + + @GridToStringInclude + private IgniteCheckedException cause; + + /** + * Creates new DynamicCacheChangeFailureMessage instance. + * + * @param exchId Exchange Id. + * @param lastVer Last version. + * @param cause + * @param reqs Cache change requests. + */ + public DynamicCacheChangeFailureMessage( + ClusterNode localNode, + GridDhtPartitionExchangeId exchId, + @Nullable GridCacheVersion lastVer, + IgniteCheckedException cause, + Collection reqs) + { + this.id = IgniteUuid.fromUuid(localNode.id()); + this.exchId = exchId; + this.lastVer = lastVer; + this.cause = cause; + this.reqs = reqs; + } + + /** {@inheritDoc} */ + @Override public IgniteUuid id() { + return id; + } + + /** + * @param id Message ID. + */ + public void id(IgniteUuid id) { + this.id = id; + } + + /** + * @return Collection of change requests. + */ + public Collection requests() { + return reqs; + } + + public IgniteCheckedException getError() { + return cause; + } + + /** + * @return Last used version among all nodes. + */ + @Nullable public GridCacheVersion lastVersion() { + return lastVer; + } + + /** + * @return Exchange version. + */ + @Nullable public GridDhtPartitionExchangeId exchangeId() { + return exchId; + } + + /** {@inheritDoc} */ + @Nullable @Override public DiscoveryCustomMessage ackMessage() { + return null; + } + + /** {@inheritDoc} */ + @Override public boolean isMutable() { + return false; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(DynamicCacheChangeFailureMessage.class, this); + } +} 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 c62ffd2aec79c..a994bd46f1f71 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,17 @@ else if (customEvt.customMessage() instanceof CacheAffinityChangeMessage) { exchFut = exchangeFuture(exchId, evt, cache, null, msg); } } - else if (msg.exchangeId().topologyVersion().topologyVersion() >= affinityTopologyVersion(cctx.discovery().localJoinEvent()).topologyVersion()) - exchangeFuture(msg.exchangeId(), null, null, null, null).onAffinityChangeMessage(customEvt.eventNode(), msg); + else if (msg.exchangeId().topologyVersion().topologyVersion() >= + affinityTopologyVersion(cctx.discovery().localJoinEvent()).topologyVersion()) + exchangeFuture(msg.exchangeId(), null, null, null, null) + .onAffinityChangeMessage(customEvt.eventNode(), msg); + } + else if (customEvt.customMessage() instanceof DynamicCacheChangeFailureMessage) { + DynamicCacheChangeFailureMessage msg = + (DynamicCacheChangeFailureMessage)customEvt.customMessage(); + + exchangeFuture(msg.exchangeId(), null, null, null, null) + .onDynamicCacheChangeFail(customEvt.eventNode(), msg); } } 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 35074e9ba7cc7..2fd181dba75b8 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 @@ -1747,6 +1747,20 @@ public void blockGateway(DynamicCacheChangeRequest req) { } } + /** + * Called during exchange rollback in order to stop the given cache + * even if it's not fully initialized (e.g. fail on cache init stage). + */ + public void forceCloseCache(DynamicCacheChangeRequest req) { + assert req.stop() : req; + + registeredCaches.remove(maskNull(req.cacheName())); + + stopGateway(req); + + prepareCacheStop(req, true); + } + /** * @param req Request. */ @@ -1850,7 +1864,17 @@ else if (req.close() && req.initiatingNodeId().equals(ctx.localNodeId()) || forc } } } + } + } + } + /** + * @param reqs Collection of requests to complete future for. + * @param err Error to be passed to futures. + */ + public void completeStartFutures(Collection reqs, @Nullable Throwable err) { + if (!F.isEmpty(reqs)) { + for (DynamicCacheChangeRequest req : reqs) { completeStartFuture(req, err); } } @@ -2597,10 +2621,26 @@ public boolean onCustomEvent(DiscoveryCustomMessage msg, AffinityTopologyVersion topVer) { if (msg instanceof CacheAffinityChangeMessage) return sharedCtx.affinity().onCustomEvent(((CacheAffinityChangeMessage)msg)); + else if (msg instanceof DynamicCacheChangeFailureMessage) + return onCacheChangeRequested((DynamicCacheChangeFailureMessage)msg); return msg instanceof DynamicCacheChangeBatch && onCacheChangeRequested((DynamicCacheChangeBatch)msg, topVer); } + /** + * @param failMsg Dynamic change change request fail message. + * @return {@code True} if minor topology version should be increased. + */ + private boolean onCacheChangeRequested(DynamicCacheChangeFailureMessage failMsg) { + for (DynamicCacheChangeRequest req : failMsg.requests()) { + registeredCaches.remove(maskNull(req.cacheName())); + + ctx.discovery().removeCacheFilter(req.cacheName(), true); + } + + return false; + } + /** * @param batch Change request batch. * @param topVer Current topology version. @@ -2738,7 +2778,7 @@ private boolean onCacheChangeRequested(DynamicCacheChangeBatch batch, assert old != null : "Dynamic cache map was concurrently modified [req=" + req + ']'; - ctx.discovery().removeCacheFilter(req.cacheName()); + ctx.discovery().removeCacheFilter(req.cacheName(), false); needExchange = true; } 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 d3e3701936786..6a7145442f49e 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 @@ -51,6 +51,7 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.affinity.GridAffinityAssignmentCache; import org.apache.ignite.internal.processors.cache.CacheAffinityChangeMessage; +import org.apache.ignite.internal.processors.cache.DynamicCacheChangeFailureMessage; import org.apache.ignite.internal.processors.cache.DynamicCacheChangeRequest; import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor; import org.apache.ignite.internal.processors.cache.GridCacheContext; @@ -74,6 +75,7 @@ import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteInClosure; +import org.apache.ignite.lang.IgniteProductVersion; import org.apache.ignite.lang.IgniteRunnable; import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentHashMap8; @@ -82,6 +84,7 @@ import static org.apache.ignite.events.EventType.EVT_NODE_FAILED; import static org.apache.ignite.events.EventType.EVT_NODE_JOINED; import static org.apache.ignite.events.EventType.EVT_NODE_LEFT; +import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_EXCHANGE_ROLLBACK_SUPPORTED; import static org.apache.ignite.internal.events.DiscoveryCustomEvent.EVT_DISCOVERY_CUSTOM_EVT; import static org.apache.ignite.internal.managers.communication.GridIoPolicy.SYSTEM_POOL; @@ -161,10 +164,10 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter singleMsgs = new ConcurrentHashMap8<>(); + private Map singleMsgs = new ConcurrentHashMap8<>(); /** Messages received from new coordinator. */ - private final Map fullMsgs = new ConcurrentHashMap8<>(); + private Map fullMsgs = new ConcurrentHashMap8<>(); /** */ @SuppressWarnings({"FieldCanBeLocal", "UnusedDeclaration"}) @@ -198,6 +201,12 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter exchangeGlobalExceptions = new ConcurrentHashMap8<>(); + /** Forced Rebalance future. */ private GridCompoundFuture forcedRebFut; @@ -536,11 +545,46 @@ public void init() throws IgniteInterruptedCheckedException { catch (IgniteNeedReconnectException e) { onDone(e); } + catch (Exception e) { + if (reconnectOnError(e)) { + onDone(new IgniteNeedReconnectException(cctx.localNode(), e)); + + return; + } + + U.error(log, "Failed to reinitialize local partitions (rebalancing will be stopped): " + exchId, e); + + if (cctx.kernalContext().clientNode() || !isRollbackSupported()) { + onDone(e); + + return; + } + + exchangeLocalException = new IgniteCheckedException( + "Failed to initialize exchange locally [locNodeId=" + cctx.localNodeId() + "]", e); + + exchangeGlobalExceptions.put(cctx.localNodeId(), exchangeLocalException); + + if (crd.isLocal()) { + boolean isRemainingEmpty = false; + + synchronized (mux) { + isRemainingEmpty = remaining.isEmpty(); + } + + if (isRemainingEmpty) + onAllReceived(false); + } + else + sendPartitions(crd); + + initDone(); + } catch (Throwable 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); + U.error(log, "Failed to reinitialize local partitions (rebalancing will be stopped): " + exchId, e); onDone(e); } @@ -977,6 +1021,9 @@ private void sendLocalPartitions(ClusterNode node) clientOnlyExchange, true); + if (exchangeLocalException != null) + m.setError(exchangeLocalException); + if (log.isDebugEnabled()) log.debug("Sending local partitions [nodeId=" + node.id() + ", exchId=" + exchId + ", msg=" + m + ']'); @@ -1084,6 +1131,8 @@ private void sendPartitions(ClusterNode oldestNode) { cctx.cache().onExchangeDone(exchId.topologyVersion(), reqs, err, false); + cctx.cache().completeStartFutures(reqs, err); + if (super.onDone(res, err) && realExchange) { if (log.isDebugEnabled()) log.debug("Completed partition exchange [localNode=" + cctx.localNodeId() + ", exchange= " + this + @@ -1130,10 +1179,12 @@ private void sendPartitions(ClusterNode oldestNode) { * Cleans up resources to avoid excessive memory usage. */ public void cleanUp() { - singleMsgs.clear(); - fullMsgs.clear(); + singleMsgs = null; + fullMsgs = null; crd = null; partReleaseFut = null; + exchangeLocalException = null; + exchangeGlobalExceptions = null; } /** @@ -1210,6 +1261,9 @@ private void processMessage(ClusterNode node, GridDhtPartitionsSingleMessage msg pendingSingleUpdates++; + if (msg.getError() != null) + exchangeGlobalExceptions.put(node.id(), msg.getError()); + allReceived = remaining.isEmpty(); } } @@ -1278,6 +1332,82 @@ private void onAffinityInitialized(IgniteInternalFuture globalExceptions) { + IgniteCheckedException ex; + + if (exchangeLocalException != null) + ex = exchangeLocalException; + else + ex = new IgniteCheckedException("Failed to complete exchange process (will try to rollback)."); + + for (Map.Entry entry : globalExceptions.entrySet()) { + // avoid self-suppression + if (ex != entry.getValue()) + ex.addSuppressed(entry.getValue()); + } + + return ex; + } + + /** + * Returns {@code true} if the given {@code discoEvt} supports the rollback procedure. + * + * @return {@code true} if the given {@code discoEvt} supports the rollback procedure. + */ + private boolean isRollbackSupported() { + boolean rollbackSupported = false; + + for (ClusterNode node : discoCache.allNodes()) { + Boolean exchangeSupported = node.attribute(ATTR_EXCHANGE_ROLLBACK_SUPPORTED); + if (exchangeSupported == null || !exchangeSupported) + return false; + } + + // Currently the rollback process is supported for dynamically started caches. + if (discoEvt.type() == EVT_DISCOVERY_CUSTOM_EVT && !F.isEmpty(reqs)) { + for (DynamicCacheChangeRequest req : reqs) { + if (req.start()) { + rollbackSupported = true; + + break; + } + } + } + + return rollbackSupported; + } + + /** + * Tries to revert all the changes that were done during initialization phase + * in case of the given {@code discoEvt} supports the rollback procedure. + */ + private void rollbackExchange() { + if (discoEvt.type() == EVT_DISCOVERY_CUSTOM_EVT && !F.isEmpty(reqs)) { + for (DynamicCacheChangeRequest req : reqs) { + if (req.start()) { + DynamicCacheChangeRequest stopReq = + new DynamicCacheChangeRequest(req.cacheName(), cctx.localNodeId()); + + stopReq.stop(true); + stopReq.deploymentId(req.deploymentId()); + + // cleanup GridCacheProcessor + cctx.cache().forceCloseCache(stopReq); + + // cleanup CacheAffinitySharedManager + cctx.affinity().forceCloseCache(this, crd.isLocal(), Collections.singletonList(stopReq)); + } + } + } + } + /** * @param discoThread If {@code true} completes future from another thread (to do not block discovery thread). */ @@ -1285,6 +1415,24 @@ private void onAllReceived(boolean discoThread) { try { assert crd.isLocal(); + if (!F.isEmpty(exchangeGlobalExceptions) && isRollbackSupported()) { + updateLastVersion(cctx.versions().last()); + + cctx.versions().onExchange(lastVer.get().order()); + + IgniteCheckedException err = createExchangeException(exchangeGlobalExceptions); + + DynamicCacheChangeFailureMessage msg = new DynamicCacheChangeFailureMessage( + cctx.localNode(), exchId, lastVer.get(), err, reqs); + + if (log.isDebugEnabled()) + log.debug("Dynamic cache change failed. Send message to all participating nodes: " + msg); + + cctx.discovery().sendCustomEvent(msg); + + return; + } + if (!crd.equals(discoCache.serverNodes().get(0))) { for (GridCacheContext cacheCtx : cctx.cacheContexts()) { if (!cacheCtx.isLocal()) @@ -1488,6 +1636,37 @@ private void updatePartitionSingleMap(GridDhtPartitionsSingleMessage msg) { } } + /** + * Cache change failure message callback, processed from the discovery thread. + * + * @param node Message sender node. + * @param msg Message. + */ + public void onDynamicCacheChangeFail(final ClusterNode node, final DynamicCacheChangeFailureMessage msg) { + assert exchId.equals(msg.exchangeId()) : msg; + assert msg.lastVersion() != null : msg; + + onDiscoveryEvent(new IgniteRunnable() { + @Override public void run() { + if (isDone() || !enterBusy()) + return; + + try { + if (isRollbackSupported()) + rollbackExchange(); + + if (!crd.isLocal()) + cctx.versions().onExchange(msg.lastVersion().order()); + + onDone(exchId.topologyVersion(), msg.getError()); + } + finally { + leaveBusy(); + } + } + }); + } + /** * Affinity change message callback, processed from the same thread as {@link #onNodeLeft}. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java index bf08f0abf8832..abc05c0afb83e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java @@ -64,6 +64,14 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes /** Serialized partitions counters. */ private byte[] partCntrsBytes; + /** Exception. */ + @GridToStringInclude + @GridDirectTransient + private Exception err; + + /** */ + private byte[] errBytes; + /** */ private boolean client; @@ -149,6 +157,20 @@ public Map partitionUpdateCounters(int cacheId) { return Collections.emptyMap(); } + /** + * @param ex Exception. + */ + public void setError(Exception ex) { + this.err = ex; + } + + /** + * @return Not null exception if exchange processing failed. + */ + @Nullable public Exception getError() { + return err; + } + /** * @return Local partitions. */ @@ -156,8 +178,7 @@ public Map partitions() { return parts; } - /** {@inheritDoc} - * @param ctx*/ + /** {@inheritDoc} */ @Override public void prepareMarshal(GridCacheSharedContext ctx) throws IgniteCheckedException { super.prepareMarshal(ctx); @@ -166,6 +187,7 @@ public Map partitions() { if (marshal) { byte[] partsBytes0 = null; byte[] partCntrsBytes0 = null; + byte[] errBytes0 = null; if (parts != null && partsBytes == null) partsBytes0 = U.marshal(ctx, parts); @@ -173,15 +195,20 @@ public Map partitions() { if (partCntrs != null && partCntrsBytes == null) partCntrsBytes0 = U.marshal(ctx, partCntrs); + if (err != null && errBytes == null) + errBytes0 = U.marshal(ctx, err); + if (compress) { assert !compressed(); try { byte[] partsBytesZip = U.zip(partsBytes0); byte[] partCntrsBytesZip = U.zip(partCntrsBytes0); + byte[] exBytesZip = U.zip(errBytes0); partsBytes0 = partsBytesZip; partCntrsBytes0 = partCntrsBytesZip; + errBytes0 = exBytesZip; compressed(true); } @@ -192,6 +219,7 @@ public Map partitions() { partsBytes = partsBytes0; partCntrsBytes = partCntrsBytes0; + errBytes = errBytes0; } } @@ -213,6 +241,13 @@ public Map partitions() { partCntrs = U.unmarshal(ctx, partCntrsBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); } + if (errBytes != null && err == null) { + if (compressed()) + err = U.unmarshalZip(ctx.marshaller(), errBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + else + err = U.unmarshal(ctx, errBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + } + if (dupPartsData != null) { assert parts != null; @@ -262,12 +297,18 @@ public Map partitions() { writer.incrementState(); case 8: - if (!writer.writeByteArray("partCntrsBytes", partCntrsBytes)) + if (!writer.writeByteArray("errBytes", errBytes)) return false; writer.incrementState(); case 9: + if (!writer.writeByteArray("partCntrsBytes", partCntrsBytes)) + return false; + + writer.incrementState(); + + case 10: if (!writer.writeByteArray("partsBytes", partsBytes)) return false; @@ -306,7 +347,7 @@ public Map partitions() { reader.incrementState(); case 8: - partCntrsBytes = reader.readByteArray("partCntrsBytes"); + errBytes = reader.readByteArray("errBytes"); if (!reader.isLastRead()) return false; @@ -314,6 +355,14 @@ public Map partitions() { reader.incrementState(); case 9: + partCntrsBytes = reader.readByteArray("partCntrsBytes"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 10: partsBytes = reader.readByteArray("partsBytes"); if (!reader.isLastRead()) @@ -333,7 +382,7 @@ public Map partitions() { /** {@inheritDoc} */ @Override public byte fieldsCount() { - return 10; + return 11; } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartFailSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartFailSelfTest.java new file mode 100644 index 0000000000000..fba54aa80ed92 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartFailSelfTest.java @@ -0,0 +1,421 @@ +/* + * 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.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.Callable; +import javax.cache.CacheException; +import javax.cache.configuration.Factory; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.affinity.AffinityFunctionContext; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; +import org.apache.ignite.cache.store.CacheStore; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.events.EventType; +import org.apache.ignite.internal.IgniteNodeAttributes; +import org.apache.ignite.lang.IgnitePredicate; +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.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +@SuppressWarnings("unchecked") +public class IgniteDynamicCacheStartFailSelfTest extends GridCommonAbstractTest { + /** */ + private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); + + /** */ + private static final String DYNAMIC_CACHE_NAME = "TestDynamicCache"; + + /** Coordinator node. */ + private Ignite crd; + + /** Coordinator node index. */ + private int crdIdx; + + /** + * @return Number of nodes for this test. + */ + public int nodeCount() { + return 3; + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + crdIdx = 0; + crd = startGrid(crdIdx); + + for (int i = 1; i < nodeCount(); ++i) + startGrid(i); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + } + + public void testBrokenAffinityFunctionOnAllNodes() { + final boolean failOnAllNodes = true; + final int unluckyNode = 0; + final int unluckyCfg = 1; + final int numberOfCaches = 3; + final int initiator = 0; + + testDynamicCacheStart( + createCacheConfigsWithBrokenAffinityFunction( + failOnAllNodes, unluckyNode, unluckyCfg, numberOfCaches, false), + initiator); + } + + public void testBrokenAffinityFunctionOnInitiator() { + final boolean failOnAllNodes = false; + final int unluckyNode = 1; + final int unluckyCfg = 1; + final int numberOfCaches = 3; + final int initiator = 1; + + testDynamicCacheStart( + createCacheConfigsWithBrokenAffinityFunction( + failOnAllNodes, unluckyNode, unluckyCfg, numberOfCaches, false), + initiator); + } + + public void testBrokenAffinityFunctionOnNonInitiator() { + final boolean failOnAllNodes = false; + final int unluckyNode = 1; + final int unluckyCfg = 1; + final int numberOfCaches = 3; + final int initiator = 2; + + testDynamicCacheStart( + createCacheConfigsWithBrokenAffinityFunction( + failOnAllNodes, unluckyNode, unluckyCfg, numberOfCaches, false), + initiator); + } + + public void testBrokenAffinityFunctionOnCoordinatorDiffInitiator() { + final boolean failOnAllNodes = false; + final int unluckyNode = crdIdx; + final int unluckyCfg = 1; + final int numberOfCaches = 3; + final int initiator = (crdIdx + 1) % nodeCount(); + + testDynamicCacheStart( + createCacheConfigsWithBrokenAffinityFunction( + failOnAllNodes, unluckyNode, unluckyCfg, numberOfCaches, false), + initiator); + } + + public void testBrokenAffinityFunctionOnCoordinator() { + final boolean failOnAllNodes = false; + final int unluckyNode = crdIdx; + final int unluckyCfg = 1; + final int numberOfCaches = 3; + final int initiator = crdIdx; + + testDynamicCacheStart( + createCacheConfigsWithBrokenAffinityFunction( + failOnAllNodes, unluckyNode, unluckyCfg, numberOfCaches, false), + initiator); + } + + public void testBrokenAffinityFunctionWithNodeFilter() { + final boolean failOnAllNodes = false; + final int unluckyNode = 0; + final int unluckyCfg = 0; + final int numberOfCaches = 1; + final int initiator = 0; + + testDynamicCacheStart( + createCacheConfigsWithBrokenAffinityFunction( + failOnAllNodes, unluckyNode, unluckyCfg, numberOfCaches, true), + initiator); + } + + public void testBrokenCacheStoreOnAllNodes() { + final boolean failOnAllNodes = true; + final int unluckyNode = 0; + final int unluckyCfg = 1; + final int numberOfCaches = 3; + final int initiator = 0; + + testDynamicCacheStart( + createCacheConfigsWithBrokenCacheStore( + failOnAllNodes, unluckyNode, unluckyCfg, numberOfCaches, false), + initiator); + } + + public void testBrokenCacheStoreOnInitiator() { + final boolean failOnAllNodes = false; + final int unluckyNode = 1; + final int unluckyCfg = 1; + final int numberOfCaches = 3; + final int initiator = 1; + + testDynamicCacheStart( + createCacheConfigsWithBrokenCacheStore( + failOnAllNodes, unluckyNode, unluckyCfg, numberOfCaches, false), + initiator); + } + + public void testBrokenCacheStoreOnNonInitiator() { + final boolean failOnAllNodes = false; + final int unluckyNode = 1; + final int unluckyCfg = 1; + final int numberOfCaches = 3; + final int initiator = 2; + + testDynamicCacheStart( + createCacheConfigsWithBrokenCacheStore( + failOnAllNodes, unluckyNode, unluckyCfg, numberOfCaches, false), + initiator); + } + + public void testBrokenCacheStoreOnCoordinatorDiffInitiator() { + final boolean failOnAllNodes = false; + final int unluckyNode = crdIdx; + final int unluckyCfg = 1; + final int numberOfCaches = 3; + final int initiator = (crdIdx + 1) % nodeCount(); + + testDynamicCacheStart( + createCacheConfigsWithBrokenCacheStore( + failOnAllNodes, unluckyNode, unluckyCfg, numberOfCaches, false), + initiator); + } + + public void testBrokenCacheStoreFunctionOnCoordinator() { + final boolean failOnAllNodes = false; + final int unluckyNode = crdIdx; + final int unluckyCfg = 1; + final int numberOfCaches = 3; + final int initiator = crdIdx; + + testDynamicCacheStart( + createCacheConfigsWithBrokenCacheStore( + failOnAllNodes, unluckyNode, unluckyCfg, numberOfCaches, false), + initiator); + } + + public void testCreateCacheMultipleTimes() { + final boolean failOnAllNodes = false; + final int unluckyNode = 1; + final int unluckyCfg = 0; + final int numOfAttempts = 100; + + CacheConfiguration cfg = createCacheConfigsWithBrokenAffinityFunction( + failOnAllNodes, unluckyNode, unluckyCfg, 1, false).get(0); + + for (int i = 0; i < numOfAttempts; ++i) { + try { + IgniteCache cache = ignite(0).getOrCreateCache(cfg); + + fail("Expected exception was not thrown"); + } + catch (CacheException e) { + } + } + } + + private List createCacheConfigsWithBrokenAffinityFunction( + boolean failOnAllNodes, + int unluckyNode, + final int unluckyCfg, + int cacheNum, + boolean useFilter + ) { + assert unluckyCfg >= 0 && unluckyCfg < cacheNum; + + final UUID uuid = ignite(unluckyNode).cluster().localNode().id(); + + List cfgs = new ArrayList<>(); + + for (int i = 0; i < cacheNum; ++i) { + CacheConfiguration cfg = new CacheConfiguration(); + + cfg.setName(DYNAMIC_CACHE_NAME + "-" + i); + + if (i == unluckyCfg) + cfg.setAffinity(new BrokenAffinityFunction(failOnAllNodes, getTestGridName(unluckyNode))); + + if (useFilter) + cfg.setNodeFilter(new NodeFilter(uuid)); + + cfgs.add(cfg); + } + + return cfgs; + } + + private Collection createCacheConfigsWithBrokenCacheStore( + boolean failOnAllNodes, + int unluckyNode, + int unluckyCfg, + int cacheNum, + boolean useFilter + ) { + assert unluckyCfg >= 0 && unluckyCfg < cacheNum; + + final UUID uuid = ignite(unluckyNode).cluster().localNode().id(); + + List cfgs = new ArrayList<>(); + + for (int i = 0; i < cacheNum; ++i) { + CacheConfiguration cfg = new CacheConfiguration(); + + cfg.setName(DYNAMIC_CACHE_NAME + "-" + i); + + if (i == unluckyCfg) + cfg.setCacheStoreFactory(new BrokenStoreFactory(failOnAllNodes, getTestGridName(unluckyNode))); + + if (useFilter) + cfg.setNodeFilter(new NodeFilter(uuid)); + + cfgs.add(cfg); + } + + return cfgs; + } + + private void testDynamicCacheStart(final Collection cfgs, final int initiatorId) { + assert initiatorId < nodeCount(); + + GridTestUtils.assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + grid(initiatorId).getOrCreateCaches(cfgs); + return null; + } + }, CacheException.class, null); + + for (CacheConfiguration cfg : cfgs) + assertNull("initiatorId=" + initiatorId, grid(initiatorId).cache(cfg.getName())); + } + + private static class NodeFilter implements IgnitePredicate { + /** Cache should be created node with certain UUID. */ + public UUID uuid; + + public NodeFilter() { + } + + public NodeFilter(UUID uuid) { + this.uuid = uuid; + } + + /** {@inheritDoc} */ + @Override public boolean apply(ClusterNode clusterNode) { + return clusterNode.id().equals(uuid); + } + } + + private static class BrokenAffinityFunction extends RendezvousAffinityFunction { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + @IgniteInstanceResource + private Ignite ignite; + + /** Exception should arise on all nodes. */ + private boolean exceptionOnAllNodes = false; + + /** Exception should arise on node with certain name. */ + private String gridName; + + public BrokenAffinityFunction() { + } + + public BrokenAffinityFunction(boolean exceptionOnAllNodes, String gridName) { + this.exceptionOnAllNodes = exceptionOnAllNodes; + this.gridName = gridName; + } + + /** {@inheritDoc} */ + @Override public List> assignPartitions(AffinityFunctionContext affCtx) { + if (exceptionOnAllNodes || ignite.name().equals(gridName)) + throw new IllegalStateException("Simulated exception [locNodeId=" + + ignite.cluster().localNode().id() + "]"); + else + return super.assignPartitions(affCtx); + } + + /** {@inheritDoc} */ + @Override public void writeExternal(ObjectOutput out) throws IOException { + super.writeExternal(out); + out.writeBoolean(exceptionOnAllNodes); + out.writeObject(gridName); + } + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { + super.readExternal(in); + exceptionOnAllNodes = in.readBoolean(); + gridName = (String)in.readObject(); + } + } + + private static class BrokenStoreFactory implements Factory> { + /** */ + @IgniteInstanceResource + private Ignite ignite; + + /** Exception should arise on all nodes. */ + boolean exceptionOnAllNodes = true; + + /** Exception should arise on node with certain name. */ + public static String gridName; + + public BrokenStoreFactory() { + } + + public BrokenStoreFactory(boolean exceptionOnAllNodes, String gridName) { + this.exceptionOnAllNodes = exceptionOnAllNodes; + this.gridName = gridName; + } + + /** {@inheritDoc} */ + @Override public CacheStore create() { + if (exceptionOnAllNodes || ignite.name().equals(gridName)) + throw new IllegalStateException("Simulated exception [locNodeId=" + + ignite.cluster().localNode().id() + "]"); + else + return null; + } + } +} From 07e211df9589e037e690c4d0b67034287cfd1317 Mon Sep 17 00:00:00 2001 From: sboikov Date: Thu, 24 Aug 2017 12:12:22 +0300 Subject: [PATCH 252/357] IGNITE-1094: final review --- .../managers/discovery/DiscoCache.java | 19 ++ .../discovery/GridDiscoveryManager.java | 5 +- .../cache/CacheAffinitySharedManager.java | 93 ++---- .../DynamicCacheChangeFailureMessage.java | 51 ++- .../GridCachePartitionExchangeManager.java | 11 +- .../processors/cache/GridCacheProcessor.java | 77 ++--- .../GridDhtPartitionsExchangeFuture.java | 126 +++++--- .../GridDhtPartitionsSingleMessage.java | 4 +- .../IgniteDynamicCacheStartFailSelfTest.java | 291 ++++++++++++++---- .../testsuites/IgniteCacheTestSuite4.java | 2 + 10 files changed, 420 insertions(+), 259 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoCache.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoCache.java index 5247ac1b28091..30d2b7d9a5c07 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoCache.java @@ -295,6 +295,25 @@ public void updateAlives(GridDiscoveryManager discovery) { } } + /** + * Returns {@code True} if all nodes has the given attribute and its value equals to {@code expVal}. + * + * @param Attribute Type. + * @param name Attribute name. + * @param expVal Expected value. + * @return {@code True} if all the given nodes has the given attribute and its value equals to {@code expVal}. + */ + public boolean checkAttribute(String name, T expVal) { + for (ClusterNode node : allNodes) { + T attr = node.attribute(name); + + if (attr == null || !expVal.equals(attr)) + return false; + } + + return true; + } + /** * @param nodes Cluster nodes. * @return Empty collection if nodes list is {@code null} 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 1d67869bb478e..40dea984ee56d 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 @@ -320,12 +320,11 @@ public void setCacheFilter( * Removes dynamic cache filter. * * @param cacheName Cache name. - * @param force Removes dynamic cache filter without check that it was previously created. */ - public void removeCacheFilter(String cacheName, boolean force) { + public void removeCacheFilter(String cacheName) { CachePredicate p = registeredCachesPreds.remove(cacheName); - assert force || p != null : cacheName; + assert p != null : cacheName; } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java index fc9e02ce294ca..7cbd333907058 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java @@ -322,6 +322,16 @@ public void onCacheCreated(GridCacheContext cctx) { } } + /** + * @param cacheId Cache ID. + * @return {@code True} if cache is in wait list. + */ + private boolean waitCache(int cacheId) { + synchronized (mux) { + return waitInfo != null && waitInfo.waitCaches.containsKey(cacheId); + } + } + /** * Called during exchange rollback in order to stop the given cache(s) * even if it's not fully initialized (e.g. fail on cache init stage). @@ -332,7 +342,6 @@ public void onCacheCreated(GridCacheContext cctx) { */ public void forceCloseCache(final GridDhtPartitionsExchangeFuture fut, boolean crd, Collection reqs) { - assert !F.isEmpty(reqs) : fut; for (DynamicCacheChangeRequest req : reqs) { @@ -341,6 +350,8 @@ public void forceCloseCache(final GridDhtPartitionsExchangeFuture fut, boolean c Integer cacheId = CU.cacheId(req.cacheName()); registeredCaches.remove(cacheId); + + assert !waitCache(cacheId); } Set stoppedCaches = null; @@ -363,34 +374,6 @@ public void forceCloseCache(final GridDhtPartitionsExchangeFuture fut, boolean c } } } - - if (stoppedCaches != null) { - boolean notify = false; - - synchronized (mux) { - if (waitInfo != null) { - for (Integer cacheId : stoppedCaches) { - boolean rmv = waitInfo.waitCaches.remove(cacheId) != null; - - if (rmv) { - notify = true; - - waitInfo.assignments.remove(cacheId); - } - } - } - } - - if (notify) { - final AffinityTopologyVersion topVer = affCalcVer; - - cctx.kernalContext().closure().runLocalSafe(new Runnable() { - @Override public void run() { - onCacheStopped(topVer); - } - }); - } - } } /** @@ -450,48 +433,38 @@ else if (req.start() && !req.clientStartOnly()) { Integer cacheId = CU.cacheId(req.cacheName()); if (req.start()) { - try { - cctx.cache().prepareCacheStart(req, fut.topologyVersion()); + cctx.cache().prepareCacheStart(req, fut.topologyVersion()); - if (fut.isCacheAdded(cacheId, fut.topologyVersion())) { - if (fut.discoCache().cacheAffinityNodes(req.cacheName()).isEmpty()) - U.quietAndWarn(log, "No server nodes found for cache client: " + req.cacheName()); - } + if (fut.isCacheAdded(cacheId, fut.topologyVersion())) { + if (fut.discoCache().cacheAffinityNodes(req.cacheName()).isEmpty()) + U.quietAndWarn(log, "No server nodes found for cache client: " + req.cacheName()); + } - if (!crd || !lateAffAssign) { - GridCacheContext cacheCtx = cctx.cacheContext(cacheId); + if (!crd || !lateAffAssign) { + GridCacheContext cacheCtx = cctx.cacheContext(cacheId); - if (cacheCtx != null && !cacheCtx.isLocal()) { - boolean clientCacheStarted = - req.clientStartOnly() && req.initiatingNodeId().equals(cctx.localNodeId()); + if (cacheCtx != null && !cacheCtx.isLocal()) { + boolean clientCacheStarted = + req.clientStartOnly() && req.initiatingNodeId().equals(cctx.localNodeId()); - if (clientCacheStarted) - initAffinity(cacheCtx.affinity().affinityCache(), fut, lateAffAssign); - else if (!req.clientStartOnly()) { - assert fut.topologyVersion().equals(cacheCtx.startTopologyVersion()); + if (clientCacheStarted) + initAffinity(cacheCtx.affinity().affinityCache(), fut, lateAffAssign); + else if (!req.clientStartOnly()) { + assert fut.topologyVersion().equals(cacheCtx.startTopologyVersion()); - GridAffinityAssignmentCache aff = cacheCtx.affinity().affinityCache(); + GridAffinityAssignmentCache aff = cacheCtx.affinity().affinityCache(); - assert aff.lastVersion().equals(AffinityTopologyVersion.NONE) : aff.lastVersion(); + assert aff.lastVersion().equals(AffinityTopologyVersion.NONE) : aff.lastVersion(); - List> assignment = aff.calculate(fut.topologyVersion(), - fut.discoveryEvent(), fut.discoCache()); + List> assignment = aff.calculate(fut.topologyVersion(), + fut.discoveryEvent(), fut.discoCache()); - aff.initialize(fut.topologyVersion(), assignment); - } + aff.initialize(fut.topologyVersion(), assignment); } } - else - initStartedCacheOnCoordinator(fut, cacheId); - } - catch (IgniteCheckedException | RuntimeException e) { - U.error(log, "Failed to initialize cache. Will try to rollback cache start routine. " + - "[cacheName=" + req.cacheName() + ']', e); - - cctx.cache().forceCloseCache(fut.topologyVersion(), req, e); - - throw e; } + else + initStartedCacheOnCoordinator(fut, cacheId); } else if (req.stop() || req.close()) { cctx.cache().blockGateway(req); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeFailureMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeFailureMessage.java index 162c67351cb32..c755bb1b31f85 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeFailureMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeFailureMessage.java @@ -18,14 +18,12 @@ package org.apache.ignite.internal.processors.cache; import java.util.Collection; -import java.util.Map; -import java.util.UUID; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionExchangeId; -import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.lang.IgniteUuid; import org.jetbrains.annotations.Nullable; @@ -37,9 +35,9 @@ public class DynamicCacheChangeFailureMessage implements DiscoveryCustomMessage /** */ private static final long serialVersionUID = 0L; - /** Change requests. */ + /** Cache names. */ @GridToStringInclude - private Collection reqs; + private Collection cacheNames; /** Custom message ID. */ private IgniteUuid id; @@ -48,31 +46,31 @@ public class DynamicCacheChangeFailureMessage implements DiscoveryCustomMessage private GridDhtPartitionExchangeId exchId; /** */ - private GridCacheVersion lastVer; - @GridToStringInclude private IgniteCheckedException cause; /** * Creates new DynamicCacheChangeFailureMessage instance. * + * @param locNode Local node. * @param exchId Exchange Id. - * @param lastVer Last version. - * @param cause - * @param reqs Cache change requests. + * @param cause Cache start error. + * @param cacheNames Cache names. */ public DynamicCacheChangeFailureMessage( - ClusterNode localNode, + ClusterNode locNode, GridDhtPartitionExchangeId exchId, - @Nullable GridCacheVersion lastVer, IgniteCheckedException cause, - Collection reqs) + Collection cacheNames) { - this.id = IgniteUuid.fromUuid(localNode.id()); + assert exchId != null; + assert cause != null; + assert !F.isEmpty(cacheNames) : cacheNames; + + this.id = IgniteUuid.fromUuid(locNode.id()); this.exchId = exchId; - this.lastVer = lastVer; this.cause = cause; - this.reqs = reqs; + this.cacheNames = cacheNames; } /** {@inheritDoc} */ @@ -81,30 +79,19 @@ public DynamicCacheChangeFailureMessage( } /** - * @param id Message ID. + * @return Collection of failed caches. */ - public void id(IgniteUuid id) { - this.id = id; + public Collection cacheNames() { + return cacheNames; } /** - * @return Collection of change requests. + * @return Cache start error. */ - public Collection requests() { - return reqs; - } - - public IgniteCheckedException getError() { + public IgniteCheckedException error() { return cause; } - /** - * @return Last used version among all nodes. - */ - @Nullable public GridCacheVersion lastVersion() { - return lastVer; - } - /** * @return Exchange version. */ 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 a994bd46f1f71..4809fd2c0df73 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 @@ -278,8 +278,10 @@ else if (customEvt.customMessage() instanceof DynamicCacheChangeFailureMessage) DynamicCacheChangeFailureMessage msg = (DynamicCacheChangeFailureMessage)customEvt.customMessage(); - exchangeFuture(msg.exchangeId(), null, null, null, null) - .onDynamicCacheChangeFail(customEvt.eventNode(), msg); + if (msg.exchangeId().topologyVersion().topologyVersion() >= + affinityTopologyVersion(cctx.discovery().localJoinEvent()).topologyVersion()) + exchangeFuture(msg.exchangeId(), null, null, null, null) + .onDynamicCacheChangeFail(customEvt.eventNode(), msg); } } @@ -1279,6 +1281,9 @@ private void processSinglePartitionUpdate(final ClusterNode node, final GridDhtP log.debug("Received local partition update [nodeId=" + node.id() + ", parts=" + msg + ']'); + if (msg.partitions() == null) + return; + boolean updated = false; for (Map.Entry entry : msg.partitions().entrySet()) { @@ -1593,7 +1598,7 @@ private void dumpPendingObjects(@Nullable AffinityTopologyVersion exchTopVer) { * @return {@code True} if can use compression for partition map messages. */ @SuppressWarnings("SimplifiableIfStatement") - private boolean canUsePartitionMapCompression(ClusterNode node) { + public boolean canUsePartitionMapCompression(ClusterNode node) { IgniteProductVersion ver = node.version(); if (ver.compareToIgnoreTimestamp(GridDhtPartitionsAbstractMessage.PART_MAP_COMPRESS_SINCE) >= 0) { 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 2fd181dba75b8..673294184a365 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 @@ -1750,12 +1750,12 @@ public void blockGateway(DynamicCacheChangeRequest req) { /** * Called during exchange rollback in order to stop the given cache * even if it's not fully initialized (e.g. fail on cache init stage). + * + * @param req Stop request. */ public void forceCloseCache(DynamicCacheChangeRequest req) { assert req.stop() : req; - registeredCaches.remove(maskNull(req.cacheName())); - stopGateway(req); prepareCacheStop(req, true); @@ -1795,35 +1795,18 @@ private void prepareCacheStop(DynamicCacheChangeRequest req, boolean forceClose) } } - /** - * Closes cache even if it's not fully initialized (e.g. fail on cache init stage). - * - * @param topVer Completed topology version. - * @param req Change request. - * @param err Error. - */ - void forceCloseCache( - AffinityTopologyVersion topVer, - DynamicCacheChangeRequest req, - Throwable err - ) { - onExchangeDone(topVer, Collections.singleton(req), err, true); - } - /** * Callback invoked when first exchange future for dynamic cache is completed. * * @param topVer Completed topology version. * @param reqs Change requests. * @param err Error. - * @param forceClose Close cache despite flags in requests. */ @SuppressWarnings("unchecked") public void onExchangeDone( AffinityTopologyVersion topVer, Collection reqs, - Throwable err, - boolean forceClose + Throwable err ) { for (GridCacheAdapter cache : caches.values()) { GridCacheContext cacheCtx = cache.context(); @@ -1838,43 +1821,35 @@ public void onExchangeDone( } } - if (!F.isEmpty(reqs) && (err == null || forceClose)) { + if (!F.isEmpty(reqs)) { for (DynamicCacheChangeRequest req : reqs) { - String masked = maskNull(req.cacheName()); + if (err == null) { + String masked = maskNull(req.cacheName()); - if (req.stop()) { - stopGateway(req); + if (req.stop()) { + stopGateway(req); - prepareCacheStop(req, forceClose); - } - else if (req.close() && req.initiatingNodeId().equals(ctx.localNodeId()) || forceClose) { - IgniteCacheProxy proxy = jCacheProxies.remove(masked); + prepareCacheStop(req, false); + } + else if (req.close() && req.initiatingNodeId().equals(ctx.localNodeId())) { + IgniteCacheProxy proxy = jCacheProxies.remove(masked); - if (proxy != null) { - if (proxy.context().affinityNode()) { - GridCacheAdapter cache = caches.get(masked); + if (proxy != null) { + if (proxy.context().affinityNode()) { + GridCacheAdapter cache = caches.get(masked); - if (cache != null) - jCacheProxies.put(masked, new IgniteCacheProxy(cache.context(), cache, null, false)); - } - else { - proxy.context().gate().onStopped(); + if (cache != null) + jCacheProxies.put(masked, new IgniteCacheProxy(cache.context(), cache, null, false)); + } + else { + proxy.context().gate().onStopped(); - prepareCacheStop(req, forceClose); + prepareCacheStop(req, false); + } } } } - } - } - } - /** - * @param reqs Collection of requests to complete future for. - * @param err Error to be passed to futures. - */ - public void completeStartFutures(Collection reqs, @Nullable Throwable err) { - if (!F.isEmpty(reqs)) { - for (DynamicCacheChangeRequest req : reqs) { completeStartFuture(req, err); } } @@ -2632,10 +2607,10 @@ else if (msg instanceof DynamicCacheChangeFailureMessage) * @return {@code True} if minor topology version should be increased. */ private boolean onCacheChangeRequested(DynamicCacheChangeFailureMessage failMsg) { - for (DynamicCacheChangeRequest req : failMsg.requests()) { - registeredCaches.remove(maskNull(req.cacheName())); + for (String cacheName : failMsg.cacheNames()) { + registeredCaches.remove(maskNull(cacheName)); - ctx.discovery().removeCacheFilter(req.cacheName(), true); + ctx.discovery().removeCacheFilter(cacheName); } return false; @@ -2778,7 +2753,7 @@ private boolean onCacheChangeRequested(DynamicCacheChangeBatch batch, assert old != null : "Dynamic cache map was concurrently modified [req=" + req + ']'; - ctx.discovery().removeCacheFilter(req.cacheName(), false); + ctx.discovery().removeCacheFilter(req.cacheName()); needExchange = true; } 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 6a7145442f49e..b4d02ec2bdebd 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 @@ -75,7 +75,6 @@ import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteInClosure; -import org.apache.ignite.lang.IgniteProductVersion; import org.apache.ignite.lang.IgniteRunnable; import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentHashMap8; @@ -164,10 +163,10 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter singleMsgs = new ConcurrentHashMap8<>(); + private final Map singleMsgs = new ConcurrentHashMap8<>(); /** Messages received from new coordinator. */ - private Map fullMsgs = new ConcurrentHashMap8<>(); + private final Map fullMsgs = new ConcurrentHashMap8<>(); /** */ @SuppressWarnings({"FieldCanBeLocal", "UnusedDeclaration"}) @@ -202,10 +201,10 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter exchangeGlobalExceptions = new ConcurrentHashMap8<>(); + private final Map exchangeGlobalExceptions = new ConcurrentHashMap8<>(); /** Forced Rebalance future. */ private GridCompoundFuture forcedRebFut; @@ -552,31 +551,39 @@ public void init() throws IgniteInterruptedCheckedException { return; } - U.error(log, "Failed to reinitialize local partitions (rebalancing will be stopped): " + exchId, e); + U.error(log, "Failed to initialize cache (will try to rollback). " + exchId, e); - if (cctx.kernalContext().clientNode() || !isRollbackSupported()) { + if (!isRollbackSupported()) { onDone(e); return; } - exchangeLocalException = new IgniteCheckedException( + exchangeLocE = new IgniteCheckedException( "Failed to initialize exchange locally [locNodeId=" + cctx.localNodeId() + "]", e); - exchangeGlobalExceptions.put(cctx.localNodeId(), exchangeLocalException); + exchangeGlobalExceptions.put(cctx.localNodeId(), exchangeLocE); - if (crd.isLocal()) { - boolean isRemainingEmpty = false; + if (crd != null) { + if (crd.isLocal()) { + boolean allRcvd; - synchronized (mux) { - isRemainingEmpty = remaining.isEmpty(); + synchronized (mux) { + allRcvd = remaining.isEmpty(); + } + + if (allRcvd) + onAllReceived(false); } + else { + clientOnlyExchange = cctx.kernalContext().clientNode(); - if (isRemainingEmpty) - onAllReceived(false); + if (!centralizedAff) + sendPartitions(crd); + } } else - sendPartitions(crd); + onDone(e); initDone(); } @@ -1016,13 +1023,21 @@ public boolean stopping(int cacheId) { */ private void sendLocalPartitions(ClusterNode node) throws IgniteCheckedException { - GridDhtPartitionsSingleMessage m = cctx.exchange().createPartitionsSingleMessage(node, - exchangeId(), - clientOnlyExchange, - true); + GridDhtPartitionsSingleMessage m; - if (exchangeLocalException != null) - m.setError(exchangeLocalException); + if (exchangeLocE == null) + m = cctx.exchange().createPartitionsSingleMessage(node, + exchangeId(), + clientOnlyExchange, + true); + else { + m = new GridDhtPartitionsSingleMessage(exchangeId(), + clientOnlyExchange, + cctx.versions().last(), + cctx.exchange().canUsePartitionMapCompression(node)); + + m.setError(exchangeLocE); + } if (log.isDebugEnabled()) log.debug("Sending local partitions [nodeId=" + node.id() + ", exchId=" + exchId + ", msg=" + m + ']'); @@ -1129,9 +1144,7 @@ private void sendPartitions(ClusterNode oldestNode) { cctx.exchange().onExchangeDone(this, err); - cctx.cache().onExchangeDone(exchId.topologyVersion(), reqs, err, false); - - cctx.cache().completeStartFutures(reqs, err); + cctx.cache().onExchangeDone(exchId.topologyVersion(), reqs, err); if (super.onDone(res, err) && realExchange) { if (log.isDebugEnabled()) @@ -1179,12 +1192,12 @@ private void sendPartitions(ClusterNode oldestNode) { * Cleans up resources to avoid excessive memory usage. */ public void cleanUp() { - singleMsgs = null; - fullMsgs = null; + singleMsgs.clear(); + fullMsgs.clear(); crd = null; partReleaseFut = null; - exchangeLocalException = null; - exchangeGlobalExceptions = null; + exchangeLocE = null; + exchangeGlobalExceptions.clear(); } /** @@ -1222,10 +1235,17 @@ public void onReceive(final ClusterNode node, final GridDhtPartitionsSingleMessa log.debug("Received message for finished future (will reply only to sender) [msg=" + msg + ", fut=" + this + ']'); + // Custom message (DynamicCacheChangeFailureMessage) was sent. + // Do not need sendAllPartitions. + if (!exchangeGlobalExceptions.isEmpty()) + return; + if (!centralizedAff) sendAllPartitions(node.id(), cctx.gridConfig().getNetworkSendRetryCount()); } else { + assert !msg.client(); + initFut.listen(new CI1>() { @Override public void apply(IgniteInternalFuture f) { try { @@ -1342,13 +1362,13 @@ private void onAffinityInitialized(IgniteInternalFuture globalExceptions) { IgniteCheckedException ex; - if (exchangeLocalException != null) - ex = exchangeLocalException; + if (exchangeLocE != null) + ex = exchangeLocE; else ex = new IgniteCheckedException("Failed to complete exchange process (will try to rollback)."); for (Map.Entry entry : globalExceptions.entrySet()) { - // avoid self-suppression + // Avoid self-suppression. if (ex != entry.getValue()) ex.addSuppressed(entry.getValue()); } @@ -1364,11 +1384,8 @@ private IgniteCheckedException createExchangeException(Map glob private boolean isRollbackSupported() { boolean rollbackSupported = false; - for (ClusterNode node : discoCache.allNodes()) { - Boolean exchangeSupported = node.attribute(ATTR_EXCHANGE_ROLLBACK_SUPPORTED); - if (exchangeSupported == null || !exchangeSupported) - return false; - } + if (!discoCache.checkAttribute(ATTR_EXCHANGE_ROLLBACK_SUPPORTED, Boolean.TRUE)) + return false; // Currently the rollback process is supported for dynamically started caches. if (discoEvt.type() == EVT_DISCOVERY_CUSTOM_EVT && !F.isEmpty(reqs)) { @@ -1398,10 +1415,10 @@ private void rollbackExchange() { stopReq.stop(true); stopReq.deploymentId(req.deploymentId()); - // cleanup GridCacheProcessor + // Cleanup GridCacheProcessor. cctx.cache().forceCloseCache(stopReq); - // cleanup CacheAffinitySharedManager + // Cleanup CacheAffinitySharedManager. cctx.affinity().forceCloseCache(this, crd.isLocal(), Collections.singletonList(stopReq)); } } @@ -1416,14 +1433,15 @@ private void onAllReceived(boolean discoThread) { assert crd.isLocal(); if (!F.isEmpty(exchangeGlobalExceptions) && isRollbackSupported()) { - updateLastVersion(cctx.versions().last()); + IgniteCheckedException err = createExchangeException(exchangeGlobalExceptions); - cctx.versions().onExchange(lastVer.get().order()); + List cacheNames = new ArrayList<>(reqs.size()); - IgniteCheckedException err = createExchangeException(exchangeGlobalExceptions); + for (DynamicCacheChangeRequest req : reqs) + cacheNames.add(req.cacheName()); DynamicCacheChangeFailureMessage msg = new DynamicCacheChangeFailureMessage( - cctx.localNode(), exchId, lastVer.get(), err, reqs); + cctx.localNode(), exchId, err, cacheNames); if (log.isDebugEnabled()) log.debug("Dynamic cache change failed. Send message to all participating nodes: " + msg); @@ -1572,6 +1590,8 @@ private void processMessage(ClusterNode node, GridDhtPartitionsFullMessage msg) assert msg.exchangeId().equals(exchId) : msg; assert msg.lastVersion() != null : msg; + boolean isRollbackNedeed = false; + synchronized (mux) { if (crd == null) return; @@ -1586,6 +1606,17 @@ private void processMessage(ClusterNode node, GridDhtPartitionsFullMessage msg) return; } + + if (exchangeLocE != null && isRollbackSupported()) + isRollbackNedeed = true; + } + + if (isRollbackNedeed) { + rollbackExchange(); + + onDone(exchId.topologyVersion(), exchangeLocE); + + return; } updatePartitionFullMap(msg); @@ -1625,6 +1656,9 @@ private void updatePartitionFullMap(GridDhtPartitionsFullMessage msg) { * @param msg Partitions single message. */ private void updatePartitionSingleMap(GridDhtPartitionsSingleMessage msg) { + if (msg.partitions() == null) + return; + for (Map.Entry entry : msg.partitions().entrySet()) { Integer cacheId = entry.getKey(); GridCacheContext cacheCtx = cctx.cacheContext(cacheId); @@ -1644,7 +1678,6 @@ private void updatePartitionSingleMap(GridDhtPartitionsSingleMessage msg) { */ public void onDynamicCacheChangeFail(final ClusterNode node, final DynamicCacheChangeFailureMessage msg) { assert exchId.equals(msg.exchangeId()) : msg; - assert msg.lastVersion() != null : msg; onDiscoveryEvent(new IgniteRunnable() { @Override public void run() { @@ -1655,10 +1688,7 @@ public void onDynamicCacheChangeFail(final ClusterNode node, final DynamicCacheC if (isRollbackSupported()) rollbackExchange(); - if (!crd.isLocal()) - cctx.versions().onExchange(msg.lastVersion().order()); - - onDone(exchId.topologyVersion(), msg.getError()); + onDone(exchId.topologyVersion(), msg.error()); } finally { leaveBusy(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java index abc05c0afb83e..f2664d50b8f8e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java @@ -182,7 +182,9 @@ public Map partitions() { @Override public void prepareMarshal(GridCacheSharedContext ctx) throws IgniteCheckedException { super.prepareMarshal(ctx); - boolean marshal = (parts != null && partsBytes == null) || (partCntrs != null && partCntrsBytes == null); + boolean marshal = (parts != null && partsBytes == null) || + (partCntrs != null && partCntrsBytes == null) || + (err != null && errBytes == null); if (marshal) { byte[] partsBytes0 = null; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartFailSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartFailSelfTest.java index fba54aa80ed92..2ca40249c29d3 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartFailSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartFailSelfTest.java @@ -35,8 +35,6 @@ import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; -import org.apache.ignite.events.EventType; -import org.apache.ignite.internal.IgniteNodeAttributes; import org.apache.ignite.lang.IgnitePredicate; import org.apache.ignite.resources.IgniteInstanceResource; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; @@ -45,6 +43,9 @@ import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +/** + * + */ @SuppressWarnings("unchecked") public class IgniteDynamicCacheStartFailSelfTest extends GridCommonAbstractTest { /** */ @@ -53,8 +54,8 @@ public class IgniteDynamicCacheStartFailSelfTest extends GridCommonAbstractTest /** */ private static final String DYNAMIC_CACHE_NAME = "TestDynamicCache"; - /** Coordinator node. */ - private Ignite crd; + /** */ + private static final String CLIENT_GRID_NAME = "client"; /** Coordinator node index. */ private int crdIdx; @@ -78,7 +79,8 @@ public int nodeCount() { /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { crdIdx = 0; - crd = startGrid(crdIdx); + + startGrid(crdIdx); for (int i = 1; i < nodeCount(); ++i) startGrid(i); @@ -89,156 +91,307 @@ public int nodeCount() { stopAllGrids(); } - public void testBrokenAffinityFunctionOnAllNodes() { + /** + * @throws Exception If failed. + */ + public void testBrokenAffinityFunStartOnServerFailedOnClient() throws Exception { + final String clientName = CLIENT_GRID_NAME + "testBrokenAffinityFunStartOnServerFailedOnClient"; + + IgniteConfiguration clientCfg = getConfiguration(clientName); + + clientCfg.setClientMode(true); + + Ignite client = startGrid(clientName, clientCfg); + + CacheConfiguration cfg = new CacheConfiguration(); + + cfg.setName(DYNAMIC_CACHE_NAME + "-server-1"); + + cfg.setAffinity(new BrokenAffinityFunction(false, clientName)); + + try { + IgniteCache cache = ignite(0).getOrCreateCache(cfg); + } + catch (CacheException e) { + fail("Exception should not be thrown."); + } + + stopGrid(clientName); + } + + /** + * @throws Exception If failed. + */ + public void testBrokenAffinityFunStartOnServerFailedOnServer() throws Exception { + final String clientName = CLIENT_GRID_NAME + "testBrokenAffinityFunStartOnServerFailedOnServer"; + + IgniteConfiguration clientCfg = getConfiguration(clientName); + + clientCfg.setClientMode(true); + + Ignite client = startGrid(clientName, clientCfg); + + CacheConfiguration cfg = new CacheConfiguration(); + + cfg.setName(DYNAMIC_CACHE_NAME + "-server-2"); + + cfg.setAffinity(new BrokenAffinityFunction(false, getTestGridName(0))); + + try { + IgniteCache cache = ignite(0).getOrCreateCache(cfg); + + fail("Expected exception was not thrown."); + } + catch (CacheException e) { + } + + stopGrid(clientName); + } + + /** + * @throws Exception If failed. + */ + public void testBrokenAffinityFunStartOnClientFailOnClient() throws Exception { + final String clientName = CLIENT_GRID_NAME + "testBrokenAffinityFunStartOnClientFailOnClient"; + + IgniteConfiguration clientCfg = getConfiguration(clientName); + + clientCfg.setClientMode(true); + + Ignite client = startGrid(clientName, clientCfg); + + CacheConfiguration cfg = new CacheConfiguration(); + + cfg.setName(DYNAMIC_CACHE_NAME + "-client-1"); + + cfg.setAffinity(new BrokenAffinityFunction(false, clientName)); + + try { + IgniteCache cache = client.getOrCreateCache(cfg); + + fail("Expected exception was not thrown."); + } + catch (CacheException e) { + } + + stopGrid(clientName); + } + + /** + * @throws Exception If failed. + */ + public void testBrokenAffinityFunStartOnClientFailOnServer() throws Exception { + final String clientName = CLIENT_GRID_NAME + "testBrokenAffinityFunStartOnClientFailOnServer"; + + IgniteConfiguration clientCfg = getConfiguration(clientName); + + clientCfg.setClientMode(true); + + Ignite client = startGrid(clientName, clientCfg); + + CacheConfiguration cfg = new CacheConfiguration(); + + cfg.setName(DYNAMIC_CACHE_NAME + "-client-2"); + + cfg.setAffinity(new BrokenAffinityFunction(false, getTestGridName(0))); + + try { + IgniteCache cache = client.getOrCreateCache(cfg); + + fail("Expected exception was not thrown."); + } + catch (CacheException e) { + } + + stopGrid(clientName); + } + + /** + * Test cache start with broken affinity function that throws an exception on all nodes. + */ + public void testBrokenAffinityFunOnAllNodes() { final boolean failOnAllNodes = true; final int unluckyNode = 0; final int unluckyCfg = 1; - final int numberOfCaches = 3; + final int numOfCaches = 3; final int initiator = 0; testDynamicCacheStart( - createCacheConfigsWithBrokenAffinityFunction( - failOnAllNodes, unluckyNode, unluckyCfg, numberOfCaches, false), + createCacheConfigsWithBrokenAffinityFun( + failOnAllNodes, unluckyNode, unluckyCfg, numOfCaches, false), initiator); } - public void testBrokenAffinityFunctionOnInitiator() { + /** + * Test cache start with broken affinity function that throws an exception on initiator node. + */ + public void testBrokenAffinityFunOnInitiator() { final boolean failOnAllNodes = false; final int unluckyNode = 1; final int unluckyCfg = 1; - final int numberOfCaches = 3; + final int numOfCaches = 3; final int initiator = 1; testDynamicCacheStart( - createCacheConfigsWithBrokenAffinityFunction( - failOnAllNodes, unluckyNode, unluckyCfg, numberOfCaches, false), + createCacheConfigsWithBrokenAffinityFun( + failOnAllNodes, unluckyNode, unluckyCfg, numOfCaches, false), initiator); } - public void testBrokenAffinityFunctionOnNonInitiator() { + /** + * Test cache start with broken affinity function that throws an exception on non-initiator node. + */ + public void testBrokenAffinityFunOnNonInitiator() { final boolean failOnAllNodes = false; final int unluckyNode = 1; final int unluckyCfg = 1; - final int numberOfCaches = 3; + final int numOfCaches = 3; final int initiator = 2; testDynamicCacheStart( - createCacheConfigsWithBrokenAffinityFunction( - failOnAllNodes, unluckyNode, unluckyCfg, numberOfCaches, false), + createCacheConfigsWithBrokenAffinityFun( + failOnAllNodes, unluckyNode, unluckyCfg, numOfCaches, false), initiator); } - public void testBrokenAffinityFunctionOnCoordinatorDiffInitiator() { + /** + * Test cache start with broken affinity function that throws an exception on coordinator node. + */ + public void testBrokenAffinityFunOnCoordinatorDiffInitiator() { final boolean failOnAllNodes = false; final int unluckyNode = crdIdx; final int unluckyCfg = 1; - final int numberOfCaches = 3; + final int numOfCaches = 3; final int initiator = (crdIdx + 1) % nodeCount(); testDynamicCacheStart( - createCacheConfigsWithBrokenAffinityFunction( - failOnAllNodes, unluckyNode, unluckyCfg, numberOfCaches, false), + createCacheConfigsWithBrokenAffinityFun( + failOnAllNodes, unluckyNode, unluckyCfg, numOfCaches, false), initiator); } - public void testBrokenAffinityFunctionOnCoordinator() { + /** + * Test cache start with broken affinity function that throws an exception on initiator node. + */ + public void testBrokenAffinityFunOnCoordinator() { final boolean failOnAllNodes = false; final int unluckyNode = crdIdx; final int unluckyCfg = 1; - final int numberOfCaches = 3; + final int numOfCaches = 3; final int initiator = crdIdx; testDynamicCacheStart( - createCacheConfigsWithBrokenAffinityFunction( - failOnAllNodes, unluckyNode, unluckyCfg, numberOfCaches, false), + createCacheConfigsWithBrokenAffinityFun( + failOnAllNodes, unluckyNode, unluckyCfg, numOfCaches, false), initiator); } - public void testBrokenAffinityFunctionWithNodeFilter() { + /** + * Tests cache start with node filter and broken affinity function that throws an exception on initiator node. + */ + public void testBrokenAffinityFunWithNodeFilter() { final boolean failOnAllNodes = false; final int unluckyNode = 0; final int unluckyCfg = 0; - final int numberOfCaches = 1; + final int numOfCaches = 1; final int initiator = 0; testDynamicCacheStart( - createCacheConfigsWithBrokenAffinityFunction( - failOnAllNodes, unluckyNode, unluckyCfg, numberOfCaches, true), + createCacheConfigsWithBrokenAffinityFun( + failOnAllNodes, unluckyNode, unluckyCfg, numOfCaches, true), initiator); } + /** + * Tests cache start with broken cache store that throws an exception on all nodes. + */ public void testBrokenCacheStoreOnAllNodes() { final boolean failOnAllNodes = true; final int unluckyNode = 0; final int unluckyCfg = 1; - final int numberOfCaches = 3; + final int numOfCaches = 3; final int initiator = 0; testDynamicCacheStart( createCacheConfigsWithBrokenCacheStore( - failOnAllNodes, unluckyNode, unluckyCfg, numberOfCaches, false), + failOnAllNodes, unluckyNode, unluckyCfg, numOfCaches, false), initiator); } + /** + * Tests cache start with broken cache store that throws an exception on initiator node. + */ public void testBrokenCacheStoreOnInitiator() { final boolean failOnAllNodes = false; final int unluckyNode = 1; final int unluckyCfg = 1; - final int numberOfCaches = 3; + final int numOfCaches = 3; final int initiator = 1; testDynamicCacheStart( createCacheConfigsWithBrokenCacheStore( - failOnAllNodes, unluckyNode, unluckyCfg, numberOfCaches, false), + failOnAllNodes, unluckyNode, unluckyCfg, numOfCaches, false), initiator); } + /** + * Tests cache start with broken cache store that throws an exception on non-initiator node. + */ public void testBrokenCacheStoreOnNonInitiator() { final boolean failOnAllNodes = false; final int unluckyNode = 1; final int unluckyCfg = 1; - final int numberOfCaches = 3; + final int numOfCaches = 3; final int initiator = 2; testDynamicCacheStart( createCacheConfigsWithBrokenCacheStore( - failOnAllNodes, unluckyNode, unluckyCfg, numberOfCaches, false), + failOnAllNodes, unluckyNode, unluckyCfg, numOfCaches, false), initiator); } + /** + * Tests cache start with broken cache store that throws an exception on initiator node. + */ public void testBrokenCacheStoreOnCoordinatorDiffInitiator() { final boolean failOnAllNodes = false; final int unluckyNode = crdIdx; final int unluckyCfg = 1; - final int numberOfCaches = 3; + final int numOfCaches = 3; final int initiator = (crdIdx + 1) % nodeCount(); testDynamicCacheStart( createCacheConfigsWithBrokenCacheStore( - failOnAllNodes, unluckyNode, unluckyCfg, numberOfCaches, false), + failOnAllNodes, unluckyNode, unluckyCfg, numOfCaches, false), initiator); } - public void testBrokenCacheStoreFunctionOnCoordinator() { + /** + * Tests cache start with broken cache store that throws an exception on coordinator node. + */ + public void testBrokenCacheStoreFunOnCoordinator() { final boolean failOnAllNodes = false; final int unluckyNode = crdIdx; final int unluckyCfg = 1; - final int numberOfCaches = 3; + final int numOfCaches = 3; final int initiator = crdIdx; testDynamicCacheStart( createCacheConfigsWithBrokenCacheStore( - failOnAllNodes, unluckyNode, unluckyCfg, numberOfCaches, false), + failOnAllNodes, unluckyNode, unluckyCfg, numOfCaches, false), initiator); } + /** + * Tests multiple creation of cache with broken affinity function. + */ public void testCreateCacheMultipleTimes() { final boolean failOnAllNodes = false; final int unluckyNode = 1; final int unluckyCfg = 0; final int numOfAttempts = 100; - CacheConfiguration cfg = createCacheConfigsWithBrokenAffinityFunction( + CacheConfiguration cfg = createCacheConfigsWithBrokenAffinityFun( failOnAllNodes, unluckyNode, unluckyCfg, 1, false).get(0); for (int i = 0; i < numOfAttempts; ++i) { @@ -252,7 +405,7 @@ public void testCreateCacheMultipleTimes() { } } - private List createCacheConfigsWithBrokenAffinityFunction( + private List createCacheConfigsWithBrokenAffinityFun( boolean failOnAllNodes, int unluckyNode, final int unluckyCfg, @@ -321,18 +474,18 @@ private void testDynamicCacheStart(final Collection cfgs, fi return null; } }, CacheException.class, null); - - for (CacheConfiguration cfg : cfgs) - assertNull("initiatorId=" + initiatorId, grid(initiatorId).cache(cfg.getName())); } + /** + * Filter specifying on which node the cache should be started. + */ private static class NodeFilter implements IgnitePredicate { /** Cache should be created node with certain UUID. */ public UUID uuid; - public NodeFilter() { - } - + /** + * @param uuid node ID. + */ public NodeFilter(UUID uuid) { this.uuid = uuid; } @@ -343,6 +496,9 @@ public NodeFilter(UUID uuid) { } } + /** + * Factory that throws an exception is got created. + */ private static class BrokenAffinityFunction extends RendezvousAffinityFunction { /** */ private static final long serialVersionUID = 0L; @@ -352,22 +508,30 @@ private static class BrokenAffinityFunction extends RendezvousAffinityFunction { private Ignite ignite; /** Exception should arise on all nodes. */ - private boolean exceptionOnAllNodes = false; + private boolean eOnAllNodes = false; /** Exception should arise on node with certain name. */ private String gridName; + /** + * Default constructor. + */ public BrokenAffinityFunction() { + // No-op. } - public BrokenAffinityFunction(boolean exceptionOnAllNodes, String gridName) { - this.exceptionOnAllNodes = exceptionOnAllNodes; + /** + * @param eOnAllNodes {@code True} if exception should be thrown on all nodes. + * @param gridName Exception should arise on node with certain name. + */ + public BrokenAffinityFunction(boolean eOnAllNodes, String gridName) { + this.eOnAllNodes = eOnAllNodes; this.gridName = gridName; } /** {@inheritDoc} */ @Override public List> assignPartitions(AffinityFunctionContext affCtx) { - if (exceptionOnAllNodes || ignite.name().equals(gridName)) + if (eOnAllNodes || ignite.name().equals(gridName)) throw new IllegalStateException("Simulated exception [locNodeId=" + ignite.cluster().localNode().id() + "]"); else @@ -377,7 +541,7 @@ public BrokenAffinityFunction(boolean exceptionOnAllNodes, String gridName) { /** {@inheritDoc} */ @Override public void writeExternal(ObjectOutput out) throws IOException { super.writeExternal(out); - out.writeBoolean(exceptionOnAllNodes); + out.writeBoolean(eOnAllNodes); out.writeObject(gridName); } @@ -385,33 +549,38 @@ public BrokenAffinityFunction(boolean exceptionOnAllNodes, String gridName) { @SuppressWarnings("unchecked") @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { super.readExternal(in); - exceptionOnAllNodes = in.readBoolean(); + eOnAllNodes = in.readBoolean(); gridName = (String)in.readObject(); } } + /** + * Factory that throws an exception is got created. + */ private static class BrokenStoreFactory implements Factory> { /** */ @IgniteInstanceResource private Ignite ignite; /** Exception should arise on all nodes. */ - boolean exceptionOnAllNodes = true; + boolean eOnAllNodes = true; /** Exception should arise on node with certain name. */ public static String gridName; - public BrokenStoreFactory() { - } + /** + * @param eOnAllNodes {@code True} if exception should be thrown on all nodes. + * @param gridName Exception should arise on node with certain name. + */ + public BrokenStoreFactory(boolean eOnAllNodes, String gridName) { + this.eOnAllNodes = eOnAllNodes; - public BrokenStoreFactory(boolean exceptionOnAllNodes, String gridName) { - this.exceptionOnAllNodes = exceptionOnAllNodes; - this.gridName = gridName; + BrokenStoreFactory.gridName = gridName; } /** {@inheritDoc} */ @Override public CacheStore create() { - if (exceptionOnAllNodes || ignite.name().equals(gridName)) + if (eOnAllNodes || ignite.name().equals(gridName)) throw new IllegalStateException("Simulated exception [locNodeId=" + ignite.cluster().localNode().id() + "]"); else 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 791ed6bf9d4bc..a4ec552f562ed 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 @@ -85,6 +85,7 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheTxStoreValueTest; import org.apache.ignite.internal.processors.cache.IgniteClientCacheInitializationFailTest; import org.apache.ignite.internal.processors.cache.IgniteDynamicCacheFilterTest; +import org.apache.ignite.internal.processors.cache.IgniteDynamicCacheStartFailSelfTest; import org.apache.ignite.internal.processors.cache.IgniteDynamicCacheStartNoExchangeTimeoutTest; import org.apache.ignite.internal.processors.cache.IgniteDynamicCacheStartSelfTest; import org.apache.ignite.internal.processors.cache.IgniteDynamicCacheStartStopConcurrentTest; @@ -228,6 +229,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(IgniteCacheTxPreloadNoWriteTest.class); suite.addTestSuite(IgniteDynamicCacheStartSelfTest.class); + suite.addTestSuite(IgniteDynamicCacheStartFailSelfTest.class); suite.addTestSuite(IgniteDynamicCacheWithConfigStartSelfTest.class); suite.addTestSuite(IgniteCacheDynamicStopSelfTest.class); suite.addTestSuite(IgniteDynamicCacheStartStopConcurrentTest.class); From e47a4c03bc65a55349981ad79eb6a9f7fa657a2a Mon Sep 17 00:00:00 2001 From: sk0x50 Date: Fri, 25 Aug 2017 10:48:33 +0300 Subject: [PATCH 253/357] IGNITE-1094: added cacheChangeFailureMsgSent flag --- .../preloader/GridDhtPartitionsExchangeFuture.java | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) 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 b4d02ec2bdebd..524412bf55f5f 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 @@ -206,6 +206,9 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter exchangeGlobalExceptions = new ConcurrentHashMap8<>(); + /** Used to track the fact that {@code DynamicCacheChangeFailureMessage} was sent. */ + private volatile boolean cacheChangeFailureMsgSent; + /** Forced Rebalance future. */ private GridCompoundFuture forcedRebFut; @@ -1235,9 +1238,8 @@ public void onReceive(final ClusterNode node, final GridDhtPartitionsSingleMessa log.debug("Received message for finished future (will reply only to sender) [msg=" + msg + ", fut=" + this + ']'); - // Custom message (DynamicCacheChangeFailureMessage) was sent. - // Do not need sendAllPartitions. - if (!exchangeGlobalExceptions.isEmpty()) + // Custom message (DynamicCacheChangeFailureMessage) was sent. Do not need sendAllPartitions. + if (cacheChangeFailureMsgSent) return; if (!centralizedAff) @@ -1444,10 +1446,12 @@ private void onAllReceived(boolean discoThread) { cctx.localNode(), exchId, err, cacheNames); if (log.isDebugEnabled()) - log.debug("Dynamic cache change failed. Send message to all participating nodes: " + msg); + log.debug("Dynamic cache change failed (send message to all participating nodes): " + msg); cctx.discovery().sendCustomEvent(msg); + cacheChangeFailureMsgSent = true; + return; } From d90a9207756b035294945f696fa0c00c69bdd330 Mon Sep 17 00:00:00 2001 From: sboikov Date: Fri, 25 Aug 2017 12:28:37 +0300 Subject: [PATCH 254/357] ignite-1094 review --- .../dht/preloader/GridDhtPartitionsExchangeFuture.java | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) 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 524412bf55f5f..66527298efd37 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 @@ -1448,10 +1448,10 @@ private void onAllReceived(boolean discoThread) { if (log.isDebugEnabled()) log.debug("Dynamic cache change failed (send message to all participating nodes): " + msg); - cctx.discovery().sendCustomEvent(msg); - cacheChangeFailureMsgSent = true; + cctx.discovery().sendCustomEvent(msg); + return; } @@ -1660,8 +1660,11 @@ private void updatePartitionFullMap(GridDhtPartitionsFullMessage msg) { * @param msg Partitions single message. */ private void updatePartitionSingleMap(GridDhtPartitionsSingleMessage msg) { - if (msg.partitions() == null) + if (msg.partitions() == null) { + assert msg.getError() != null : msg; + return; + } for (Map.Entry entry : msg.partitions().entrySet()) { Integer cacheId = entry.getKey(); From a7310e49948de5780845acbaada3e6ccce288265 Mon Sep 17 00:00:00 2001 From: Slava Koptilin Date: Fri, 25 Aug 2017 19:09:51 +0300 Subject: [PATCH 255/357] IGNITE-1094: review --- .../dht/preloader/GridDhtPartitionsExchangeFuture.java | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) 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 66527298efd37..91ec8d250f692 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 @@ -1660,11 +1660,8 @@ private void updatePartitionFullMap(GridDhtPartitionsFullMessage msg) { * @param msg Partitions single message. */ private void updatePartitionSingleMap(GridDhtPartitionsSingleMessage msg) { - if (msg.partitions() == null) { - assert msg.getError() != null : msg; - + if (msg.partitions() == null) return; - } for (Map.Entry entry : msg.partitions().entrySet()) { Integer cacheId = entry.getKey(); From 75febb824b3e261ec1c15224a024e1d4160f2f42 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Tue, 13 Jun 2017 19:41:55 +0300 Subject: [PATCH 256/357] 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 257/357] 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 258/357] 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 e7b1157d008d7416c60a511274f7f85aa919433e Mon Sep 17 00:00:00 2001 From: Ilya Kasnacheev Date: Mon, 21 Aug 2017 17:54:12 +0300 Subject: [PATCH 259/357] GG-12588 Backport IGNITE-5943 Communication. Server node may reject client connection during massive clients join. --- .../tcp/TcpCommunicationSpi.java | 61 ++++++- .../ignite/spi/discovery/tcp/ServerImpl.java | 24 ++- .../spi/discovery/tcp/TcpDiscoverySpi.java | 10 + .../tcp/IgniteClientConnectTest.java | 172 ++++++++++++++++++ .../IgniteSpiDiscoverySelfTestSuite.java | 3 + 5 files changed, 263 insertions(+), 7 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/IgniteClientConnectTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java index ed6789caa9843..de1f76d9f1b61 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 @@ -65,6 +65,7 @@ import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener; +import org.apache.ignite.internal.util.typedef.CI1; import org.apache.ignite.internal.util.GridConcurrentFactory; import org.apache.ignite.internal.util.GridSpinReadWriteLock; import org.apache.ignite.internal.util.future.GridFutureAdapter; @@ -129,6 +130,8 @@ import org.apache.ignite.spi.IgniteSpiTimeoutObject; import org.apache.ignite.spi.communication.CommunicationListener; import org.apache.ignite.spi.communication.CommunicationSpi; +import org.apache.ignite.spi.discovery.DiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.thread.IgniteThread; import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentLinkedDeque8; @@ -137,6 +140,7 @@ import static org.apache.ignite.events.EventType.EVT_NODE_FAILED; import static org.apache.ignite.events.EventType.EVT_NODE_LEFT; import static org.apache.ignite.internal.util.nio.GridNioSessionMetaKey.SSL_META; +import static org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi.RecoveryLastReceivedMessage.NEED_WAIT; /** * TcpCommunicationSpi is default communication SPI which uses @@ -301,6 +305,13 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter /** Connection index meta for session. */ private static final int CONN_IDX_META = GridNioSessionMetaKey.nextUniqueKey(); + /** + * Version when client is ready to wait to connect to server (could be needed when client tries to open connection + * before it starts being visible for server) + */ + private static final IgniteProductVersion VERSION_SINCE_CLIENT_COULD_WAIT_TO_CONNECT = + IgniteProductVersion.fromString("1.9.7"); + /** Message tracker meta for session. */ private static final int TRACKER_META = GridNioSessionMetaKey.nextUniqueKey(); @@ -435,7 +446,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter * @param ses Session. * @param msg Message. */ - private void onFirstMessage(GridNioSession ses, Message msg) { + private void onFirstMessage(final GridNioSession ses, Message msg) { UUID sndId; ConnectionKey connKey; @@ -459,10 +470,35 @@ private void onFirstMessage(GridNioSession ses, Message msg) { final ClusterNode rmtNode = getSpiContext().node(sndId); if (rmtNode == null) { - U.warn(log, "Close incoming connection, unknown node [nodeId=" + sndId + - ", ses=" + ses + ']'); + DiscoverySpi discoverySpi = ignite.configuration().getDiscoverySpi(); + + assert discoverySpi instanceof TcpDiscoverySpi; + + TcpDiscoverySpi tcpDiscoverySpi = (TcpDiscoverySpi) discoverySpi; + + ClusterNode node0 = tcpDiscoverySpi.getNode0(sndId); + + boolean unknownNode = true; - ses.close(); + if (node0 != null) { + assert node0.isClient() : node0; + + if (node0.version().compareTo(VERSION_SINCE_CLIENT_COULD_WAIT_TO_CONNECT) >= 0) + unknownNode = false; + } + + if (unknownNode) { + U.warn(log, "Close incoming connection, unknown node [nodeId=" + sndId + ", ses=" + ses + ']'); + + ses.close(); + } + else { + ses.send(new RecoveryLastReceivedMessage(NEED_WAIT)).listen(new CI1>() { + @Override public void apply(IgniteInternalFuture fut) { + ses.close(); + } + }); + } return; } @@ -2858,6 +2894,8 @@ protected GridCommunicationClient createTcpClient(ClusterNode node, int connIdx) IgniteSpiOperationTimeoutHelper timeoutHelper = new IgniteSpiOperationTimeoutHelper(this); + int lastWaitingTimeout = 1; + while (!conn) { // Reconnection on handshake timeout. try { SocketChannel ch = SocketChannel.open(); @@ -2920,6 +2958,18 @@ protected GridCommunicationClient createTcpClient(ClusterNode node, int connIdx) if (rcvCnt == -1) return null; + else if (rcvCnt == NEED_WAIT) { + recoveryDesc.release(); + + U.closeQuiet(ch); + + if (lastWaitingTimeout < 60000) + lastWaitingTimeout *= 2; + + U.sleep(lastWaitingTimeout); + + continue; + } } finally { if (recoveryDesc != null && rcvCnt == -1) @@ -4294,6 +4344,9 @@ public static class RecoveryLastReceivedMessage implements Message { /** */ private static final long serialVersionUID = 0L; + /** Need wait. */ + static final long NEED_WAIT = -3; + /** */ private long rcvCnt; 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 84343e46a9521..e1dff1c0ad1d3 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 @@ -1803,9 +1803,27 @@ private static void removeMetrics(TcpDiscoveryHeartbeatMessage msg, UUID nodeId) } /** - * Thread that cleans IP finder and keeps it in the correct state, unregistering addresses of the nodes that has - * left the topology.

    This thread should run only on coordinator node and will clean IP finder if and only if - * {@link org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder#isShared()} is {@code true}. + * Trying get node in any state (visible or not) + * @param nodeId Node id. + */ + ClusterNode getNode0(UUID nodeId) { + assert nodeId != null; + + UUID locNodeId0 = getLocalNodeId(); + + if (locNodeId0 != null && locNodeId0.equals(nodeId)) + // Return local node directly. + return locNode; + + return ring.node(nodeId); + } + + /** + * Thread that cleans IP finder and keeps it in the correct state, unregistering + * addresses of the nodes that has left the topology. + *

    + * This thread should run only on coordinator node and will clean IP finder + * if and only if {@link org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder#isShared()} is {@code true}. */ private class IpFinderCleaner extends IgniteSpiThread { /** 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 8c7ef99afcbff..ac423363184ea 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 @@ -437,6 +437,16 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T return impl.getNode(nodeId); } + /** + * @param id Id. + */ + public ClusterNode getNode0(UUID id) { + if (impl instanceof ServerImpl) + return ((ServerImpl)impl).getNode0(id); + + return getNode(id); + } + /** {@inheritDoc} */ @Override public boolean pingNode(UUID nodeId) { return impl.pingNode(nodeId); diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/IgniteClientConnectTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/IgniteClientConnectTest.java new file mode 100644 index 0000000000000..b1c4dbf25434d --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/IgniteClientConnectTest.java @@ -0,0 +1,172 @@ +/* + * 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.io.IOException; +import java.io.OutputStream; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.net.Socket; +import java.util.*; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteEx; +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.TcpDiscoveryNodeAddFinishedMessage; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + + +/** + * We emulate that client receive message about joining to topology earlier than some server nodes in topology. + * And make this client connect to such servers. + * To emulate this we connect client to second node in topology and pause sending message about joining finishing to + * third node. + */ +public class IgniteClientConnectTest extends GridCommonAbstractTest { + + /** Custom cache name. */ + private static final String DEFAULT_CACHE_NAME = "default-cache"; + + /** */ + private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); + + /** Latch to stop message sending. */ + private final CountDownLatch latch = new CountDownLatch(1); + + /** Start client flag. */ + private final AtomicBoolean clientJustStarted = new AtomicBoolean(false); + + /** Instance name. */ + private String igniteInstanceName; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + this.igniteInstanceName = igniteInstanceName; + + TestTcpDiscoverySpi disco = new TestTcpDiscoverySpi(); + + if (igniteInstanceName.equals("client")) { + TcpDiscoveryVmIpFinder ipFinder = new TcpDiscoveryVmIpFinder(); + + ipFinder.registerAddresses(Collections.singleton(new InetSocketAddress(InetAddress.getLoopbackAddress(), 47501))); + + disco.setIpFinder(ipFinder); + } + else + disco.setIpFinder(ipFinder); + + disco.setJoinTimeout(2 * 60_000); + disco.setSocketTimeout(1000); + disco.setNetworkTimeout(2000); + + cfg.setDiscoverySpi(disco); + + CacheConfiguration cacheConfiguration = new CacheConfiguration() + .setName(DEFAULT_CACHE_NAME) + .setCacheMode(CacheMode.PARTITIONED) + .setAffinity(new RendezvousAffinityFunction(false, 8)) + .setBackups(0); + + cfg.setCacheConfiguration(cacheConfiguration); + + return cfg; + } + + /** + * + * @throws Exception If failed. + */ + public void testClientConnectToBigTopology() throws Exception { + Ignite ignite = startGrids(3); + + IgniteCache cache = ignite.cache(DEFAULT_CACHE_NAME); + + Set keys = new HashSet<>(); + + for (int i = 0; i < 80; i++) { + cache.put(i, i); + + keys.add(i); + } + + TcpDiscoveryImpl discovery = ((TestTcpDiscoverySpi) ignite.configuration().getDiscoverySpi()).discovery(); + + assertTrue(discovery instanceof ServerImpl); + + IgniteConfiguration clientCfg = getConfiguration("client"); + + clientCfg.setClientMode(true); + + clientJustStarted.set(true); + + Ignite client = startGrid(igniteInstanceName, clientCfg); + + latch.countDown(); + + System.err.println("GET ALL"); + client.cache(DEFAULT_CACHE_NAME).getAll(keys); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** + * + */ + class TestTcpDiscoverySpi extends TcpDiscoverySpi { + /** {@inheritDoc} */ + protected void writeToSocket(Socket sock, OutputStream out, TcpDiscoveryAbstractMessage msg, long timeout) throws IOException, + IgniteCheckedException { + if (msg instanceof TcpDiscoveryNodeAddFinishedMessage) { + if (msg.senderNodeId() != null && clientJustStarted.get()) + try { + latch.await(); + + Thread.sleep(3000); + } catch (InterruptedException e) { + e.printStackTrace(); + } + + super.writeToSocket(sock, out, msg, timeout); + } + else + super.writeToSocket(sock, out, msg, timeout); + } + + /** + * + */ + TcpDiscoveryImpl discovery() { + return impl; + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java index fa470e911790a..c506ca7e95bf2 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java @@ -20,6 +20,7 @@ import junit.framework.TestSuite; import org.apache.ignite.spi.GridTcpSpiForwardingSelfTest; import org.apache.ignite.spi.discovery.AuthenticationRestartTest; +import org.apache.ignite.spi.discovery.tcp.IgniteClientConnectTest; import org.apache.ignite.spi.discovery.tcp.IgniteClientReconnectMassiveShutdownTest; import org.apache.ignite.spi.discovery.tcp.TcpClientDiscoveryMarshallerCheckSelfTest; import org.apache.ignite.spi.discovery.tcp.TcpClientDiscoverySpiFailureTimeoutSelfTest; @@ -91,6 +92,8 @@ public static TestSuite suite() throws Exception { suite.addTest(new TestSuite(TcpDiscoveryNodeAttributesUpdateOnReconnectTest.class)); suite.addTest(new TestSuite(AuthenticationRestartTest.class)); + //Client connect + suite.addTest(new TestSuite(IgniteClientConnectTest.class)); suite.addTest(new TestSuite(IgniteClientReconnectMassiveShutdownTest.class)); // SSL. From dbc340a37664f8e5313ab085af34f59ce963032b Mon Sep 17 00:00:00 2001 From: Konstantin Boudnik Date: Mon, 5 Jun 2017 19:47:02 -0700 Subject: [PATCH 260/357] 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 261/357] 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 262/357] 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 263/357] 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 264/357] 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 265/357] 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 266/357] 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 04143cfacf887b8bd1fd10f807b97aaa2af61ead Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Tue, 5 Sep 2017 19:11:12 +0300 Subject: [PATCH 267/357] GG-12699: Fix GridCacheAbstractFullApiSelfTest.testTransformResourceInjection. Squashed commit of the following: commit 3f20fe8dcc796406f7a7791e3ae9ddb5c26183ca Author: Nikolay Izhikov Date: Wed Aug 9 13:37:11 2017 +0300 IGNITE-5897 Fix session init/end logic. This fixes tests. Signed-off-by: nikolay_tikhonov (cherry picked from commit 5a559df) commit 52e89d387874a0653c58a608cc000950a76fb6b0 Author: dpavlov Date: Wed Jul 26 17:23:05 2017 +0300 IGNITE-5806 - Fixed assertion with a side-effect - Fixes #2335. Signed-off-by: Alexey Goncharuk (cherry picked from commit 9e79c4b) commit d3c40e418dce6ab640fe06e8c18ada4b93f1edf5 Author: Andrey V. Mashenkov Date: Mon Sep 4 15:57:16 2017 +0300 Fix javadoc. (cherry picked from commit d4f2885) --- .../cache/store/GridCacheStoreManagerAdapter.java | 3 ++- .../internal/processors/resource/GridResourceProcessor.java | 6 +++++- 2 files changed, 7 insertions(+), 2 deletions(-) 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 14ec92237c3d4..142e5ae28f67f 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 @@ -879,7 +879,8 @@ private void sessionEnd0(@Nullable IgniteInternalTx tx, boolean threwEx) throws lsnr.onSessionEnd(locSes, !threwEx); } - store.sessionEnd(!threwEx); + if (!sesHolder.get().ended(store)) + store.sessionEnd(!threwEx); } } catch (Exception e) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceProcessor.java index 84d07b64e50e5..02616b5e983a0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceProcessor.java @@ -20,7 +20,6 @@ import java.lang.annotation.Annotation; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; -import java.util.Collection; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cache.store.CacheStoreSession; import org.apache.ignite.compute.ComputeJob; @@ -318,6 +317,11 @@ private GridResourceInjector injectorByAnnotation(GridResourceIoc.ResourceAnnota /** * @param obj Object to inject. + * @param ann Annotation enum. + * @param dep Grid deployment object. + * @param depCls Grid deployment class. + * @param param Resource to inject. + * @return {@code True} if resource was injected. * @throws IgniteCheckedException If failed to inject. */ private boolean inject(Object obj, GridResourceIoc.ResourceAnnotation ann, @Nullable GridDeployment dep, From 733ca74fcc094c83a3c4769b291c87d56e17c4a9 Mon Sep 17 00:00:00 2001 From: Vyacheslav Daradur Date: Fri, 10 Feb 2017 16:51:37 +0300 Subject: [PATCH 268/357] GG-12647: Backport IGNITE-3196 Add support for BigDecimals with negative scale in BinaryMarshaller (cherry picked from commit 5efefcb) Signed-off-by: nikolay_tikhonov --- .../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 1153d155c0400..6831ef9bb0d66 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 adaacdda40676..3289780e3fd0e 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 @@ -403,18 +403,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 cd8a487d6d876..31d6f31eef961 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 22191eb8b045e..2c8f9f3a3cdab 100644 --- a/modules/platforms/cpp/odbc/src/utility.cpp +++ b/modules/platforms/cpp/odbc/src/utility.cpp @@ -87,8 +87,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); @@ -101,14 +107,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 71a4718130475..3a3100a6b3131 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs @@ -817,16 +817,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 9e3a0ad5d6144525ef5b058bcf3e6429611bec1b Mon Sep 17 00:00:00 2001 From: Evgeny Stanilovskiy Date: Tue, 5 Sep 2017 19:36:59 +0300 Subject: [PATCH 269/357] IGNITE-4557 "Fixed wrong affinity manager call" Signed-off-by: nikolay_tikhonov --- .../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 f0a263e74540c..aab0ed46fba15 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 d9ff61684e7d0..8d335fe46b397 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 33d0085f3ff7c..af8f016a146f7 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 @@ -955,26 +955,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 6f95e0d6a1e7e..417eb3536536f 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 @@ -934,7 +934,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 a0ccc28c33b0c..44d3f92160770 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 @@ -616,7 +616,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); } /** @@ -624,7 +624,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 628db37ec3ef9..0cea80de02aac 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 @@ -576,7 +576,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) { @@ -699,7 +699,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 8a8221b428f13..ffe68ed2cfee7 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 @@ -2490,7 +2490,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); @@ -2580,7 +2580,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, @@ -2711,7 +2711,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; @@ -2734,7 +2734,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; @@ -2773,7 +2773,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; @@ -2869,7 +2870,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); @@ -1287,7 +1287,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 260533d948e0e..c2b0c1390def7 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 @@ -625,7 +625,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; @@ -695,7 +695,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); } @@ -1386,7 +1386,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; @@ -1463,7 +1463,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 2efaed859b714..2d2e7a8154a69 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 d7a0fb5f8213d..d3e3a15831bb2 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 @@ -1373,7 +1373,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 c464b36217fd8..a8448dcfd9680 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 @@ -529,7 +529,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 b314b81bf7b50..606d70ffb17fe 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 @@ -605,7 +605,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 f9a2f90787b3c..a4132f22b2b6b 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 @@ -196,7 +196,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 9acab56ae2d8e..aed1ab0ac18de 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 @@ -350,7 +350,7 @@ else if (err != null) 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 26c361498fbab..ceebbe96dbd56 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 9c300e70adab6..acd5bf1a454a9 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 cd5babe7118b0..266c5a83cd8c9 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 @@ -1293,7 +1293,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. @@ -1712,7 +1712,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 7f161859da101..0d992286114b5 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 @@ -1106,7 +1106,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); } /** @@ -1407,7 +1407,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 f6a06c29a9fbe..66cfc4ee7338c 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 @@ -289,7 +289,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 66fb7ae94f5be..e4b0c1feb7a21 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 @@ -2106,7 +2106,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 61e2809fa3a65b96d283a44de4c9e42c4dae1a0d Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Wed, 6 Sep 2017 11:54:19 +0300 Subject: [PATCH 270/357] IGNITE-6219 - IgniteCache#loadCache executes local load in caller thread (cherry picked from commit 0e63f59) --- .../processors/task/GridTaskWorker.java | 22 +++- .../cache/store/GridStoreLoadCacheTest.java | 120 ++++++++++++++++++ .../testsuites/IgniteCacheTestSuite.java | 5 +- 3 files changed, 145 insertions(+), 2 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/cache/store/GridStoreLoadCacheTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java index c426008f5fb11..8787f4b92b9bb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java @@ -579,7 +579,7 @@ private void processMappedJobs(Map jobs) thro if (F.isEmpty(jobs)) return; - Collection jobResList = new ArrayList<>(jobs.size()); + List jobResList = new ArrayList<>(jobs.size()); Collection sibs = new ArrayList<>(jobs.size()); @@ -637,6 +637,26 @@ private void processMappedJobs(Map jobs) thro // Set mapped flag. ses.onMapped(); + // Move local jobs to the end of the list, because + // they will be invoked in current thread that will hold other + // jobs. + int jobResSize = jobResList.size(); + + if (jobResSize > 1) { + UUID locId = ctx.discovery().localNode().id(); + + for (int i = 0; i < jobResSize; i++) { + UUID jobNodeId = jobResList.get(i).getNode().id(); + + if (jobNodeId.equals(locId) && i < jobResSize - 1) { + Collections.swap(jobResList, i, jobResSize - 1); + + jobResSize--; + i--; + } + } + } + // Send out all remote mappedJobs. for (GridJobResultImpl res : jobResList) { evtLsnr.onJobSend(this, res.getSibling()); diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/GridStoreLoadCacheTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/GridStoreLoadCacheTest.java new file mode 100644 index 0000000000000..d88c4318ee27e --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/cache/store/GridStoreLoadCacheTest.java @@ -0,0 +1,120 @@ +/* + * 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.store; + +import java.util.concurrent.BrokenBarrierException; +import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import javax.cache.Cache; +import javax.cache.configuration.Factory; +import javax.cache.integration.CacheLoaderException; +import javax.cache.integration.CacheWriterException; +import org.apache.ignite.IgniteException; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.lang.IgniteBiInClosure; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Test checks that local cacheLoad task never blocks remote + * cacheLoad. + */ +public class GridStoreLoadCacheTest extends GridCommonAbstractTest { + /** Barrier. */ + private static final CyclicBarrier BARRIER = new CyclicBarrier(3); + + /** Cache name. */ + public static final String CACHE_NAME = "test"; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + //noinspection unchecked + cfg.setCacheConfiguration(new CacheConfiguration(CACHE_NAME).setCacheStoreFactory(new TestFactory())); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** + * @throws Exception If failed. + */ + public void test() throws Exception { + for (int i = 0; i < 3; i++) { + IgniteEx srv1 = startGrid(0); + startGrid(1); + startGrid(2); + + awaitPartitionMapExchange(); + + srv1.cache(CACHE_NAME).loadCache(null); + + stopAllGrids(); + } + } + + /** + * + */ + private static class TestFactory implements Factory { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Override public CacheStore create() { + return new TestStore(); + } + } + + /** + * + */ + private static class TestStore extends CacheStoreAdapter { + /** {@inheritDoc} */ + @Override public void loadCache(IgniteBiInClosure clo, Object... args) { + try { + BARRIER.await(3, TimeUnit.SECONDS); + } + catch (InterruptedException | BrokenBarrierException | TimeoutException e) { + throw new IgniteException(e); + } + } + + /** {@inheritDoc} */ + @Override public Object load(Object 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 + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java index a24f020103e54..995a2023b8de9 100755 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java @@ -31,6 +31,7 @@ import org.apache.ignite.cache.affinity.local.LocalAffinityFunctionTest; import org.apache.ignite.cache.store.GridCacheBalancingStoreSelfTest; import org.apache.ignite.cache.store.GridCacheLoadOnlyStoreAdapterSelfTest; +import org.apache.ignite.cache.store.GridStoreLoadCacheTest; import org.apache.ignite.cache.store.StoreResourceInjectionSelfTest; import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreBinaryMarshallerSelfTest; import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreBinaryMarshallerWithSqlEscapeSelfTest; @@ -40,12 +41,12 @@ import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreTest; import org.apache.ignite.cache.store.jdbc.GridCacheJdbcBlobStoreMultithreadedSelfTest; import org.apache.ignite.cache.store.jdbc.GridCacheJdbcBlobStoreSelfTest; +import org.apache.ignite.cache.store.jdbc.JdbcTypesDefaultTransformerTest; import org.apache.ignite.internal.managers.communication.IgniteCommunicationBalanceMultipleConnectionsTest; import org.apache.ignite.internal.managers.communication.IgniteCommunicationBalancePairedConnectionsTest; import org.apache.ignite.internal.managers.communication.IgniteCommunicationBalanceTest; import org.apache.ignite.internal.managers.communication.IgniteIoTestMessagesTest; import org.apache.ignite.internal.managers.communication.IgniteVariousConnectionNumberTest; -import org.apache.ignite.cache.store.jdbc.JdbcTypesDefaultTransformerTest; import org.apache.ignite.internal.processors.cache.CacheAffinityCallSelfTest; import org.apache.ignite.internal.processors.cache.CacheDeferredDeleteSanitySelfTest; import org.apache.ignite.internal.processors.cache.CacheEntryProcessorCopySelfTest; @@ -346,6 +347,8 @@ public static TestSuite suite(Set ignoredTests) throws Exception { suite.addTestSuite(IgniteCommunicationBalanceMultipleConnectionsTest.class); suite.addTestSuite(IgniteIoTestMessagesTest.class); + suite.addTestSuite(GridStoreLoadCacheTest.class); + return suite; } } From 5b1fdfaee7422c02177513f6f8c1365a262fd49b Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Wed, 6 Sep 2017 18:33:31 +0300 Subject: [PATCH 271/357] IGNITE-6256: DiscoCache should always contains local node. --- .../managers/discovery/GridDiscoveryManager.java | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) 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 40dea984ee56d..8a625fc728da3 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 @@ -613,7 +613,7 @@ else if (type == EVT_CLIENT_NODE_DISCONNECTED) { topHist.clear(); topSnap.set(new Snapshot(AffinityTopologyVersion.ZERO, - createDiscoCache(locNode, Collections.emptySet()))); + createDiscoCache(locNode, Collections.singleton(locNode)))); } else if (type == EVT_CLIENT_NODE_RECONNECTED) { assert locNode.isClient() : locNode; @@ -1969,6 +1969,8 @@ public void reconnect() { * @return Newly created discovery cache. */ @NotNull private DiscoCache createDiscoCache(ClusterNode loc, Collection topSnapshot) { + assert topSnapshot.contains(loc); + HashSet alives = U.newHashSet(topSnapshot.size()); HashMap nodeMap = U.newHashMap(topSnapshot.size()); @@ -2172,15 +2174,15 @@ public void scheduleSegmentCheck() { lastChk = now; if (!segValid) { - List empty = Collections.emptyList(); - ClusterNode node = getSpi().getLocalNode(); + Collection locNodeOnlyTopology = Collections.singleton(node); + discoWrk.addEvent(EVT_NODE_SEGMENTED, AffinityTopologyVersion.NONE, node, - createDiscoCache(node, empty), - empty, + createDiscoCache(node, locNodeOnlyTopology), + locNodeOnlyTopology, null); lastSegChkRes.set(false); From e1f1a51670bff70c4cd906505e0df6fa85b7a38c Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Tue, 5 Sep 2017 19:11:12 +0300 Subject: [PATCH 272/357] GG-12699: Fix GridCacheAbstractFullApiSelfTest.testTransformResourceInjection. Squashed commit of the following: commit 3f20fe8dcc796406f7a7791e3ae9ddb5c26183ca Author: Nikolay Izhikov Date: Wed Aug 9 13:37:11 2017 +0300 IGNITE-5897 Fix session init/end logic. This fixes tests. Signed-off-by: nikolay_tikhonov (cherry picked from commit 5a559df) commit 52e89d387874a0653c58a608cc000950a76fb6b0 Author: dpavlov Date: Wed Jul 26 17:23:05 2017 +0300 IGNITE-5806 - Fixed assertion with a side-effect - Fixes #2335. Signed-off-by: Alexey Goncharuk (cherry picked from commit 9e79c4b) commit d3c40e418dce6ab640fe06e8c18ada4b93f1edf5 Author: Andrey V. Mashenkov Date: Mon Sep 4 15:57:16 2017 +0300 Fix javadoc. --- .../cache/store/CacheStoreManager.java | 4 +++- .../store/GridCacheStoreManagerAdapter.java | 23 +++++++++---------- .../cache/transactions/IgniteTxAdapter.java | 9 ++++++-- .../resource/GridResourceProcessor.java | 6 ++++- 4 files changed, 26 insertions(+), 16 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java index 67c9334fa6ed2..459c7029fbde8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java @@ -165,9 +165,11 @@ public boolean removeAll(@Nullable IgniteInternalTx tx, Collection keys) /** * @param tx Transaction. * @param commit Commit. + * @param last {@code True} if this is last store in transaction. + * @param storeSessionEnded {@code True} if session for underlying store already ended. * @throws IgniteCheckedException If failed. */ - public void sessionEnd(IgniteInternalTx tx, boolean commit, boolean last) throws IgniteCheckedException; + public void sessionEnd(IgniteInternalTx tx, boolean commit, boolean last, boolean storeSessionEnded) throws IgniteCheckedException; /** * End session initiated by write-behind store. 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 c35b8fb9d2c01..6ab8c56578593 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 @@ -777,7 +777,8 @@ private void loadAllFromStore(@Nullable IgniteInternalTx tx, } /** {@inheritDoc} */ - @Override public void sessionEnd(IgniteInternalTx tx, boolean commit, boolean last) throws IgniteCheckedException { + @Override public void sessionEnd(IgniteInternalTx tx, boolean commit, boolean last, + boolean storeSessionEnded) throws IgniteCheckedException { assert store != null; sessionInit0(tx); @@ -788,7 +789,7 @@ private void loadAllFromStore(@Nullable IgniteInternalTx tx, lsnr.onSessionEnd(locSes, commit); } - if (!sesHolder.get().ended(store)) + if (!sesHolder.get().ended(store) && !storeSessionEnded) store.sessionEnd(commit); } catch (Throwable e) { @@ -855,7 +856,7 @@ private void sessionInit0(@Nullable IgniteInternalTx tx) { sesHolder.set(ses); - if (sesLsnrs != null && !ses.started(this)) { + if (!ses.started(store) && sesLsnrs != null) { for (CacheStoreSessionListener lsnr : sesLsnrs) lsnr.onSessionStart(locSes); } @@ -872,7 +873,8 @@ private void sessionEnd0(@Nullable IgniteInternalTx tx, boolean threwEx) throws lsnr.onSessionEnd(locSes, !threwEx); } - store.sessionEnd(!threwEx); + if (!sesHolder.get().ended(store)) + store.sessionEnd(!threwEx); } } catch (Exception e) { @@ -914,11 +916,8 @@ private static class SessionData { private Object attachment; /** */ - private final Set started = - new GridSetWrapper<>(new IdentityHashMap()); - - /** */ - private final Set ended = new GridSetWrapper<>(new IdentityHashMap()); + private final Set started = + new GridSetWrapper<>(new IdentityHashMap()); /** * @param tx Current transaction. @@ -981,8 +980,8 @@ private void cacheName(String cacheName) { /** * @return If session is started. */ - private boolean started(CacheStoreManager mgr) { - return !started.add(mgr); + private boolean started(CacheStore store) { + return !started.add(store); } /** @@ -990,7 +989,7 @@ private boolean started(CacheStoreManager mgr) { * @return Whether session already ended on this store instance. */ private boolean ended(CacheStore store) { - return !ended.add(store); + return !started.remove(store); } /** {@inheritDoc} */ 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 b07a1175c436d..266c5a83cd8c9 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 @@ -27,6 +27,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.IdentityHashMap; import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; @@ -40,6 +41,7 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.cache.CacheWriteSynchronizationMode; +import org.apache.ignite.cache.store.CacheStore; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; @@ -61,6 +63,7 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheVersionConflictContext; import org.apache.ignite.internal.processors.cache.version.GridCacheVersionedEntryEx; import org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException; +import org.apache.ignite.internal.util.GridSetWrapper; import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.lang.GridMetadataAwareAdapter; import org.apache.ignite.internal.util.lang.GridTuple; @@ -1238,13 +1241,15 @@ protected boolean isWriteToStoreFromDhtValid(Collection store * @param commit Commit flag. * @throws IgniteCheckedException In case of error. */ - protected void sessionEnd(Collection stores, boolean commit) throws IgniteCheckedException { + protected void sessionEnd(final Collection stores, boolean commit) throws IgniteCheckedException { Iterator it = stores.iterator(); + Set visited = new GridSetWrapper<>(new IdentityHashMap()); + while (it.hasNext()) { CacheStoreManager store = it.next(); - store.sessionEnd(this, commit, !it.hasNext()); + store.sessionEnd(this, commit, !it.hasNext(), !visited.add(store.store())); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceProcessor.java index 84d07b64e50e5..02616b5e983a0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceProcessor.java @@ -20,7 +20,6 @@ import java.lang.annotation.Annotation; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; -import java.util.Collection; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cache.store.CacheStoreSession; import org.apache.ignite.compute.ComputeJob; @@ -318,6 +317,11 @@ private GridResourceInjector injectorByAnnotation(GridResourceIoc.ResourceAnnota /** * @param obj Object to inject. + * @param ann Annotation enum. + * @param dep Grid deployment object. + * @param depCls Grid deployment class. + * @param param Resource to inject. + * @return {@code True} if resource was injected. * @throws IgniteCheckedException If failed to inject. */ private boolean inject(Object obj, GridResourceIoc.ResourceAnnotation ann, @Nullable GridDeployment dep, From c746277c2c6ce07871ba29fa3cd1bab7a78f8662 Mon Sep 17 00:00:00 2001 From: Denis Mekhanikov Date: Tue, 5 Sep 2017 18:04:54 +0300 Subject: [PATCH 273/357] ignite-5145 Support multiple service deployment in API (cherry picked from commit 0b6da97) --- .../org/apache/ignite/IgniteServices.java | 57 +- .../ignite/internal/IgniteServicesImpl.java | 25 +- .../GridServiceDeploymentCompoundFuture.java | 196 +++++ .../service/GridServiceProcessor.java | 481 ++++++++--- .../service/PreparedConfigurations.java | 53 ++ .../service/ServiceDeploymentException.java | 78 ++ .../util/future/GridCompoundFuture.java | 15 +- .../util/future/GridFutureAdapter.java | 2 +- ...rviceDeploymentCompoundFutureSelfTest.java | 241 ++++++ ...idServiceProcessorBatchDeploySelfTest.java | 765 ++++++++++++++++++ .../testsuites/IgniteKernalSelfTestSuite.java | 4 + 11 files changed, 1786 insertions(+), 131 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceDeploymentCompoundFuture.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/service/PreparedConfigurations.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceDeploymentException.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceDeploymentCompoundFutureSelfTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorBatchDeploySelfTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteServices.java b/modules/core/src/main/java/org/apache/ignite/IgniteServices.java index 8365ec7e9523b..ae3958fc269cf 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteServices.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteServices.java @@ -20,6 +20,7 @@ import java.util.Collection; import org.apache.ignite.cluster.ClusterGroup; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.processors.service.ServiceDeploymentException; import org.apache.ignite.lang.IgniteAsyncSupport; import org.apache.ignite.lang.IgniteAsyncSupported; import org.apache.ignite.resources.IgniteInstanceResource; @@ -154,10 +155,10 @@ public interface IgniteServices extends IgniteAsyncSupport { * * @param name Service name. * @param svc Service instance. - * @throws IgniteException If failed to deploy service. + * @throws ServiceDeploymentException If failed to deploy service. */ @IgniteAsyncSupported - public void deployClusterSingleton(String name, Service svc) throws IgniteException; + public void deployClusterSingleton(String name, Service svc) throws ServiceDeploymentException; /** * Deploys a per-node singleton service. Ignite will guarantee that there is always @@ -170,10 +171,10 @@ public interface IgniteServices extends IgniteAsyncSupport { * * @param name Service name. * @param svc Service instance. - * @throws IgniteException If failed to deploy service. + * @throws ServiceDeploymentException If failed to deploy service. */ @IgniteAsyncSupported - public void deployNodeSingleton(String name, Service svc) throws IgniteException; + public void deployNodeSingleton(String name, Service svc) throws ServiceDeploymentException; /** * Deploys one instance of this service on the primary node for a given affinity key. @@ -204,11 +205,11 @@ public interface IgniteServices extends IgniteAsyncSupport { * @param cacheName Name of the cache on which affinity for key should be calculated, {@code null} for * default cache. * @param affKey Affinity cache key. - * @throws IgniteException If failed to deploy service. + * @throws ServiceDeploymentException If failed to deploy service. */ @IgniteAsyncSupported public void deployKeyAffinitySingleton(String name, Service svc, @Nullable String cacheName, Object affKey) - throws IgniteException; + throws ServiceDeploymentException; /** * Deploys multiple instances of the service on the grid. Ignite will deploy a @@ -238,10 +239,11 @@ public void deployKeyAffinitySingleton(String name, Service svc, @Nullable Strin * @param svc Service instance. * @param totalCnt Maximum number of deployed services in the grid, {@code 0} for unlimited. * @param maxPerNodeCnt Maximum number of deployed services on each node, {@code 0} for unlimited. - * @throws IgniteException If failed to deploy service. + * @throws ServiceDeploymentException If failed to deploy service. */ @IgniteAsyncSupported - public void deployMultiple(String name, Service svc, int totalCnt, int maxPerNodeCnt) throws IgniteException; + public void deployMultiple(String name, Service svc, int totalCnt, int maxPerNodeCnt) + throws ServiceDeploymentException; /** * Deploys multiple instances of the service on the grid according to provided @@ -277,10 +279,31 @@ public void deployKeyAffinitySingleton(String name, Service svc, @Nullable Strin * * * @param cfg Service configuration. - * @throws IgniteException If failed to deploy service. + * @throws ServiceDeploymentException If failed to deploy service. */ @IgniteAsyncSupported - public void deploy(ServiceConfiguration cfg) throws IgniteException; + public void deploy(ServiceConfiguration cfg) throws ServiceDeploymentException; + + /** + * Deploys multiple services described by provided configurations. Depending on specified parameters, multiple + * instances of the same service may be deployed (see {@link ServiceConfiguration}). + * Whenever topology changes, Ignite will automatically rebalance + * the deployed services within cluster to make sure that each node will end up with + * about equal number of deployed instances whenever possible. + * + * If deployment fails, then {@link ServiceDeploymentException} containing a list of failed services will be + * thrown. It is guaranteed that all services that were provided to this method and are not present in the list of + * failed services are successfully deployed by the moment of the exception being thrown. + * + * @param cfgs {@link Collection} of service configurations to be deployed. + * @param allOrNone Specifies behavior in case when errors during deployment occur. If {@code true}, then two + * outcomes are possible: either all services will be deployed, or none of them. If {@code false}, then partial + * deployments are permitted. + * @throws ServiceDeploymentException If failed to deploy services. + * @see IgniteServices#deploy(ServiceConfiguration) + */ + @IgniteAsyncSupported + public void deployAll(Collection cfgs, boolean allOrNone) throws ServiceDeploymentException; /** * Cancels service deployment. If a service with specified name was deployed on the grid, @@ -298,6 +321,20 @@ public void deployKeyAffinitySingleton(String name, Service svc, @Nullable Strin @IgniteAsyncSupported public void cancel(String name) throws IgniteException; + /** + * Cancels services with specified names. + *

    + * Note that depending on user logic, it may still take extra time for a service to + * finish execution, even after it was cancelled. + *

    + * Supports asynchronous execution (see {@link IgniteAsyncSupport}). + * + * @param names Names of services to cancel. + * @throws IgniteException If failed to cancel services. + */ + @IgniteAsyncSupported + public void cancelAll(Collection names) throws IgniteException; + /** * Cancels all deployed services. *

    diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteServicesImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteServicesImpl.java index df6e5df4fc411..a51f3f1ba95fd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteServicesImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteServicesImpl.java @@ -23,6 +23,7 @@ import java.io.ObjectOutput; import java.io.ObjectStreamException; import java.util.Collection; +import java.util.Collections; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteServices; @@ -150,10 +151,17 @@ public IgniteServicesImpl(GridKernalContext ctx, ClusterGroupAdapter prj, boolea @Override public void deploy(ServiceConfiguration cfg) { A.notNull(cfg, "cfg"); + deployAll(Collections.singleton(cfg), false); + } + + /** {@inheritDoc} */ + @Override public void deployAll(Collection cfgs, boolean allOrNone) { + A.notNull(cfgs, "cfgs"); + guard(); try { - saveOrGet(ctx.service().deploy(cfg)); + saveOrGet(ctx.service().deployAll(cfgs, allOrNone)); } catch (IgniteCheckedException e) { throw U.convertException(e); @@ -180,6 +188,21 @@ public IgniteServicesImpl(GridKernalContext ctx, ClusterGroupAdapter prj, boolea } } + /** {@inheritDoc} */ + @Override public void cancelAll(Collection names) { + guard(); + + try { + saveOrGet(ctx.service().cancelAll(names)); + } + catch (IgniteCheckedException e) { + throw U.convertException(e); + } + finally { + unguard(); + } + } + /** {@inheritDoc} */ @Override public void cancelAll() { guard(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceDeploymentCompoundFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceDeploymentCompoundFuture.java new file mode 100644 index 0000000000000..12b88e5f119f5 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceDeploymentCompoundFuture.java @@ -0,0 +1,196 @@ +/* + * 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.service; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.util.future.GridCompoundFuture; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteInClosure; +import org.apache.ignite.services.ServiceConfiguration; +import org.jetbrains.annotations.Nullable; + +/** + * Service deployment compound future, {@code allOrNone} parameter specifies failing policy. + *

    + * If {@code allOrNone} parameter is set to {@code false}, then this future waits for completion of all child futures. + * If any exceptions are thrown during deployment, then {@link IgniteCheckedException} with {@link + * ServiceDeploymentException} as a cause will be thrown from {@link IgniteInternalFuture#get get()} method after all + * futures complete or fail. Inner exception will contain configurations of failed services. + */ +public class GridServiceDeploymentCompoundFuture extends GridCompoundFuture { + /** */ + private final boolean allOrNone; + + /** Kernal context. */ + private final GridKernalContext ctx; + + /** Logger. */ + private final IgniteLogger log; + + /** Names of services written to cache during current deployment. */ + private Collection svcsToRollback; + + /** */ + private volatile ServiceDeploymentException err; + + /** + * @param allOrNone Failing policy. + * @param ctx Kernal context. + */ + GridServiceDeploymentCompoundFuture(boolean allOrNone, GridKernalContext ctx) { + this.allOrNone = allOrNone; + this.ctx = ctx; + this.log = ctx.log(getClass()); + } + + /** {@inheritDoc} */ + @Override protected boolean processFailure(Throwable err, IgniteInternalFuture fut) { + assert fut instanceof GridServiceDeploymentFuture : fut; + + GridServiceDeploymentFuture depFut = (GridServiceDeploymentFuture)fut; + + if (allOrNone) { + if (initialized()) { + onDone(new IgniteCheckedException( + new ServiceDeploymentException("Failed to deploy provided services.", err, getConfigurations()))); + } + else { + synchronized (this) { + if (this.err == null) { + this.err = new ServiceDeploymentException("Failed to deploy provided services.", err, + new ArrayList()); + } + else + this.err.addSuppressed(err); + } + } + } + else { + synchronized (this) { + if (this.err == null) + this.err = new ServiceDeploymentException("Failed to deploy some services.", + new ArrayList()); + + this.err.getFailedConfigurations().add(depFut.configuration()); + this.err.addSuppressed(err); + } + } + + return true; + } + + /** + * Marks this future as initialized. Will complete with error if failures before initialization occurred and + * all-or-none policy is followed. + */ + public void serviceDeploymentMarkInitialized() { + if (allOrNone && this.err != null) { + this.err.getFailedConfigurations().addAll(getConfigurations()); + + onDone(new IgniteCheckedException(this.err)); + } + else + super.markInitialized(); + } + + /** {@inheritDoc} */ + @Override protected boolean onDone(@Nullable final Object res, @Nullable Throwable err, final boolean cancel) { + final Throwable resErr; + + if (err == null && this.err != null) + resErr = new IgniteCheckedException(this.err); + else + resErr = err; + + if (allOrNone && this.err != null && svcsToRollback != null) { + U.warn(log, "Failed to deploy provided services. The following services will be cancelled:" + svcsToRollback); + + IgniteInternalFuture fut = ctx.service().cancelAll(svcsToRollback); + + /* + Can not call fut.get() since it is possible we are in system pool now and + fut also should be completed from system pool. + */ + fut.listen(new IgniteInClosure() { + @Override public void apply(IgniteInternalFuture fut) { + try { + fut.get(); + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to cancel deployed services.", e); + } + finally { + svcsToRollback = null; + } + + GridServiceDeploymentCompoundFuture.super.onDone(res, resErr, cancel); + } + }); + + return false; + } + + return super.onDone(res, resErr, cancel); + } + + /** + * @param fut Child future. + * @param own If {@code true}, then corresponding service will be cancelled on failure. + */ + public void add(GridServiceDeploymentFuture fut, boolean own) { + super.add(fut); + + if (own) { + if (svcsToRollback == null) + svcsToRollback = new ArrayList<>(); + + svcsToRollback.add(fut.configuration().getName()); + } + } + + /** + * @return Collection of names of services that were written to cache during current deployment. + */ + public Collection servicesToRollback() { + if (svcsToRollback != null) + return svcsToRollback; + else + return Collections.emptyList(); + } + + /** + * @return Collection of configurations, stored in child futures. + */ + private Collection getConfigurations() { + Collection> futs = futures(); + + List cfgs = new ArrayList<>(futs.size()); + + for (IgniteInternalFuture fut : futs) + cfgs.add(((GridServiceDeploymentFuture)fut).configuration()); + + return cfgs; + } +} 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 c961eb0443cdc..f457d8091a4ac 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 @@ -18,6 +18,7 @@ package org.apache.ignite.internal.processors.service; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.Comparator; @@ -92,6 +93,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.SecurityException; import org.apache.ignite.plugin.security.SecurityPermission; import org.apache.ignite.resources.IgniteInstanceResource; import org.apache.ignite.resources.JobContextResource; @@ -101,6 +103,7 @@ import org.apache.ignite.services.ServiceDescriptor; import org.apache.ignite.spi.IgniteNodeValidationResult; import org.apache.ignite.thread.IgniteThreadFactory; +import org.apache.ignite.transactions.Transaction; import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentHashMap8; @@ -111,6 +114,7 @@ import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_SERVICES_COMPATIBILITY_MODE; import static org.apache.ignite.internal.processors.cache.GridCacheUtils.UTILITY_CACHE_NAME; 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; /** @@ -294,19 +298,13 @@ public void onContinuousProcessorStarted(GridKernalContext ctx) throws IgniteChe ServiceConfiguration[] cfgs = ctx.config().getServiceConfiguration(); if (cfgs != null) { - Collection> futs = new ArrayList<>(); - 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) - f.get(); + deployAll(Arrays.asList(cfgs), true).get(); } if (log.isDebugEnabled()) @@ -487,130 +485,276 @@ public IgniteInternalFuture deployKeyAffinitySingleton(String name, Service s } /** - * @param cfg Service configuration. - * @return Future for deployment. + * @param cfgs Service configurations. + * @param allOrNone Failure processing policy. + * @return Configurations to deploy. */ - public IgniteInternalFuture deploy(ServiceConfiguration cfg) { - A.notNull(cfg, "cfg"); + private PreparedConfigurations prepareServiceConfigurations(Collection cfgs, boolean allOrNone) { + List cfgsCp = new ArrayList<>(cfgs.size()); ServicesCompatibilityState state = markCompatibilityStateAsUsed(); - validate(cfg); - - ctx.security().authorize(cfg.getName(), SecurityPermission.SERVICE_DEPLOY, null); + Marshaller marsh = ctx.config().getMarshaller(); - if (!state.srvcCompatibility) { - Marshaller marsh = ctx.config().getMarshaller(); + List failedFuts = null; - LazyServiceConfiguration cfg0; + for (ServiceConfiguration cfg : cfgs) { + Exception err = null; try { - byte[] srvcBytes = U.marshal(marsh, cfg.getService()); - - cfg0 = new LazyServiceConfiguration(cfg, srvcBytes); + validate(cfg); } - catch (IgniteCheckedException e) { - U.error(log, "Failed to marshal service with configured marshaller [srvc=" + cfg.getService() - + ", marsh=" + marsh + "]", e); + catch (Exception e) { + U.error(log, "Failed to validate service configuration [name=" + cfg.getName() + + ", srvc=" + cfg.getService() + ']', e); - return new GridFinishedFuture<>(e); + err = e; } - cfg = cfg0; - } + if (err == null) { + try { + ctx.security().authorize(cfg.getName(), SecurityPermission.SERVICE_DEPLOY, null); + } + catch (Exception e) { + U.error(log, "Failed to authorize service creation [name=" + cfg.getName() + + ", srvc=" + cfg.getService() + ']', e); - GridServiceDeploymentFuture fut = new GridServiceDeploymentFuture(cfg); + err = e; + } + } - GridServiceDeploymentFuture old = depFuts.putIfAbsent(cfg.getName(), fut); + if (err == null) { + if (!state.srvcCompatibility) { + try { + byte[] srvcBytes = U.marshal(marsh, cfg.getService()); - if (old != null) { - if (!old.configuration().equalsIgnoreNodeFilter(cfg)) { - fut.onDone(new IgniteCheckedException("Failed to deploy service (service already exists with " + - "different configuration) [deployed=" + old.configuration() + ", new=" + cfg + ']')); + cfgsCp.add(new LazyServiceConfiguration(cfg, srvcBytes)); + } + catch (Exception e) { + U.error(log, "Failed to marshal service with configured marshaller [name=" + cfg.getName() + + ", srvc=" + cfg.getService() + ", marsh=" + marsh + "]", e); - return fut; + err = e; + } + } + else + cfgsCp.add(cfg); } - return old; - } + if (err != null) { + if (allOrNone) { + return new PreparedConfigurations(null, + null, + new IgniteCheckedException( + new ServiceDeploymentException("None of the provided services were deplyed.", err, cfgs))); + } + else { + if (failedFuts == null) + failedFuts = new ArrayList<>(); - if (ctx.clientDisconnected()) { - fut.onDone(new IgniteClientDisconnectedCheckedException(ctx.cluster().clientReconnectFuture(), - "Failed to deploy service, client node disconnected.")); + GridServiceDeploymentFuture fut = new GridServiceDeploymentFuture(cfg); + + fut.onDone(err); - depFuts.remove(cfg.getName(), fut); + failedFuts.add(fut); + } + } } - while (true) { - try { - GridServiceDeploymentKey key = new GridServiceDeploymentKey(cfg.getName()); + return new PreparedConfigurations(cfgsCp, failedFuts, null); + } - if (ctx.deploy().enabled()) - ctx.cache().context().deploy().ignoreOwnership(true); + /** + * @param cfgs Service configurations. + * @param allOrNone Failure processing policy. + * @return Future for deployment. + */ + public IgniteInternalFuture deployAll(Collection cfgs, boolean allOrNone) { + assert cfgs != null; - try { - GridServiceDeployment dep = (GridServiceDeployment)cache.getAndPutIfAbsent(key, - new GridServiceDeployment(ctx.localNodeId(), cfg)); + PreparedConfigurations srvCfg = prepareServiceConfigurations(cfgs, allOrNone); - if (dep != null) { - if (!dep.configuration().equalsIgnoreNodeFilter(cfg)) { - // Remove future from local map. - depFuts.remove(cfg.getName(), fut); + if (srvCfg.err != null) + return new GridFinishedFuture<>(srvCfg.err); - fut.onDone(new IgniteCheckedException("Failed to deploy service (service already exists with " + - "different configuration) [deployed=" + dep.configuration() + ", new=" + cfg + ']')); - } - else { - Iterator> it = serviceEntries( - ServiceAssignmentsPredicate.INSTANCE); + List cfgsCp = srvCfg.cfgs; + + List failedFuts = srvCfg.failedFuts; + + Collections.sort(cfgsCp, new Comparator() { + @Override public int compare(ServiceConfiguration cfg1, ServiceConfiguration cfg2) { + return cfg1.getName().compareTo(cfg2.getName()); + } + }); - while (it.hasNext()) { - Cache.Entry e = it.next(); + GridServiceDeploymentCompoundFuture res; - if (e.getKey() instanceof GridServiceAssignmentsKey) { - GridServiceAssignments assigns = (GridServiceAssignments)e.getValue(); + while (true) { + res = new GridServiceDeploymentCompoundFuture(allOrNone, ctx); - if (assigns.name().equals(cfg.getName())) { - // Remove future from local map. - depFuts.remove(cfg.getName(), fut); + if (ctx.deploy().enabled()) + ctx.cache().context().deploy().ignoreOwnership(true); - fut.onDone(); + try { + if (cfgsCp.size() == 1) + writeServiceToCache(res, cfgsCp.get(0)); + else if (cfgsCp.size() > 1) { + try (Transaction tx = cache.txStart(PESSIMISTIC, READ_COMMITTED)) { + for (ServiceConfiguration cfg : cfgsCp) { + try { + writeServiceToCache(res, cfg); + } + catch (IgniteCheckedException e) { + if (X.hasCause(e, ClusterTopologyCheckedException.class)) + throw e; // Retry. - break; - } + if (allOrNone) { + for (String name : res.servicesToRollback()) + depFuts.remove(name).onDone(e); + + res.onDone(new IgniteCheckedException(new ServiceDeploymentException( + "Failed to deploy provided services.", e, cfgs))); + + return res; } } - - if (!dep.configuration().equalsIgnoreNodeFilter(cfg)) - U.warn(log, "Service already deployed with different configuration (will ignore) " + - "[deployed=" + dep.configuration() + ", new=" + cfg + ']'); } + + tx.commit(); } } - finally { - if (ctx.deploy().enabled()) - ctx.cache().context().deploy().ignoreOwnership(false); + + break; + } + catch (IgniteException | IgniteCheckedException e) { + for (String name : res.servicesToRollback()) + depFuts.remove(name).onDone(e); + + if (X.hasCause(e, ClusterTopologyCheckedException.class)) { + if (log.isDebugEnabled()) + log.debug("Topology changed while deploying services (will retry): " + e.getMessage()); } + else { + res.onDone(new IgniteCheckedException( + new ServiceDeploymentException("Failed to deploy provided services.", e, cfgs))); - return fut; + return res; + } } - catch (ClusterTopologyCheckedException e) { - if (log.isDebugEnabled()) - log.debug("Topology changed while deploying service (will retry): " + e.getMessage()); + finally { + if (ctx.deploy().enabled()) + ctx.cache().context().deploy().ignoreOwnership(false); } - catch (IgniteCheckedException e) { - if (e.hasCause(ClusterTopologyCheckedException.class)) { - if (log.isDebugEnabled()) - log.debug("Topology changed while deploying service (will retry): " + e.getMessage()); + } - continue; + if (ctx.clientDisconnected()) { + IgniteClientDisconnectedCheckedException err = + new IgniteClientDisconnectedCheckedException(ctx.cluster().clientReconnectFuture(), + "Failed to deploy services, client node disconnected: " + cfgs); + + for (String name : res.servicesToRollback()) { + GridServiceDeploymentFuture fut = depFuts.remove(name); + + if (fut != null) + fut.onDone(err); + } + + return new GridFinishedFuture<>(err); + } + + if (failedFuts != null) { + for (GridServiceDeploymentFuture fut : failedFuts) + res.add(fut, false); + } + + res.serviceDeploymentMarkInitialized(); + + return res; + } + + /** + * @param res Resulting compound future. + * @param cfg Service configuration. + * @throws IgniteCheckedException If operation failed. + */ + private void writeServiceToCache(GridServiceDeploymentCompoundFuture res, ServiceConfiguration cfg) + throws IgniteCheckedException { + String name = cfg.getName(); + + GridServiceDeploymentFuture fut = new GridServiceDeploymentFuture(cfg); + + GridServiceDeploymentFuture old = depFuts.putIfAbsent(name, fut); + + try { + if (old != null) { + if (!old.configuration().equalsIgnoreNodeFilter(cfg)) + throw new IgniteCheckedException("Failed to deploy service (service already exists with different " + + "configuration) [deployed=" + old.configuration() + ", new=" + cfg + ']'); + else { + res.add(old, false); + + return; } + } + + GridServiceDeploymentKey key = new GridServiceDeploymentKey(name); + + GridServiceDeployment dep = (GridServiceDeployment)cache.getAndPutIfAbsent(key, + new GridServiceDeployment(ctx.localNodeId(), cfg)); + + if (dep != null) { + if (!dep.configuration().equalsIgnoreNodeFilter(cfg)) { + String err = "Failed to deploy service (service already exists with different " + + "configuration) [deployed=" + dep.configuration() + ", new=" + cfg + ']'; + + U.error(log, err); + + throw new IgniteCheckedException(err); + } + else { + res.add(fut, false); + + Iterator> it = serviceEntries(ServiceAssignmentsPredicate.INSTANCE); + + while (it.hasNext()) { + Cache.Entry e = it.next(); + + if (e.getKey() instanceof GridServiceAssignmentsKey) { + GridServiceAssignments assigns = (GridServiceAssignments)e.getValue(); - U.error(log, "Failed to deploy service: " + cfg.getName(), e); + if (assigns.name().equals(name)) { + fut.onDone(); - return new GridFinishedFuture<>(e); + depFuts.remove(name, fut); + + break; + } + } + } + } } + else + res.add(fut, true); } + catch (IgniteCheckedException e) { + fut.onDone(e); + + res.add(fut, false); + + depFuts.remove(name, fut); + + throw e; + } + } + + /** + * @param cfg Service configuration. + * @return Future for deployment. + */ + public IgniteInternalFuture deploy(ServiceConfiguration cfg) { + A.notNull(cfg, "cfg"); + + return deployAll(Collections.singleton(cfg), false); } /** @@ -635,37 +779,19 @@ 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<>(); - - GridFutureAdapter old; - - if ((old = undepFuts.putIfAbsent(name, fut)) != null) - fut = old; - else { - GridServiceDeploymentKey key = new GridServiceDeploymentKey(name); - - if (cache.getAndRemove(key) == null) { - // Remove future from local map if service was not deployed. - undepFuts.remove(name); - - fut.onDone(); - } - } - - return fut; + return removeServiceFromCache(name).fut; } - catch (ClusterTopologyCheckedException e) { - if (log.isDebugEnabled()) - log.debug("Topology changed while deploying service (will retry): " + e.getMessage()); - } - catch (IgniteCheckedException e) { - log.error("Failed to undeploy service: " + name, e); + catch (IgniteException | IgniteCheckedException e) { + if (X.hasCause(e, ClusterTopologyCheckedException.class)) { + if (log.isDebugEnabled()) + log.debug("Topology changed while cancelling service (will retry): " + e.getMessage()); + } else { + U.error(log, "Failed to undeploy service: " + name, e); - return new GridFinishedFuture<>(e); + return new GridFinishedFuture<>(e); + } } } } @@ -677,7 +803,7 @@ public IgniteInternalFuture cancel(String name) { public IgniteInternalFuture cancelAll() { Iterator> it = serviceEntries(ServiceDeploymentPredicate.INSTANCE); - GridCompoundFuture res = null; + List svcNames = new ArrayList<>(); while (it.hasNext()) { Cache.Entry e = it.next(); @@ -687,11 +813,68 @@ public IgniteInternalFuture cancelAll() { GridServiceDeployment dep = (GridServiceDeployment)e.getValue(); - if (res == null) - res = new GridCompoundFuture<>(); + svcNames.add(dep.configuration().getName()); + } + + return cancelAll(svcNames); + } + + /** + * @param svcNames Name of service to deploy. + * @return Future. + */ + @SuppressWarnings("unchecked") + public IgniteInternalFuture cancelAll(Collection svcNames) { + List svcNamesCp = new ArrayList<>(svcNames); + + Collections.sort(svcNamesCp); + + GridCompoundFuture res; + + while (true) { + res = null; + + List toRollback = new ArrayList<>(); + + try (Transaction tx = cache.txStart(PESSIMISTIC, READ_COMMITTED)) { + for (String name : svcNames) { + if (res == null) + res = new GridCompoundFuture<>(); + + try { + CancelResult cr = removeServiceFromCache(name); + + if (cr.rollback) + toRollback.add(name); - // Cancel each service separately. - res.add(cancel(dep.configuration().getName())); + res.add(cr.fut); + } + catch (IgniteException | IgniteCheckedException e) { + if (X.hasCause(e, ClusterTopologyCheckedException.class)) + throw e; // Retry. + else { + U.error(log, "Failed to undeploy service: " + name, e); + + res.add(new GridFinishedFuture<>(e)); + } + } + } + + tx.commit(); + + break; + } + catch (IgniteException | IgniteCheckedException e) { + for (String name : toRollback) + undepFuts.remove(name).onDone(e); + + if (X.hasCause(e, ClusterTopologyCheckedException.class)) { + if (log.isDebugEnabled()) + log.debug("Topology changed while cancelling service (will retry): " + e.getMessage()); + } + else + return new GridFinishedFuture<>(e); + } } if (res != null) { @@ -703,6 +886,50 @@ public IgniteInternalFuture cancelAll() { return new GridFinishedFuture<>(); } + /** + * @param name Name of service to remove from internal cache. + * @return Cancellation future and a flag whether it should be completed and removed on error. + * @throws IgniteCheckedException If operation failed. + */ + private CancelResult removeServiceFromCache(String name) throws IgniteCheckedException { + try { + ctx.security().authorize(name, SecurityPermission.SERVICE_CANCEL, null); + } + catch (SecurityException e) { + return new CancelResult(new GridFinishedFuture<>(e), false); + } + + GridFutureAdapter fut = new GridFutureAdapter<>(); + + GridFutureAdapter old = undepFuts.putIfAbsent(name, fut); + + if (old != null) + return new CancelResult(old, false); + else { + GridServiceDeploymentKey key = new GridServiceDeploymentKey(name); + + try { + if (cache.getAndRemove(key) == null) { + // Remove future from local map if service was not deployed. + undepFuts.remove(name, fut); + + fut.onDone(); + + return new CancelResult(fut, false); + } + else + return new CancelResult(fut, true); + } + catch (IgniteCheckedException e) { + undepFuts.remove(name, fut); + + fut.onDone(e); + + throw e; + } + } + } + /** * @param name Service name. * @param timeout If greater than 0 limits task execution time. Cannot be negative. @@ -1854,6 +2081,26 @@ private void undeploy(String name) { } } + /** + * + */ + private static class CancelResult { + /** */ + IgniteInternalFuture fut; + + /** */ + boolean rollback; + + /** + * @param fut Future. + * @param rollback {@code True} if service was cancelled during current call. + */ + CancelResult(IgniteInternalFuture fut, boolean rollback) { + this.fut = fut; + this.rollback = rollback; + } + } + /** * */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/PreparedConfigurations.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/PreparedConfigurations.java new file mode 100644 index 0000000000000..a581e159348a8 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/PreparedConfigurations.java @@ -0,0 +1,53 @@ +/* + * 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.service; + +import java.util.List; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.services.ServiceConfiguration; + +/** + * Result of services validation before deployment. + */ +class PreparedConfigurations { + /** */ + final List cfgs; + + /** */ + final List failedFuts; + + /** */ + final Exception err; + + /** + * @param cfgs Configurations to deploy. + * @param failedFuts Finished futures for failed configurations. + * @param err Error if need to stop deploy. + */ + PreparedConfigurations(List cfgs, List failedFuts, + Exception err) { + this.cfgs = cfgs; + this.failedFuts = failedFuts; + this.err = err; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(PreparedConfigurations.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceDeploymentException.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceDeploymentException.java new file mode 100644 index 0000000000000..32fbf6f254250 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceDeploymentException.java @@ -0,0 +1,78 @@ +/* + * 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.service; + +import java.util.Collection; +import org.apache.ignite.IgniteException; +import org.apache.ignite.services.ServiceConfiguration; +import org.jetbrains.annotations.Nullable; + +/** + * Exception indicating service deployment failure. + */ +public class ServiceDeploymentException extends IgniteException { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private final Collection cfgs; + + /** + * Creates service deployment exception with error message. + * + * @param msg Error message. + * @param cfgs Configurations of services that failed to deploy. + */ + public ServiceDeploymentException(String msg, Collection cfgs) { + super(msg); + + this.cfgs = cfgs; + } + + /** + * Creates service deployment exception with {@link Throwable} as a cause. + * + * @param cause Cause. + * @param cfgs Configurations of services that failed to deploy. + */ + public ServiceDeploymentException(Throwable cause, Collection cfgs) { + super(cause); + + this.cfgs = cfgs; + } + + /** + * Creates service deployment exception with error message and {@link Throwable} as a cause. + * + * @param msg Error message. + * @param cause Cause. + * @param cfgs Configurations of services that failed to deploy. + */ + public ServiceDeploymentException(String msg, @Nullable Throwable cause, Collection cfgs) { + super(msg, cause); + + this.cfgs = cfgs; + } + + /** + * @return Configurations of services that failed to deploy. + */ + public Collection getFailedConfigurations() { + return cfgs; + } +} \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java index 7abd3673df9c8..0e9575fe0d70d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java @@ -113,11 +113,11 @@ public GridCompoundFuture(@Nullable IgniteReducer rdc) { } catch (IgniteTxOptimisticCheckedException | IgniteFutureCancelledCheckedException | ClusterTopologyCheckedException e) { - if (!ignoreFailure(e)) + if (!processFailure(e, fut)) onDone(e); } catch (IgniteCheckedException e) { - if (!ignoreFailure(e)) { + if (!processFailure(e, fut)) { U.error(null, "Failed to execute compound future reducer: " + this, e); onDone(e); @@ -182,6 +182,17 @@ protected boolean ignoreFailure(Throwable err) { return false; } + /** + * Processes error thrown by some of the inner futures. + * + * @param err Thrown exception. + * @param fut Failed future. + * @return {@code True} if this error should be ignored. + */ + protected boolean processFailure(Throwable err, IgniteInternalFuture fut) { + return ignoreFailure(err); + } + /** * Checks if there are pending futures. This is not the same as * {@link #isDone()} because child classes may override {@link #onDone(Object, Throwable)} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java index 723dff7a3e38f..257f1996ddd08 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java @@ -361,7 +361,7 @@ public boolean onDone(@Nullable R res, @Nullable Throwable err) { * @param cancel {@code True} if future is being cancelled. * @return {@code True} if result was set by this call. */ - private boolean onDone(@Nullable R res, @Nullable Throwable err, boolean cancel) { + protected boolean onDone(@Nullable R res, @Nullable Throwable err, boolean cancel) { boolean notify = false; try { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceDeploymentCompoundFutureSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceDeploymentCompoundFutureSelfTest.java new file mode 100644 index 0000000000000..51c3407e65da1 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceDeploymentCompoundFutureSelfTest.java @@ -0,0 +1,241 @@ +/* + * 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.service; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException; +import org.apache.ignite.internal.IgniteKernal; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.services.ServiceConfiguration; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** */ +public class GridServiceDeploymentCompoundFutureSelfTest extends GridCommonAbstractTest { + /** */ + private static GridKernalContext ctx; + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + IgniteKernal kernal = (IgniteKernal)startGrid(0); + + ctx = kernal.context(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + } + + /** + * @throws Exception If failed. + */ + public void testWaitForCompletionOnFailingFuturePartial() throws Exception { + GridServiceDeploymentCompoundFuture compFut = new GridServiceDeploymentCompoundFuture(false, ctx); + + int failingFutsNum = 2; + + int completingFutsNum = 5; + + Collection failingFuts = new ArrayList<>(completingFutsNum); + + for (int i = 0; i < failingFutsNum; i++) { + ServiceConfiguration failingCfg = config("Failed-" + i); + + GridServiceDeploymentFuture failingFut = new GridServiceDeploymentFuture(failingCfg); + + failingFuts.add(failingFut); + + compFut.add(failingFut); + } + + List> futs = new ArrayList<>(completingFutsNum); + + for (int i = 0; i < completingFutsNum; i++) { + GridServiceDeploymentFuture fut = new GridServiceDeploymentFuture(config(String.valueOf(i))); + + futs.add(fut); + + compFut.add(fut); + } + + compFut.serviceDeploymentMarkInitialized(); + + List causes = new ArrayList<>(); + + for (GridServiceDeploymentFuture fut : failingFuts) { + Exception cause = new Exception("Test error"); + + causes.add(cause); + + fut.onDone(cause); + } + + try { + compFut.get(100); + + fail("Should never reach here."); + } + catch (IgniteFutureTimeoutCheckedException e) { + log.info("Expected exception: " + e.getMessage()); + } + + for (GridFutureAdapter fut : futs) + fut.onDone(); + + try { + compFut.get(); + + fail("Should never reach here."); + } + catch (IgniteCheckedException ce) { + log.info("Expected exception: " + ce.getMessage()); + + IgniteException e = U.convertException(ce); + + assertTrue(e instanceof ServiceDeploymentException); + + Throwable[] supErrs = e.getSuppressed(); + + assertEquals(failingFutsNum, supErrs.length); + + for (int i = 0; i < failingFutsNum; i++) + assertEquals(causes.get(i), supErrs[i].getCause()); + } + } + + /** + * @throws Exception if failed. + */ + public void testFailAllAfterInitialized() throws Exception { + GridServiceDeploymentCompoundFuture compFut = new GridServiceDeploymentCompoundFuture(true, ctx); + + ServiceConfiguration failingCfg = config("Failed"); + + GridServiceDeploymentFuture failingFut = new GridServiceDeploymentFuture(failingCfg); + + compFut.add(failingFut); + + int futsNum = 5; + + List cfgs = new ArrayList<>(futsNum + 1); + + cfgs.add(failingCfg); + + for (int i = 0; i < futsNum; i++) { + ServiceConfiguration cfg = config(String.valueOf(i)); + + cfgs.add(cfg); + + compFut.add(new GridServiceDeploymentFuture(cfg)); + } + + compFut.serviceDeploymentMarkInitialized(); + + Exception expCause = new Exception("Test error"); + + failingFut.onDone(expCause); + + assertFailAll(compFut, cfgs, expCause); + } + + /** + * @throws Exception if failed. + */ + public void testFailAllBeforeInitialized() throws Exception { + GridServiceDeploymentCompoundFuture compFut = new GridServiceDeploymentCompoundFuture(true, ctx); + + ServiceConfiguration failingCfg = config("Failed"); + + GridServiceDeploymentFuture failingFut = new GridServiceDeploymentFuture(failingCfg); + + Exception expCause = new Exception("Test error"); + + failingFut.onDone(expCause); + + compFut.add(failingFut); + + assertFalse(compFut.isDone()); + + int futsNum = 5; + + List cfgs = new ArrayList<>(futsNum + 1); + + cfgs.add(failingCfg); + + for (int i = 0; i < futsNum; i++) { + ServiceConfiguration cfg = config(String.valueOf(i)); + + cfgs.add(cfg); + + compFut.add(new GridServiceDeploymentFuture(cfg)); + } + + compFut.serviceDeploymentMarkInitialized(); + + assertFailAll(compFut, cfgs, expCause); + } + + /** + * Try waiting for the future completion and check that a proper exception is thrown. + * + * @param fut Future. + * @param expCfgs Expected cfgs. + * @param expCause Expected cause. + */ + private void assertFailAll(GridServiceDeploymentCompoundFuture fut, Collection expCfgs, + Exception expCause) { + try { + fut.get(); + + fail("Should never reach here."); + } + catch (IgniteCheckedException ce) { + log.info("Expected exception: " + ce.getMessage()); + + IgniteException e = U.convertException(ce); + + assertTrue(e instanceof ServiceDeploymentException); + + assertEqualsCollections(expCfgs, ((ServiceDeploymentException)e).getFailedConfigurations()); + + Throwable actCause = e.getCause(); + + assertTrue(actCause instanceof IgniteCheckedException); + + assertEquals(expCause, actCause.getCause()); + } + } + + /** + * @param name Name. + * @return Dummy configuration with a specified name. + */ + private ServiceConfiguration config(String name) { + ServiceConfiguration cfg = new ServiceConfiguration(); + + cfg.setName(name); + + return cfg; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorBatchDeploySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorBatchDeploySelfTest.java new file mode 100644 index 0000000000000..c3232f23e9c2f --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorBatchDeploySelfTest.java @@ -0,0 +1,765 @@ +/* + * 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.service; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteServices; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.IgniteNodeAttributes; +import org.apache.ignite.lang.IgnitePredicate; +import org.apache.ignite.services.ServiceConfiguration; +import org.apache.ignite.services.ServiceDescriptor; +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.common.GridCommonAbstractTest; + +import static org.apache.ignite.testframework.GridTestUtils.runAsync; + +/** + * Test for deployment of multiple configurations at a time. + */ +public class GridServiceProcessorBatchDeploySelfTest extends GridCommonAbstractTest { + /** Number of services to be deployed. */ + private static final int NUM_SERVICES = 100; + + /** Number of nodes in the test cluster. */ + private static final int NUM_NODES = 4; + + /** Client node name. */ + private static final String CLIENT_NODE_NAME = "client"; + + /** IP finder. */ + private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration c = super.getConfiguration(igniteInstanceName); + + TcpDiscoverySpi discoSpi = new TcpDiscoverySpi(); + + discoSpi.setIpFinder(ipFinder); + + c.setDiscoverySpi(discoSpi); + + return c; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + for (int i = 0; i < NUM_NODES; i++) + startGrid(i); + + startGrid(CLIENT_NODE_NAME, getConfiguration(CLIENT_NODE_NAME).setClientMode(true)); + + DummyService.reset(); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** + * @throws Exception If failed. + */ + public void testDeployAll() throws Exception { + Ignite client = grid(CLIENT_NODE_NAME); + + CountDownLatch latch = new CountDownLatch(NUM_SERVICES); + + List cfgs = getConfigs(client.cluster().forServers().predicate(), NUM_SERVICES); + + subscribeExeLatch(cfgs, latch); + + client.services().deployAll(cfgs, false); + + assertTrue("Waiting for services deployment timed out.", latch.await(30, TimeUnit.SECONDS)); + + assertDeployedServices(client, cfgs); + } + + /** + * @throws Exception If failed. + */ + public void testDeployAllAsync() throws Exception { + Ignite client = grid(CLIENT_NODE_NAME); + + CountDownLatch latch = new CountDownLatch(NUM_SERVICES); + + List cfgs = getConfigs(client.cluster().forServers().predicate(), NUM_SERVICES); + + subscribeExeLatch(cfgs, latch); + + IgniteServices services = client.services().withAsync(); + + services.deployAll(cfgs, false); + + services.future().get(); + + assertTrue("Waiting for services deployment timed out.", latch.await(30, TimeUnit.SECONDS)); + + assertDeployedServices(client, cfgs); + } + + /** + * TODO: enable when IGNITE-6259 is fixed + * + * @throws Exception If failed. + */ + public void _testDeployAllTopologyChange() throws Exception { + Ignite client = grid(CLIENT_NODE_NAME); + + final AtomicBoolean finished = new AtomicBoolean(); + + IgniteInternalFuture topChangeFut = runTopChanger(finished); + + try { + int numServices = 500; + int batchSize = 5; + + CountDownLatch latch = new CountDownLatch(numServices); + + IgnitePredicate depPred = client.cluster().forServers() + .forPredicate(new IgnitePredicate() { + @Override public boolean apply(ClusterNode node) { + String gridName = node.attribute(IgniteNodeAttributes.ATTR_GRID_NAME); + + assert gridName != null; + + return gridName.startsWith(getTestGridName()); + } + }).predicate(); + + List cfgs = getConfigs(depPred, numServices); + + subscribeExeLatch(cfgs, latch); + + IgniteServices services = client.services().withAsync(); + + int from = 0; + + while (from < numServices) { + int to = Math.min(numServices, from + batchSize); + + services.deployAll(cfgs.subList(from, to), false); + + services.future().get(5000); + + from = to; + } + + assertTrue(latch.await(30, TimeUnit.SECONDS)); + + assertDeployedServices(client, cfgs); + } + finally { + finished.set(true); + } + + topChangeFut.get(); + } + + /** + * TODO: enable when IGNITE-6259 is fixed + * + * @throws Exception If failed. + */ + public void _testDeployAllTopologyChangeFail() throws Exception { + final Ignite client = grid(CLIENT_NODE_NAME); + + final AtomicBoolean finished = new AtomicBoolean(); + + IgniteInternalFuture topChangeFut = runTopChanger(finished); + + try { + int numServices = 500; + int batchSize = 5; + + CountDownLatch latch = new CountDownLatch(numServices); + + IgnitePredicate depPred = client.cluster().forServers() + .forPredicate(new IgnitePredicate() { + @Override public boolean apply(ClusterNode node) { + String gridName = node.attribute(IgniteNodeAttributes.ATTR_GRID_NAME); + + assert gridName != null; + + return gridName.startsWith(getTestGridName()); + } + }).predicate(); + + List cfgs = getConfigs(depPred, numServices); + + List failingCfgs = new ArrayList<>(); + + subscribeExeLatch(cfgs, latch); + + IgniteServices services = client.services().withAsync(); + + int from = 0; + + while (from < numServices) { + int to = Math.min(numServices, from + batchSize); + + List cfgsBatch = cfgs.subList(from, to); + + ServiceConfiguration failingCfg = cfgsBatch.get(0); + + failingCfg.setName(null); + + failingCfgs.add(failingCfg); + + try { + services.deployAll(cfgsBatch, false); + + services.future().get(5000); + + fail("Should never reach here."); + } + catch (ServiceDeploymentException e) { + assertEquals(1, e.getFailedConfigurations().size()); + + ServiceConfiguration actFailedCfg = copyService(e.getFailedConfigurations().iterator().next()); + + assertEquals(failingCfg, actFailedCfg); + + latch.countDown(); + } + + from = to; + } + + assertTrue(latch.await(30, TimeUnit.SECONDS)); + + cfgs.removeAll(failingCfgs); + + assertDeployedServices(client, cfgs); + } + finally { + finished.set(true); + } + + topChangeFut.get(); + } + + /** + * @throws Exception If failed. + */ + public void testDeployAllFailAll() throws Exception { + deployAllFail(false, true); + } + + /** + * @throws Exception If failed. + */ + public void testDeployAllPartial() throws Exception { + deployAllFail(false, false); + } + + /** + * @throws Exception If failed. + */ + public void testDeployAllAsyncFailAll() throws Exception { + deployAllFail(true, true); + } + + /** + * @throws Exception If failed. + */ + public void testDeployAllAsyncFailPartial() throws Exception { + deployAllFail(true, false); + } + + /** + * @param async If {@code true}, then asynchronous method of deployment will be performed. + * @param allOrNone Failing strategy. + * @throws Exception If failed. + */ + private void deployAllFail(boolean async, boolean allOrNone) throws Exception { + Ignite client = grid(CLIENT_NODE_NAME); + + CountDownLatch latch = new CountDownLatch(NUM_SERVICES - 1); + + List cfgs = getConfigs(client.cluster().forServers().predicate(), NUM_SERVICES); + + subscribeExeLatch(cfgs, latch); + + ServiceConfiguration failingCfg = cfgs.get(cfgs.size() - 1); + + failingCfg.setName(null); + + assertFailingDeploy(client, async, allOrNone, cfgs, failingCfg); + + if (allOrNone) { + assertFalse("Some of the services were deployed.", latch.await(2, TimeUnit.SECONDS)); + + assertEquals(NUM_SERVICES - 1, latch.getCount()); + + assertTrue(client.services().serviceDescriptors().isEmpty()); + } + else { + assertTrue("Waiting for services deployment timed out.", latch.await(30, TimeUnit.SECONDS)); + + assertDeployedServices(client, cfgs.subList(0, cfgs.size() - 1)); + } + } + + /** + * @throws Exception If failed. + */ + public void testClashingNames() throws Exception { + Ignite client = grid(CLIENT_NODE_NAME); + + CountDownLatch latch = new CountDownLatch(NUM_SERVICES); + + List cfgs = getConfigs(client.cluster().forServers().predicate(), NUM_SERVICES); + + subscribeExeLatch(cfgs, latch); + + List fstBatch = cfgs.subList(0, NUM_SERVICES / 2); + List sndBatch = cfgs.subList(NUM_SERVICES / 4, NUM_SERVICES); + + IgniteServices svcs1 = client.services().withAsync(); + IgniteServices svcs2 = client.services().withAsync(); + + svcs1.deployAll(fstBatch, false); + svcs2.deployAll(sndBatch, false); + + svcs1.future().get(); + svcs2.future().get(); + + assertTrue("Waiting for services deployment timed out.", latch.await(30, TimeUnit.SECONDS)); + + assertDeployedServices(client, cfgs); + } + + /** + * @throws Exception If failed. + */ + public void testClashingNamesFailAll() throws Exception { + clashingNamesFail(true); + } + + /** + * @throws Exception If failed. + */ + public void testClashingNamesPartial() throws Exception { + clashingNamesFail(false); + } + + /** + * @param allOrNone Failing strategy. + * @throws Exception If failed. + */ + private void clashingNamesFail(boolean allOrNone) throws Exception { + Ignite client = grid(CLIENT_NODE_NAME); + + List cfgs = getConfigs(client.cluster().forServers().predicate(), NUM_SERVICES); + + int numDepSvcs; + + if (allOrNone) + numDepSvcs = NUM_SERVICES / 2; + else + numDepSvcs = NUM_SERVICES - 1; + + CountDownLatch latch = new CountDownLatch(numDepSvcs); + + List fstBatch = cfgs.subList(0, NUM_SERVICES / 2); + List sndBatch = cfgs.subList(NUM_SERVICES / 4, NUM_SERVICES); + + subscribeExeLatch(cfgs, latch); + + IgniteServices services = client.services().withAsync(); + + services.deployAll(fstBatch, false); + + ServiceConfiguration failingCfg = cfgs.get(NUM_SERVICES - 1); + + failingCfg.setName(null); + + assertFailingDeploy(client, false, allOrNone, sndBatch, failingCfg); + + services.future().get(); + + assertTrue("Waiting for services deployment timed out.", latch.await(30, TimeUnit.SECONDS)); + + assertDeployedServices(client, cfgs.subList(0, numDepSvcs)); + } + + /** + * @throws Exception If failed. + */ + public void testClashingNameDifferentConfigFailAll() throws Exception { + testClashingNameDifferentConfig(true); + } + + /** + * @throws Exception If failed. + */ + public void testClashingNameDifferentConfigPartial() throws Exception { + testClashingNameDifferentConfig(false); + } + + /** + * @param allOrNone Failing strategy. + * @throws Exception If failed. + */ + private void testClashingNameDifferentConfig(boolean allOrNone) throws Exception { + Ignite client = grid(CLIENT_NODE_NAME); + + List cfgs = getConfigs(client.cluster().forServers().predicate(), NUM_SERVICES); + + int numDepSvcs; + + if (allOrNone) + numDepSvcs = NUM_SERVICES / 2; + else + numDepSvcs = NUM_SERVICES - 1; + + + CountDownLatch latch = new CountDownLatch(numDepSvcs); + + List fstBatch = cfgs.subList(0, NUM_SERVICES / 2); + List sndBatch = cfgs.subList(NUM_SERVICES / 4, NUM_SERVICES - 1); + + subscribeExeLatch(cfgs, latch); + + client.services().deployAll(fstBatch, false); + + ServiceConfiguration failingCfg = copyService(cfgs.get(NUM_SERVICES - 1)); + + // Same name, different config. + failingCfg.setName(fstBatch.get(0).getName()); + failingCfg.setTotalCount(fstBatch.get(0).getTotalCount() + 1); + + sndBatch.add(failingCfg); + + assertFailingDeploy(client, false, allOrNone, sndBatch, failingCfg); + + assertTrue("Waiting for services deployment timed out.", latch.await(30, TimeUnit.SECONDS)); + + assertDeployedServices(client, cfgs.subList(0, numDepSvcs)); + } + + /** + * @throws Exception If failed. + */ + public void testCancelAll() throws Exception { + Ignite client = grid(CLIENT_NODE_NAME); + + List cfgs = getConfigs(client.cluster().forServers().predicate(), NUM_SERVICES); + + CountDownLatch latch = new CountDownLatch(NUM_SERVICES); + + subscribeExeLatch(cfgs, latch); + + client.services().deployAll(cfgs, true); + + latch.await(30, TimeUnit.SECONDS); + + client.services().cancelAll(); + + assertDeployedServices(client, Collections.emptyList()); + } + + /** + * @throws Exception If failed. + */ + public void testCancelAllAsync() throws Exception { + Ignite client = grid(CLIENT_NODE_NAME); + + List cfgs = getConfigs(client.cluster().forServers().predicate(), NUM_SERVICES); + + CountDownLatch latch = new CountDownLatch(NUM_SERVICES); + + subscribeExeLatch(cfgs, latch); + + client.services().deployAll(cfgs, true); + + latch.await(30, TimeUnit.SECONDS); + + IgniteServices services = client.services().withAsync(); + + services.cancelAll(); + + services.future().get(); + + assertDeployedServices(client, Collections.emptyList()); + } + + /** + * TODO: enable when IGNITE-6259 is fixed + * + * @throws Exception If failed. + */ + public void _testCancelAllTopologyChange() throws Exception { + Ignite client = grid(CLIENT_NODE_NAME); + + int numServices = 500; + + List cfgs = getConfigs(client.cluster().forServers().predicate(), numServices); + + CountDownLatch latch = new CountDownLatch(numServices); + + subscribeExeLatch(cfgs, latch); + + client.services().deployAll(cfgs, true); + + latch.await(30, TimeUnit.SECONDS); + + final AtomicBoolean finished = new AtomicBoolean(); + + IgniteInternalFuture topChangeFut = runTopChanger(finished); + + List names = new ArrayList<>(); + + for (ServiceConfiguration cfg : cfgs) + names.add(cfg.getName()); + + try { + IgniteServices services = client.services().withAsync(); + + int batchSize = 5; + int from = 0; + + while (from < numServices) { + int to = Math.min(numServices, from + batchSize); + + log.info("Trying to cancel services [" + from + ".." + to + ")"); + + services.cancelAll(names.subList(from, to)); + + services.future().get(5000); + + from = to; + } + + assertDeployedServices(client, Collections.emptyList()); + } + finally { + finished.set(true); + } + + topChangeFut.get(); + } + + /** + * @throws Exception If failed. + */ + public void testCancelAllClashingNames() throws Exception { + Ignite client = grid(CLIENT_NODE_NAME); + + List cfgs = getConfigs(client.cluster().forServers().predicate(), NUM_SERVICES); + + CountDownLatch latch = new CountDownLatch(NUM_SERVICES); + + subscribeExeLatch(cfgs, latch); + + client.services().deployAll(cfgs, true); + + latch.await(30, TimeUnit.SECONDS); + + List names = new ArrayList<>(); + + for (ServiceConfiguration cfg : cfgs) + names.add(cfg.getName()); + + int batchSize = 5; + int from = 0; + + while (from < NUM_SERVICES) { + int to = Math.min(NUM_SERVICES, from + batchSize); + + List toCancel = new ArrayList<>(names.subList(from, to)); + + toCancel.add(toCancel.get(0)); + + client.services().cancelAll(toCancel); + + from = to; + } + + assertDeployedServices(client, Collections.emptyList()); + } + + /** + * @param client Client. + * @param async If {@code true}, then async version of deploy method will be used. + * @param cfgs Service configurations. + * @param allOrNone Failing policy. + * @param failingCfg Configuration of the failing service. + * @throws Exception If failed. + */ + private void assertFailingDeploy(Ignite client, boolean async, boolean allOrNone, List cfgs, + ServiceConfiguration failingCfg) throws Exception { + + IgniteServices services = client.services(); + + if (async) { + services = services.withAsync(); + + services.deployAll(cfgs, allOrNone); + } + + try { + if (async) + services.future().get(); + else + services.deployAll(cfgs, allOrNone); + + fail("Should never reach here."); + } + catch (ServiceDeploymentException e) { + info("Expected exception: " + e.getMessage()); + + Collection expFails; + + if (allOrNone) + expFails = cfgs; + else + expFails = Collections.singleton(failingCfg); + + Collection actFails = e.getFailedConfigurations(); + + // Some cfgs may be lazy. Construct ServiceConfiguration from them for comparison. + Collection actFailsCp = new ArrayList<>(actFails.size()); + + for (ServiceConfiguration cfg : actFails) + actFailsCp.add(copyService(cfg)); + + assertEqualsCollections(expFails, actFailsCp); + } + } + + /** + * @param cfg Config. + * @return Copy of provided configuration. + */ + private ServiceConfiguration copyService(ServiceConfiguration cfg) { + ServiceConfiguration cfgCp = new ServiceConfiguration(); + + cfgCp.setName(cfg.getName()); + + cfgCp.setMaxPerNodeCount(cfg.getMaxPerNodeCount()); + + cfgCp.setTotalCount(cfg.getTotalCount()); + + cfgCp.setAffinityKey(cfg.getAffinityKey()); + + cfgCp.setCacheName(cfg.getCacheName()); + + cfgCp.setName(cfg.getName()); + + cfgCp.setService(cfg.getService()); + + cfgCp.setNodeFilter(cfg.getNodeFilter()); + + return cfgCp; + } + + /** + * @param client Client Ignite instance. + * @param expCfgs Configurations of services that are expected to be deployed. + */ + private void assertDeployedServices(Ignite client, Collection expCfgs) { + Set expNames = new HashSet<>(); + Set actNames = new HashSet<>(); + + for (ServiceConfiguration cfg : expCfgs) + expNames.add(cfg.getName()); + + for (ServiceDescriptor desc : client.services().serviceDescriptors()) + actNames.add(desc.name()); + + assertEquals(expNames, actNames); + } + + /** + * @param nodePred Node predicate. + * @param numServices Number of configurations to generate. + * @return Generated services configurations. + */ + private List getConfigs(IgnitePredicate nodePred, int numServices) { + List cfgs = new ArrayList<>(numServices); + + for (int i = 0; i < numServices; i++) { + String name = "testService-" + i; + + ServiceConfiguration cfg = new ServiceConfiguration(); + + cfg.setName(name); + cfg.setTotalCount(1); + cfg.setMaxPerNodeCount(1); + cfg.setService(new DummyService()); + cfg.setNodeFilter(nodePred); + + cfgs.add(cfg); + } + return cfgs; + } + + /** + * @param cfgs Configurations. + * @param latch Latch. + */ + private void subscribeExeLatch(List cfgs, CountDownLatch latch) { + for (ServiceConfiguration cfg : cfgs) + DummyService.exeLatch(cfg.getName(), latch); + } + + /** + * @param finished Finished flag. + * @throws Exception If failed. + * @return Future. + */ + private IgniteInternalFuture runTopChanger(final AtomicBoolean finished) throws Exception { + return runAsync(new Callable() { + @Override public Object call() throws Exception { + String namePrefix = "extra-node-"; + + int extraNodesNum = 3; + + while (!finished.get()) { + for (int i = 0; i < extraNodesNum; i++) + startGrid(namePrefix + i); + + for (int i = 0; i < extraNodesNum; i++) + stopGrid(namePrefix + i); + + awaitPartitionMapExchange(); + } + + return null; + } + }); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java index 43bd3a44c6388..8ff393c9eea32 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java @@ -50,7 +50,9 @@ import org.apache.ignite.internal.processors.port.GridPortProcessorSelfTest; import org.apache.ignite.internal.processors.service.GridServiceClientNodeTest; import org.apache.ignite.internal.processors.service.GridServiceContinuousQueryRedeploy; +import org.apache.ignite.internal.processors.service.GridServiceDeploymentCompoundFutureSelfTest; import org.apache.ignite.internal.processors.service.GridServicePackagePrivateSelfTest; +import org.apache.ignite.internal.processors.service.GridServiceProcessorBatchDeploySelfTest; import org.apache.ignite.internal.processors.service.GridServiceProcessorMultiNodeConfigSelfTest; import org.apache.ignite.internal.processors.service.GridServiceProcessorMultiNodeSelfTest; import org.apache.ignite.internal.processors.service.GridServiceProcessorProxySelfTest; @@ -146,6 +148,8 @@ public static TestSuite suite(Set ignoredTests) throws Exception { suite.addTestSuite(IgniteServiceDynamicCachesSelfTest.class); suite.addTestSuite(GridServiceContinuousQueryRedeploy.class); suite.addTestSuite(ServiceThreadPoolSelfTest.class); + suite.addTestSuite(GridServiceProcessorBatchDeploySelfTest.class); + suite.addTestSuite(GridServiceDeploymentCompoundFutureSelfTest.class); suite.addTestSuite(IgniteServiceDeploymentClassLoadingDefaultMarshallerTest.class); suite.addTestSuite(IgniteServiceDeploymentClassLoadingOptimizedMarshallerTest.class); From ec4cd64a894f85f20d4ff8d24b143a0da61d1fc5 Mon Sep 17 00:00:00 2001 From: sboikov Date: Tue, 5 Sep 2017 17:03:57 +0300 Subject: [PATCH 274/357] ignite-6273 Moved ServiceDeploymentException to org.apache.ignite.services (cherry picked from commit d7e215d) --- .../core/src/main/java/org/apache/ignite/IgniteServices.java | 2 +- .../service/GridServiceDeploymentCompoundFuture.java | 1 + .../internal/processors/service/GridServiceProcessor.java | 1 + .../service => services}/ServiceDeploymentException.java | 3 +-- .../service/GridServiceDeploymentCompoundFutureSelfTest.java | 1 + .../service/GridServiceProcessorBatchDeploySelfTest.java | 1 + 6 files changed, 6 insertions(+), 3 deletions(-) rename modules/core/src/main/java/org/apache/ignite/{internal/processors/service => services}/ServiceDeploymentException.java (95%) diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteServices.java b/modules/core/src/main/java/org/apache/ignite/IgniteServices.java index ae3958fc269cf..369baebfb6237 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteServices.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteServices.java @@ -20,7 +20,7 @@ import java.util.Collection; import org.apache.ignite.cluster.ClusterGroup; import org.apache.ignite.configuration.IgniteConfiguration; -import org.apache.ignite.internal.processors.service.ServiceDeploymentException; +import org.apache.ignite.services.ServiceDeploymentException; import org.apache.ignite.lang.IgniteAsyncSupport; import org.apache.ignite.lang.IgniteAsyncSupported; import org.apache.ignite.resources.IgniteInstanceResource; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceDeploymentCompoundFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceDeploymentCompoundFuture.java index 12b88e5f119f5..bbf03702a226f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceDeploymentCompoundFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceDeploymentCompoundFuture.java @@ -29,6 +29,7 @@ import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.services.ServiceConfiguration; +import org.apache.ignite.services.ServiceDeploymentException; import org.jetbrains.annotations.Nullable; /** 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 f457d8091a4ac..2a30bad8d4950 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 @@ -100,6 +100,7 @@ import org.apache.ignite.resources.LoggerResource; import org.apache.ignite.services.Service; import org.apache.ignite.services.ServiceConfiguration; +import org.apache.ignite.services.ServiceDeploymentException; import org.apache.ignite.services.ServiceDescriptor; import org.apache.ignite.spi.IgniteNodeValidationResult; import org.apache.ignite.thread.IgniteThreadFactory; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceDeploymentException.java b/modules/core/src/main/java/org/apache/ignite/services/ServiceDeploymentException.java similarity index 95% rename from modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceDeploymentException.java rename to modules/core/src/main/java/org/apache/ignite/services/ServiceDeploymentException.java index 32fbf6f254250..843a8fe37c59a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceDeploymentException.java +++ b/modules/core/src/main/java/org/apache/ignite/services/ServiceDeploymentException.java @@ -15,11 +15,10 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.service; +package org.apache.ignite.services; import java.util.Collection; import org.apache.ignite.IgniteException; -import org.apache.ignite.services.ServiceConfiguration; import org.jetbrains.annotations.Nullable; /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceDeploymentCompoundFutureSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceDeploymentCompoundFutureSelfTest.java index 51c3407e65da1..ca95198cf2fc9 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceDeploymentCompoundFutureSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceDeploymentCompoundFutureSelfTest.java @@ -28,6 +28,7 @@ import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.services.ServiceConfiguration; +import org.apache.ignite.services.ServiceDeploymentException; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; /** */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorBatchDeploySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorBatchDeploySelfTest.java index c3232f23e9c2f..337e69565968b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorBatchDeploySelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorBatchDeploySelfTest.java @@ -35,6 +35,7 @@ import org.apache.ignite.internal.IgniteNodeAttributes; import org.apache.ignite.lang.IgnitePredicate; import org.apache.ignite.services.ServiceConfiguration; +import org.apache.ignite.services.ServiceDeploymentException; import org.apache.ignite.services.ServiceDescriptor; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; From 1831f41d80497012b9bfff38d62099734a3af245 Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Wed, 6 Sep 2017 13:27:25 +0300 Subject: [PATCH 275/357] IGNITE-6271 .NET: Propagate ServiceDeploymentException (cherry picked from commit 56860d2) --- .../services/PlatformAbstractService.java | 8 ++ .../platform/services/PlatformServices.java | 63 ++++++++-- .../platform/utils/PlatformUtils.java | 2 +- .../Services/ServicesAsyncWrapper.cs | 9 +- .../Services/ServicesTest.cs | 80 +++++++++++-- .../Apache.Ignite.Core.csproj | 1 + .../Apache.Ignite.Core/Impl/ExceptionUtils.cs | 6 +- .../Impl/Services/ServiceProxySerializer.cs | 36 ++++++ .../Impl/Services/Services.cs | 29 ++++- .../Impl/Unmanaged/UnmanagedCallbacks.cs | 37 +++++- .../Services/ServiceDeploymentException.cs | 111 ++++++++++++++++++ .../dotnet/Apache.Ignite.sln.DotSettings | 1 + 12 files changed, 350 insertions(+), 33 deletions(-) create mode 100644 modules/platforms/dotnet/Apache.Ignite.Core/Services/ServiceDeploymentException.cs diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformAbstractService.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformAbstractService.java index d6a6e16f0f2c0..fc0fa14ebc5dd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformAbstractService.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformAbstractService.java @@ -94,6 +94,14 @@ public PlatformAbstractService(Object svc, PlatformContext ctx, boolean srvKeepB out.synchronize(); ptr = platformCtx.gateway().serviceInit(mem.pointer()); + + PlatformInputStream in = mem.input(); + + in.synchronize(); + + BinaryRawReaderEx reader = platformCtx.reader(in); + + PlatformUtils.readInvocationResult(platformCtx, reader); } catch (IgniteCheckedException e) { throw U.convertException(e); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java index c266986eb6e76..4259220752647 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java @@ -27,6 +27,7 @@ import org.apache.ignite.internal.processors.platform.PlatformContext; import org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetService; import org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetServiceImpl; +import org.apache.ignite.internal.processors.platform.utils.PlatformFutureUtils; import org.apache.ignite.internal.processors.platform.utils.PlatformUtils; import org.apache.ignite.internal.processors.platform.utils.PlatformWriterBiClosure; import org.apache.ignite.internal.processors.platform.utils.PlatformWriterClosure; @@ -37,6 +38,7 @@ import org.apache.ignite.services.Service; import org.apache.ignite.services.ServiceConfiguration; import org.apache.ignite.services.ServiceDescriptor; +import org.jetbrains.annotations.NotNull; import java.lang.reflect.Method; import java.util.ArrayList; @@ -103,6 +105,9 @@ public class PlatformServices extends PlatformAbstractTarget { private static final CopyOnWriteConcurrentMap, Method> SVC_METHODS = new CopyOnWriteConcurrentMap<>(); + /** Future result writer. */ + private static final PlatformFutureUtils.Writer RESULT_WRITER = new ServiceDeploymentResultWriter(); + /** */ private final IgniteServices services; @@ -147,20 +152,10 @@ private ServiceDescriptor findDescriptor(String name) { @Override protected long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException { switch (type) { - case OP_DOTNET_DEPLOY: { - dotnetDeploy(reader, services); - - return TRUE; - } - case OP_DOTNET_DEPLOY_ASYNC: { dotnetDeploy(reader, servicesAsync); - return readAndListenFuture(reader); - } - - case OP_DOTNET_DEPLOY_MULTIPLE: { - dotnetDeployMultiple(reader, services); + readAndListenFuture(reader, currentFuture(), RESULT_WRITER); return TRUE; } @@ -168,7 +163,9 @@ private ServiceDescriptor findDescriptor(String name) { case OP_DOTNET_DEPLOY_MULTIPLE_ASYNC: { dotnetDeployMultiple(reader, servicesAsync); - return readAndListenFuture(reader); + readAndListenFuture(reader, currentFuture(), RESULT_WRITER); + + return TRUE; } case OP_CANCEL: { @@ -217,6 +214,32 @@ private ServiceDescriptor findDescriptor(String name) { return; } + case OP_DOTNET_DEPLOY: { + try { + dotnetDeploy(reader, services); + + PlatformUtils.writeInvocationResult(writer, null, null); + } + catch (Exception e) { + PlatformUtils.writeInvocationResult(writer, null, e); + } + + return; + } + + case OP_DOTNET_DEPLOY_MULTIPLE: { + try { + dotnetDeployMultiple(reader, services); + + PlatformUtils.writeInvocationResult(writer, null, null); + } + catch (Exception e) { + PlatformUtils.writeInvocationResult(writer, null, e); + } + + return; + } + default: super.processInStreamOutStream(type, reader, writer); } @@ -579,4 +602,20 @@ public void put(K key, V val) { } } } + + /** + * Writes an EventBase. + */ + private static class ServiceDeploymentResultWriter implements PlatformFutureUtils.Writer { + /** */ + @Override public void write(BinaryRawWriterEx writer, Object obj, Throwable err) { + PlatformUtils.writeInvocationResult(writer, obj, err); + } + + /** */ + @Override public boolean canWrite(Object obj, Throwable err) { + return true; + } + } + } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java index 959ff68a4e19f..b1f3cb0ef8d0c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java @@ -765,7 +765,7 @@ public static byte[] errorData(Throwable err) { * @param resObj Result. * @param err Error. */ - public static void writeInvocationResult(BinaryRawWriterEx writer, Object resObj, Exception err) + public static void writeInvocationResult(BinaryRawWriterEx writer, Object resObj, Throwable err) { if (err == null) { writer.writeBoolean(true); diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesAsyncWrapper.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesAsyncWrapper.cs index f0740e04761cf..470804c018292 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesAsyncWrapper.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesAsyncWrapper.cs @@ -104,7 +104,14 @@ public Task DeployMultipleAsync(string name, IService service, int totalCount, i /** */ public void Deploy(ServiceConfiguration configuration) { - _services.DeployAsync(configuration).Wait(); + try + { + _services.DeployAsync(configuration).Wait(); + } + catch (AggregateException ex) + { + throw ex.InnerException; + } } /** */ diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs index 0558d1120496e..26650863c0cd8 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs @@ -431,17 +431,71 @@ public void TestWithKeepBinaryBoth() /// Tests exception in Initialize. /// [Test] - public void TestInitException() + public void TestDeployMultipleException([Values(true, false)] bool keepBinary) + { + VerifyDeploymentException((services, svc) => + services.DeployMultiple(SvcName, svc, Grids.Length, 1), keepBinary); + } + + /// + /// Tests exception in Initialize. + /// + [Test] + public void TestDeployException([Values(true, false)] bool keepBinary) + { + VerifyDeploymentException((services, svc) => + services.Deploy(new ServiceConfiguration + { + Name = SvcName, + Service = svc, + TotalCount = Grids.Length, + MaxPerNodeCount = 1 + }), keepBinary); + } + + /// + /// Verifies the deployment exception. + /// + private void VerifyDeploymentException(Action deploy, bool keepBinary) { var svc = new TestIgniteServiceSerializable { ThrowInit = true }; - var ex = Assert.Throws(() => Services.DeployMultiple(SvcName, svc, Grids.Length, 1)); + var services = Services; + + if (keepBinary) + { + services = services.WithKeepBinary(); + } + + var deploymentException = Assert.Throws(() => deploy(services, svc)); + + var text = keepBinary + ? "Service deployment failed with a binary error. Examine BinaryCause for details." + : "Service deployment failed with an exception. Examine InnerException for details."; + + Assert.AreEqual(text, deploymentException.Message); + + Exception ex; + + if (keepBinary) + { + Assert.IsNull(deploymentException.InnerException); + + ex = deploymentException.BinaryCause.Deserialize(); + } + else + { + Assert.IsNull(deploymentException.BinaryCause); + + ex = deploymentException.InnerException; + } + + Assert.IsNotNull(ex); Assert.AreEqual("Expected exception", ex.Message); - Assert.IsNotNull(ex.InnerException); - Assert.IsTrue(ex.InnerException.Message.Contains("PlatformCallbackUtils.serviceInit(Native Method)")); + Assert.IsTrue(ex.StackTrace.Trim().StartsWith( + "at Apache.Ignite.Core.Tests.Services.ServicesTest.TestIgniteServiceSerializable.Init")); var svc0 = Services.GetService(SvcName); - Assert.IsNull(svc0); } @@ -480,19 +534,27 @@ public void TestCancelException() AssertNoService(); } + /// + /// Tests exception in binarizable implementation. + /// [Test] public void TestMarshalExceptionOnRead() { var svc = new TestIgniteServiceBinarizableErr(); - var ex = Assert.Throws(() => Services.DeployMultiple(SvcName, svc, Grids.Length, 1)); - Assert.AreEqual("Expected exception", ex.Message); + var ex = Assert.Throws(() => + Services.DeployMultiple(SvcName, svc, Grids.Length, 1)); + + Assert.AreEqual("Expected exception", ex.InnerException.Message); var svc0 = Services.GetService(SvcName); Assert.IsNull(svc0); } + /// + /// Tests exception in binarizable implementation. + /// [Test] public void TestMarshalExceptionOnWrite() { @@ -506,6 +568,9 @@ public void TestMarshalExceptionOnWrite() Assert.IsNull(svc0); } + /// + /// Tests Java service invocation. + /// [Test] public void TestCallJavaService() { @@ -777,6 +842,7 @@ private class TestIgniteServiceSerializable : IService, ITestIgniteService /** */ public Guid NodeId { + // ReSharper disable once InconsistentlySynchronizedField get { return _grid.GetCluster().GetLocalNode().Id; } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj index f945efea27d3c..4f9d41e675b08 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj @@ -473,6 +473,7 @@ + diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs index ddbdd86fc559b..e4b2b062e7883 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs @@ -30,6 +30,7 @@ namespace Apache.Ignite.Core.Impl using Apache.Ignite.Core.Common; using Apache.Ignite.Core.Compute; using Apache.Ignite.Core.Impl.Binary; + using Apache.Ignite.Core.Services; using Apache.Ignite.Core.Transactions; /// @@ -103,9 +104,12 @@ static ExceptionUtils() Exs["org.apache.ignite.IgniteAuthenticationException"] = (i, m, e) => new SecurityException(m, e); Exs["org.apache.ignite.plugin.security.GridSecurityException"] = (i, m, e) => new SecurityException(m, e); - // Future exceptions + // Future exceptions. Exs["org.apache.ignite.lang.IgniteFutureCancelledException"] = (i, m, e) => new IgniteFutureCancelledException(m, e); Exs["org.apache.ignite.internal.IgniteFutureCancelledCheckedException"] = (i, m, e) => new IgniteFutureCancelledException(m, e); + + // Service exceptions. + Exs["org.apache.ignite.services.ServiceDeploymentException"] = (i, m, e) => new ServiceDeploymentException(m, e); } /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/ServiceProxySerializer.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/ServiceProxySerializer.cs index 8e44360ff7b65..bd15a7cabaef0 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/ServiceProxySerializer.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/ServiceProxySerializer.cs @@ -154,6 +154,42 @@ public static object ReadInvocationResult(IBinaryStream stream, Marshaller marsh "Examine InnerException for details.", (Exception) err); } + /// + /// Reads service deployment result. + /// + /// Stream. + /// Marshaller. + /// Binary flag. + /// + /// Method invocation result, or exception in case of error. + /// + public static void ReadDeploymentResult(IBinaryStream stream, Marshaller marsh, bool keepBinary) + { + Debug.Assert(stream != null); + Debug.Assert(marsh != null); + + var mode = keepBinary ? BinaryMode.ForceBinary : BinaryMode.Deserialize; + + var reader = marsh.StartUnmarshal(stream, mode); + + object err; + + BinaryUtils.ReadInvocationResult(reader, out err); + + if (err == null) + { + return; + } + + var binErr = err as IBinaryObject; + + throw binErr != null + ? new ServiceDeploymentException("Service deployment failed with a binary error. " + + "Examine BinaryCause for details.", binErr) + : new ServiceDeploymentException("Service deployment failed with an exception. " + + "Examine InnerException for details.", (Exception) err); + } + /// /// Writes the argument in platform-compatible format. /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/Services.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/Services.cs index 88d2a76f4dbf9..2f309313f7f9e 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/Services.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/Services.cs @@ -25,6 +25,7 @@ namespace Apache.Ignite.Core.Impl.Services using Apache.Ignite.Core.Binary; using Apache.Ignite.Core.Cluster; using Apache.Ignite.Core.Impl.Binary; + using Apache.Ignite.Core.Impl.Binary.IO; using Apache.Ignite.Core.Impl.Common; using Apache.Ignite.Core.Impl.Unmanaged; using Apache.Ignite.Core.Services; @@ -204,13 +205,13 @@ public void DeployMultiple(string name, IService service, int totalCount, int ma IgniteArgumentCheck.NotNullOrEmpty(name, "name"); IgniteArgumentCheck.NotNull(service, "service"); - DoOutOp(OpDeployMultiple, w => + DoOutInOp(OpDeployMultiple, w => { w.WriteString(name); w.WriteObject(service); w.WriteInt(totalCount); w.WriteInt(maxPerNodeCount); - }); + }, s => ReadDeploymentResult(s)); } /** */ @@ -225,7 +226,7 @@ public Task DeployMultipleAsync(string name, IService service, int totalCount, i w.WriteObject(service); w.WriteInt(totalCount); w.WriteInt(maxPerNodeCount); - }); + }, _keepBinary, s => ReadDeploymentResult(s)); } /** */ @@ -233,7 +234,7 @@ public void Deploy(ServiceConfiguration configuration) { IgniteArgumentCheck.NotNull(configuration, "configuration"); - DoOutOp(OpDeploy, w => WriteServiceConfiguration(configuration, w)); + DoOutInOp(OpDeploy, w => WriteServiceConfiguration(configuration, w), s => ReadDeploymentResult(s)); } /** */ @@ -241,7 +242,8 @@ public Task DeployAsync(ServiceConfiguration configuration) { IgniteArgumentCheck.NotNull(configuration, "configuration"); - return DoOutOpAsync(OpDeployAsync, w => WriteServiceConfiguration(configuration, w)); + return DoOutOpAsync(OpDeployAsync, w => WriteServiceConfiguration(configuration, w), + _keepBinary, ReadDeploymentResult); } /** */ @@ -400,5 +402,22 @@ private static void WriteServiceConfiguration(ServiceConfiguration configuration else w.WriteObject(null); } + + /// + /// Reads the deployment result. + /// + private object ReadDeploymentResult(BinaryReader r) + { + return r != null ? ReadDeploymentResult(r.Stream) : null; + } + + /// + /// Reads the deployment result. + /// + private object ReadDeploymentResult(IBinaryStream s) + { + ServiceProxySerializer.ReadDeploymentResult(s, Marshaller, _keepBinary); + return null; + } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs index cc205e829d476..cd642ad1a27f6 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs @@ -942,22 +942,47 @@ private void EventFilterDestroy(void* target, long ptr) #region IMPLEMENTATION: SERVICES + [SuppressMessage("Microsoft.Design", "CA1031:DoNotCatchGeneralExceptionTypes", + Justification = "User processor can throw any exception")] private long ServiceInit(void* target, long memPtr) { return SafeCall(() => { using (var stream = IgniteManager.Memory.Get(memPtr).GetStream()) { - var reader = _ignite.Marshaller.StartUnmarshal(stream); + try + { + var reader = _ignite.Marshaller.StartUnmarshal(stream); - bool srvKeepBinary = reader.ReadBoolean(); - var svc = reader.ReadObject(); + var srvKeepBinary = reader.ReadBoolean(); + var svc = reader.ReadObject(); - ResourceProcessor.Inject(svc, _ignite); + ResourceProcessor.Inject(svc, _ignite); - svc.Init(new ServiceContext(_ignite.Marshaller.StartUnmarshal(stream, srvKeepBinary))); + svc.Init(new ServiceContext(_ignite.Marshaller.StartUnmarshal(stream, srvKeepBinary))); - return _handleRegistry.Allocate(svc); + stream.Reset(); + + stream.WriteBool(true); // Success. + + stream.SynchronizeOutput(); + + return _handleRegistry.Allocate(svc); + } + catch (Exception e) + { + stream.Reset(); + + var writer = _ignite.Marshaller.StartMarshal(stream); + + BinaryUtils.WriteInvocationResult(writer, false, e); + + _ignite.Marshaller.FinishMarshal(writer); + + stream.SynchronizeOutput(); + + return 0; + } } }); } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Services/ServiceDeploymentException.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Services/ServiceDeploymentException.cs new file mode 100644 index 0000000000000..825f91e5b3e5e --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Services/ServiceDeploymentException.cs @@ -0,0 +1,111 @@ +/* + * 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. + */ + +namespace Apache.Ignite.Core.Services +{ + using System; + using System.Diagnostics.CodeAnalysis; + using System.Runtime.Serialization; + using Apache.Ignite.Core.Binary; + using Apache.Ignite.Core.Common; + + /// + /// Indicates an error during Grid Services deployment. + /// + [Serializable] + public class ServiceDeploymentException : IgniteException + { + /** Serializer key. */ + private const string KeyBinaryCause = "BinaryCause"; + + /** Cause. */ + private readonly IBinaryObject _binaryCause; + + /// + /// Initializes a new instance of the class. + /// + public ServiceDeploymentException() + { + // No-op. + } + + /// + /// Initializes a new instance of the class. + /// + /// The message that describes the error. + public ServiceDeploymentException(string message) : base(message) + { + // No-op. + } + + /// + /// Initializes a new instance of the class. + /// + /// The message. + /// The cause. + public ServiceDeploymentException(string message, Exception cause) : base(message, cause) + { + // No-op. + } + + /// + /// Initializes a new instance of the class. + /// + /// The message. + /// The binary cause. + public ServiceDeploymentException(string message, IBinaryObject binaryCause) + : base(message) + { + _binaryCause = binaryCause; + } + + /// + /// Initializes a new instance of the class. + /// + /// Serialization information. + /// Streaming context. + protected ServiceDeploymentException(SerializationInfo info, StreamingContext ctx) + : base(info, ctx) + { + _binaryCause = (IBinaryObject)info.GetValue(KeyBinaryCause, typeof(IBinaryObject)); + } + + /// + /// Gets the binary cause. + /// + public IBinaryObject BinaryCause + { + get { return _binaryCause; } + } + + /// + /// When overridden in a derived class, sets the + /// with information about the exception. + /// + /// The that holds the serialized object data + /// about the exception being thrown. + /// The that contains contextual information + /// about the source or destination. + [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods")] + public override void GetObjectData(SerializationInfo info, StreamingContext context) + { + info.AddValue(KeyBinaryCause, _binaryCause); + + base.GetObjectData(info, context); + } + } +} \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.sln.DotSettings b/modules/platforms/dotnet/Apache.Ignite.sln.DotSettings index 078e9fbdb9dbb..9d5b728a9c554 100644 --- a/modules/platforms/dotnet/Apache.Ignite.sln.DotSettings +++ b/modules/platforms/dotnet/Apache.Ignite.sln.DotSettings @@ -7,4 +7,5 @@ True True DO_NOT_SHOW + True \ No newline at end of file From d3be7960878a302d2b156bba89be440a49b7c4ef Mon Sep 17 00:00:00 2001 From: Denis Mekhanikov Date: Thu, 7 Sep 2017 14:47:05 +0300 Subject: [PATCH 276/357] ignite-6289 Remove "allOrNone" flag from IgniteServices#deployAll method --- .../org/apache/ignite/IgniteServices.java | 19 +-- .../ignite/internal/IgniteServicesImpl.java | 6 +- .../GridServiceDeploymentCompoundFuture.java | 129 ++--------------- .../service/GridServiceProcessor.java | 64 +++----- .../service/PreparedConfigurations.java | 8 +- ...rviceDeploymentCompoundFutureSelfTest.java | 121 +--------------- ...idServiceProcessorBatchDeploySelfTest.java | 137 +++++------------- 7 files changed, 81 insertions(+), 403 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteServices.java b/modules/core/src/main/java/org/apache/ignite/IgniteServices.java index 369baebfb6237..edcecd34738ee 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteServices.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteServices.java @@ -286,24 +286,21 @@ public void deployMultiple(String name, Service svc, int totalCnt, int maxPerNod /** * Deploys multiple services described by provided configurations. Depending on specified parameters, multiple - * instances of the same service may be deployed (see {@link ServiceConfiguration}). - * Whenever topology changes, Ignite will automatically rebalance - * the deployed services within cluster to make sure that each node will end up with - * about equal number of deployed instances whenever possible. + * instances of the same service may be deployed (see {@link ServiceConfiguration}). Whenever topology changes, + * Ignite will automatically rebalance the deployed services within cluster to make sure that each node will end up + * with about equal number of deployed instances whenever possible. * - * If deployment fails, then {@link ServiceDeploymentException} containing a list of failed services will be - * thrown. It is guaranteed that all services that were provided to this method and are not present in the list of - * failed services are successfully deployed by the moment of the exception being thrown. + * If deployment of some of the provided services fails, then {@link ServiceDeploymentException} containing a list + * of failed services will be thrown. It is guaranteed that all services that were provided to this method and are + * not present in the list of failed services are successfully deployed by the moment of the exception being thrown. + * Note that if exception is thrown, then partial deployment may have occurred. * * @param cfgs {@link Collection} of service configurations to be deployed. - * @param allOrNone Specifies behavior in case when errors during deployment occur. If {@code true}, then two - * outcomes are possible: either all services will be deployed, or none of them. If {@code false}, then partial - * deployments are permitted. * @throws ServiceDeploymentException If failed to deploy services. * @see IgniteServices#deploy(ServiceConfiguration) */ @IgniteAsyncSupported - public void deployAll(Collection cfgs, boolean allOrNone) throws ServiceDeploymentException; + public void deployAll(Collection cfgs) throws ServiceDeploymentException; /** * Cancels service deployment. If a service with specified name was deployed on the grid, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteServicesImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteServicesImpl.java index a51f3f1ba95fd..58b3a2a96ff1c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteServicesImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteServicesImpl.java @@ -151,17 +151,17 @@ public IgniteServicesImpl(GridKernalContext ctx, ClusterGroupAdapter prj, boolea @Override public void deploy(ServiceConfiguration cfg) { A.notNull(cfg, "cfg"); - deployAll(Collections.singleton(cfg), false); + deployAll(Collections.singleton(cfg)); } /** {@inheritDoc} */ - @Override public void deployAll(Collection cfgs, boolean allOrNone) { + @Override public void deployAll(Collection cfgs) { A.notNull(cfgs, "cfgs"); guard(); try { - saveOrGet(ctx.service().deployAll(cfgs, allOrNone)); + saveOrGet(ctx.service().deployAll(cfgs)); } catch (IgniteCheckedException e) { throw U.convertException(e); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceDeploymentCompoundFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceDeploymentCompoundFuture.java index bbf03702a226f..45ccc24b9fa86 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceDeploymentCompoundFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceDeploymentCompoundFuture.java @@ -20,140 +20,51 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; -import java.util.List; import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.IgniteLogger; -import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.util.future.GridCompoundFuture; -import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.services.ServiceConfiguration; import org.apache.ignite.services.ServiceDeploymentException; import org.jetbrains.annotations.Nullable; /** - * Service deployment compound future, {@code allOrNone} parameter specifies failing policy. - *

    - * If {@code allOrNone} parameter is set to {@code false}, then this future waits for completion of all child futures. - * If any exceptions are thrown during deployment, then {@link IgniteCheckedException} with {@link - * ServiceDeploymentException} as a cause will be thrown from {@link IgniteInternalFuture#get get()} method after all - * futures complete or fail. Inner exception will contain configurations of failed services. + * Service deployment compound future. If any exceptions are thrown during deployment, then {@link + * IgniteCheckedException} with {@link ServiceDeploymentException} as a cause will be thrown from {@link + * IgniteInternalFuture#get get()} method after all futures complete or fail. Inner exception will contain + * configurations of failed services. */ public class GridServiceDeploymentCompoundFuture extends GridCompoundFuture { - /** */ - private final boolean allOrNone; - - /** Kernal context. */ - private final GridKernalContext ctx; - - /** Logger. */ - private final IgniteLogger log; - /** Names of services written to cache during current deployment. */ private Collection svcsToRollback; /** */ private volatile ServiceDeploymentException err; - /** - * @param allOrNone Failing policy. - * @param ctx Kernal context. - */ - GridServiceDeploymentCompoundFuture(boolean allOrNone, GridKernalContext ctx) { - this.allOrNone = allOrNone; - this.ctx = ctx; - this.log = ctx.log(getClass()); - } - /** {@inheritDoc} */ @Override protected boolean processFailure(Throwable err, IgniteInternalFuture fut) { assert fut instanceof GridServiceDeploymentFuture : fut; GridServiceDeploymentFuture depFut = (GridServiceDeploymentFuture)fut; - if (allOrNone) { - if (initialized()) { - onDone(new IgniteCheckedException( - new ServiceDeploymentException("Failed to deploy provided services.", err, getConfigurations()))); - } - else { - synchronized (this) { - if (this.err == null) { - this.err = new ServiceDeploymentException("Failed to deploy provided services.", err, - new ArrayList()); - } - else - this.err.addSuppressed(err); - } + synchronized (this) { + if (this.err == null) { + this.err = new ServiceDeploymentException("Failed to deploy some services.", + new ArrayList()); } - } - else { - synchronized (this) { - if (this.err == null) - this.err = new ServiceDeploymentException("Failed to deploy some services.", - new ArrayList()); - this.err.getFailedConfigurations().add(depFut.configuration()); - this.err.addSuppressed(err); - } + this.err.getFailedConfigurations().add(depFut.configuration()); + this.err.addSuppressed(err); } return true; } - /** - * Marks this future as initialized. Will complete with error if failures before initialization occurred and - * all-or-none policy is followed. - */ - public void serviceDeploymentMarkInitialized() { - if (allOrNone && this.err != null) { - this.err.getFailedConfigurations().addAll(getConfigurations()); - - onDone(new IgniteCheckedException(this.err)); - } - else - super.markInitialized(); - } - /** {@inheritDoc} */ - @Override protected boolean onDone(@Nullable final Object res, @Nullable Throwable err, final boolean cancel) { - final Throwable resErr; - + @Override protected boolean onDone(@Nullable Object res, @Nullable Throwable err, boolean cancel) { if (err == null && this.err != null) - resErr = new IgniteCheckedException(this.err); - else - resErr = err; - - if (allOrNone && this.err != null && svcsToRollback != null) { - U.warn(log, "Failed to deploy provided services. The following services will be cancelled:" + svcsToRollback); - - IgniteInternalFuture fut = ctx.service().cancelAll(svcsToRollback); - - /* - Can not call fut.get() since it is possible we are in system pool now and - fut also should be completed from system pool. - */ - fut.listen(new IgniteInClosure() { - @Override public void apply(IgniteInternalFuture fut) { - try { - fut.get(); - } - catch (IgniteCheckedException e) { - U.error(log, "Failed to cancel deployed services.", e); - } - finally { - svcsToRollback = null; - } - - GridServiceDeploymentCompoundFuture.super.onDone(res, resErr, cancel); - } - }); - - return false; - } + err = new IgniteCheckedException(this.err); - return super.onDone(res, resErr, cancel); + return super.onDone(res, err, cancel); } /** @@ -180,18 +91,4 @@ public Collection servicesToRollback() { else return Collections.emptyList(); } - - /** - * @return Collection of configurations, stored in child futures. - */ - private Collection getConfigurations() { - Collection> futs = futures(); - - List cfgs = new ArrayList<>(futs.size()); - - for (IgniteInternalFuture fut : futs) - cfgs.add(((GridServiceDeploymentFuture)fut).configuration()); - - return cfgs; - } } 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 2a30bad8d4950..8c9ca053baec1 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 @@ -305,7 +305,7 @@ public void onContinuousProcessorStarted(GridKernalContext ctx) throws IgniteChe c.setNodeFilter(ctx.cluster().get().forServers().predicate()); } - deployAll(Arrays.asList(cfgs), true).get(); + deployAll(Arrays.asList(cfgs)).get(); } if (log.isDebugEnabled()) @@ -487,10 +487,9 @@ public IgniteInternalFuture deployKeyAffinitySingleton(String name, Service s /** * @param cfgs Service configurations. - * @param allOrNone Failure processing policy. * @return Configurations to deploy. */ - private PreparedConfigurations prepareServiceConfigurations(Collection cfgs, boolean allOrNone) { + private PreparedConfigurations prepareServiceConfigurations(Collection cfgs) { List cfgsCp = new ArrayList<>(cfgs.size()); ServicesCompatibilityState state = markCompatibilityStateAsUsed(); @@ -543,40 +542,28 @@ private PreparedConfigurations prepareServiceConfigurations(Collection(); + if (failedFuts == null) + failedFuts = new ArrayList<>(); - GridServiceDeploymentFuture fut = new GridServiceDeploymentFuture(cfg); + GridServiceDeploymentFuture fut = new GridServiceDeploymentFuture(cfg); - fut.onDone(err); + fut.onDone(err); - failedFuts.add(fut); - } + failedFuts.add(fut); } } - return new PreparedConfigurations(cfgsCp, failedFuts, null); + return new PreparedConfigurations(cfgsCp, failedFuts); } /** * @param cfgs Service configurations. - * @param allOrNone Failure processing policy. * @return Future for deployment. */ - public IgniteInternalFuture deployAll(Collection cfgs, boolean allOrNone) { + public IgniteInternalFuture deployAll(Collection cfgs) { assert cfgs != null; - PreparedConfigurations srvCfg = prepareServiceConfigurations(cfgs, allOrNone); - - if (srvCfg.err != null) - return new GridFinishedFuture<>(srvCfg.err); + PreparedConfigurations srvCfg = prepareServiceConfigurations(cfgs); List cfgsCp = srvCfg.cfgs; @@ -591,7 +578,7 @@ public IgniteInternalFuture deployAll(Collection cfgs, GridServiceDeploymentCompoundFuture res; while (true) { - res = new GridServiceDeploymentCompoundFuture(allOrNone, ctx); + res = new GridServiceDeploymentCompoundFuture(); if (ctx.deploy().enabled()) ctx.cache().context().deploy().ignoreOwnership(true); @@ -608,16 +595,8 @@ else if (cfgsCp.size() > 1) { catch (IgniteCheckedException e) { if (X.hasCause(e, ClusterTopologyCheckedException.class)) throw e; // Retry. - - if (allOrNone) { - for (String name : res.servicesToRollback()) - depFuts.remove(name).onDone(e); - - res.onDone(new IgniteCheckedException(new ServiceDeploymentException( - "Failed to deploy provided services.", e, cfgs))); - - return res; - } + else + U.error(log, e.getMessage()); } } @@ -651,7 +630,7 @@ else if (cfgsCp.size() > 1) { if (ctx.clientDisconnected()) { IgniteClientDisconnectedCheckedException err = new IgniteClientDisconnectedCheckedException(ctx.cluster().clientReconnectFuture(), - "Failed to deploy services, client node disconnected: " + cfgs); + "Failed to deploy services, client node disconnected: " + cfgs); for (String name : res.servicesToRollback()) { GridServiceDeploymentFuture fut = depFuts.remove(name); @@ -668,7 +647,7 @@ else if (cfgsCp.size() > 1) { res.add(fut, false); } - res.serviceDeploymentMarkInitialized(); + res.markInitialized(); return res; } @@ -705,12 +684,8 @@ private void writeServiceToCache(GridServiceDeploymentCompoundFuture res, Servic if (dep != null) { if (!dep.configuration().equalsIgnoreNodeFilter(cfg)) { - String err = "Failed to deploy service (service already exists with different " + - "configuration) [deployed=" + dep.configuration() + ", new=" + cfg + ']'; - - U.error(log, err); - - throw new IgniteCheckedException(err); + throw new IgniteCheckedException("Failed to deploy service (service already exists with " + + "different configuration) [deployed=" + dep.configuration() + ", new=" + cfg + ']'); } else { res.add(fut, false); @@ -755,7 +730,7 @@ private void writeServiceToCache(GridServiceDeploymentCompoundFuture res, Servic public IgniteInternalFuture deploy(ServiceConfiguration cfg) { A.notNull(cfg, "cfg"); - return deployAll(Collections.singleton(cfg), false); + return deployAll(Collections.singleton(cfg)); } /** @@ -788,7 +763,8 @@ public IgniteInternalFuture cancel(String name) { if (X.hasCause(e, ClusterTopologyCheckedException.class)) { if (log.isDebugEnabled()) log.debug("Topology changed while cancelling service (will retry): " + e.getMessage()); - } else { + } + else { U.error(log, "Failed to undeploy service: " + name, e); return new GridFinishedFuture<>(e); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/PreparedConfigurations.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/PreparedConfigurations.java index a581e159348a8..dc41c22311722 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/PreparedConfigurations.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/PreparedConfigurations.java @@ -31,19 +31,13 @@ class PreparedConfigurations { /** */ final List failedFuts; - /** */ - final Exception err; - /** * @param cfgs Configurations to deploy. * @param failedFuts Finished futures for failed configurations. - * @param err Error if need to stop deploy. */ - PreparedConfigurations(List cfgs, List failedFuts, - Exception err) { + PreparedConfigurations(List cfgs, List failedFuts) { this.cfgs = cfgs; this.failedFuts = failedFuts; - this.err = err; } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceDeploymentCompoundFutureSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceDeploymentCompoundFutureSelfTest.java index ca95198cf2fc9..bcc050400835b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceDeploymentCompoundFutureSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceDeploymentCompoundFutureSelfTest.java @@ -22,9 +22,7 @@ import java.util.List; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; -import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException; -import org.apache.ignite.internal.IgniteKernal; import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.services.ServiceConfiguration; @@ -33,16 +31,6 @@ /** */ public class GridServiceDeploymentCompoundFutureSelfTest extends GridCommonAbstractTest { - /** */ - private static GridKernalContext ctx; - - /** {@inheritDoc} */ - @Override protected void beforeTestsStarted() throws Exception { - IgniteKernal kernal = (IgniteKernal)startGrid(0); - - ctx = kernal.context(); - } - /** {@inheritDoc} */ @Override protected void afterTestsStopped() throws Exception { stopAllGrids(); @@ -51,8 +39,8 @@ public class GridServiceDeploymentCompoundFutureSelfTest extends GridCommonAbstr /** * @throws Exception If failed. */ - public void testWaitForCompletionOnFailingFuturePartial() throws Exception { - GridServiceDeploymentCompoundFuture compFut = new GridServiceDeploymentCompoundFuture(false, ctx); + public void testWaitForCompletionOnFailingFuture() throws Exception { + GridServiceDeploymentCompoundFuture compFut = new GridServiceDeploymentCompoundFuture(); int failingFutsNum = 2; @@ -80,7 +68,7 @@ public void testWaitForCompletionOnFailingFuturePartial() throws Exception { compFut.add(fut); } - compFut.serviceDeploymentMarkInitialized(); + compFut.markInitialized(); List causes = new ArrayList<>(); @@ -125,109 +113,6 @@ public void testWaitForCompletionOnFailingFuturePartial() throws Exception { } } - /** - * @throws Exception if failed. - */ - public void testFailAllAfterInitialized() throws Exception { - GridServiceDeploymentCompoundFuture compFut = new GridServiceDeploymentCompoundFuture(true, ctx); - - ServiceConfiguration failingCfg = config("Failed"); - - GridServiceDeploymentFuture failingFut = new GridServiceDeploymentFuture(failingCfg); - - compFut.add(failingFut); - - int futsNum = 5; - - List cfgs = new ArrayList<>(futsNum + 1); - - cfgs.add(failingCfg); - - for (int i = 0; i < futsNum; i++) { - ServiceConfiguration cfg = config(String.valueOf(i)); - - cfgs.add(cfg); - - compFut.add(new GridServiceDeploymentFuture(cfg)); - } - - compFut.serviceDeploymentMarkInitialized(); - - Exception expCause = new Exception("Test error"); - - failingFut.onDone(expCause); - - assertFailAll(compFut, cfgs, expCause); - } - - /** - * @throws Exception if failed. - */ - public void testFailAllBeforeInitialized() throws Exception { - GridServiceDeploymentCompoundFuture compFut = new GridServiceDeploymentCompoundFuture(true, ctx); - - ServiceConfiguration failingCfg = config("Failed"); - - GridServiceDeploymentFuture failingFut = new GridServiceDeploymentFuture(failingCfg); - - Exception expCause = new Exception("Test error"); - - failingFut.onDone(expCause); - - compFut.add(failingFut); - - assertFalse(compFut.isDone()); - - int futsNum = 5; - - List cfgs = new ArrayList<>(futsNum + 1); - - cfgs.add(failingCfg); - - for (int i = 0; i < futsNum; i++) { - ServiceConfiguration cfg = config(String.valueOf(i)); - - cfgs.add(cfg); - - compFut.add(new GridServiceDeploymentFuture(cfg)); - } - - compFut.serviceDeploymentMarkInitialized(); - - assertFailAll(compFut, cfgs, expCause); - } - - /** - * Try waiting for the future completion and check that a proper exception is thrown. - * - * @param fut Future. - * @param expCfgs Expected cfgs. - * @param expCause Expected cause. - */ - private void assertFailAll(GridServiceDeploymentCompoundFuture fut, Collection expCfgs, - Exception expCause) { - try { - fut.get(); - - fail("Should never reach here."); - } - catch (IgniteCheckedException ce) { - log.info("Expected exception: " + ce.getMessage()); - - IgniteException e = U.convertException(ce); - - assertTrue(e instanceof ServiceDeploymentException); - - assertEqualsCollections(expCfgs, ((ServiceDeploymentException)e).getFailedConfigurations()); - - Throwable actCause = e.getCause(); - - assertTrue(actCause instanceof IgniteCheckedException); - - assertEquals(expCause, actCause.getCause()); - } - } - /** * @param name Name. * @return Dummy configuration with a specified name. diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorBatchDeploySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorBatchDeploySelfTest.java index 337e69565968b..ab012b37ea551 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorBatchDeploySelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorBatchDeploySelfTest.java @@ -100,7 +100,7 @@ public void testDeployAll() throws Exception { subscribeExeLatch(cfgs, latch); - client.services().deployAll(cfgs, false); + client.services().deployAll(cfgs); assertTrue("Waiting for services deployment timed out.", latch.await(30, TimeUnit.SECONDS)); @@ -121,7 +121,7 @@ public void testDeployAllAsync() throws Exception { IgniteServices services = client.services().withAsync(); - services.deployAll(cfgs, false); + services.deployAll(cfgs); services.future().get(); @@ -131,7 +131,7 @@ public void testDeployAllAsync() throws Exception { } /** - * TODO: enable when IGNITE-6259 is fixed + * TODO: enable when IGNITE-6259 is fixed. * * @throws Exception If failed. */ @@ -170,7 +170,7 @@ public void _testDeployAllTopologyChange() throws Exception { while (from < numServices) { int to = Math.min(numServices, from + batchSize); - services.deployAll(cfgs.subList(from, to), false); + services.deployAll(cfgs.subList(from, to)); services.future().get(5000); @@ -189,7 +189,7 @@ public void _testDeployAllTopologyChange() throws Exception { } /** - * TODO: enable when IGNITE-6259 is fixed + * TODO: enable when IGNITE-6259 is fixed. * * @throws Exception If failed. */ @@ -239,7 +239,7 @@ public void _testDeployAllTopologyChangeFail() throws Exception { failingCfgs.add(failingCfg); try { - services.deployAll(cfgsBatch, false); + services.deployAll(cfgsBatch); services.future().get(5000); @@ -274,37 +274,22 @@ public void _testDeployAllTopologyChangeFail() throws Exception { /** * @throws Exception If failed. */ - public void testDeployAllFailAll() throws Exception { - deployAllFail(false, true); + public void testDeployAllFail() throws Exception { + deployAllFail(false); } /** * @throws Exception If failed. */ - public void testDeployAllPartial() throws Exception { - deployAllFail(false, false); - } - - /** - * @throws Exception If failed. - */ - public void testDeployAllAsyncFailAll() throws Exception { - deployAllFail(true, true); - } - - /** - * @throws Exception If failed. - */ - public void testDeployAllAsyncFailPartial() throws Exception { - deployAllFail(true, false); + public void testDeployAllAsyncFail() throws Exception { + deployAllFail(true); } /** * @param async If {@code true}, then asynchronous method of deployment will be performed. - * @param allOrNone Failing strategy. * @throws Exception If failed. */ - private void deployAllFail(boolean async, boolean allOrNone) throws Exception { + private void deployAllFail(boolean async) throws Exception { Ignite client = grid(CLIENT_NODE_NAME); CountDownLatch latch = new CountDownLatch(NUM_SERVICES - 1); @@ -317,20 +302,11 @@ private void deployAllFail(boolean async, boolean allOrNone) throws Exception { failingCfg.setName(null); - assertFailingDeploy(client, async, allOrNone, cfgs, failingCfg); + assertFailingDeploy(client, async, cfgs, failingCfg); - if (allOrNone) { - assertFalse("Some of the services were deployed.", latch.await(2, TimeUnit.SECONDS)); - - assertEquals(NUM_SERVICES - 1, latch.getCount()); - - assertTrue(client.services().serviceDescriptors().isEmpty()); - } - else { - assertTrue("Waiting for services deployment timed out.", latch.await(30, TimeUnit.SECONDS)); + assertTrue("Waiting for services deployment timed out.", latch.await(30, TimeUnit.SECONDS)); - assertDeployedServices(client, cfgs.subList(0, cfgs.size() - 1)); - } + assertDeployedServices(client, cfgs.subList(0, cfgs.size() - 1)); } /** @@ -351,8 +327,8 @@ public void testClashingNames() throws Exception { IgniteServices svcs1 = client.services().withAsync(); IgniteServices svcs2 = client.services().withAsync(); - svcs1.deployAll(fstBatch, false); - svcs2.deployAll(sndBatch, false); + svcs1.deployAll(fstBatch); + svcs2.deployAll(sndBatch); svcs1.future().get(); svcs2.future().get(); @@ -365,32 +341,12 @@ public void testClashingNames() throws Exception { /** * @throws Exception If failed. */ - public void testClashingNamesFailAll() throws Exception { - clashingNamesFail(true); - } - - /** - * @throws Exception If failed. - */ - public void testClashingNamesPartial() throws Exception { - clashingNamesFail(false); - } - - /** - * @param allOrNone Failing strategy. - * @throws Exception If failed. - */ - private void clashingNamesFail(boolean allOrNone) throws Exception { + public void testClashingNamesFail() throws Exception { Ignite client = grid(CLIENT_NODE_NAME); List cfgs = getConfigs(client.cluster().forServers().predicate(), NUM_SERVICES); - int numDepSvcs; - - if (allOrNone) - numDepSvcs = NUM_SERVICES / 2; - else - numDepSvcs = NUM_SERVICES - 1; + int numDepSvcs = NUM_SERVICES - 1; CountDownLatch latch = new CountDownLatch(numDepSvcs); @@ -401,13 +357,13 @@ private void clashingNamesFail(boolean allOrNone) throws Exception { IgniteServices services = client.services().withAsync(); - services.deployAll(fstBatch, false); + services.deployAll(fstBatch); ServiceConfiguration failingCfg = cfgs.get(NUM_SERVICES - 1); failingCfg.setName(null); - assertFailingDeploy(client, false, allOrNone, sndBatch, failingCfg); + assertFailingDeploy(client, false, sndBatch, failingCfg); services.future().get(); @@ -419,33 +375,12 @@ private void clashingNamesFail(boolean allOrNone) throws Exception { /** * @throws Exception If failed. */ - public void testClashingNameDifferentConfigFailAll() throws Exception { - testClashingNameDifferentConfig(true); - } - - /** - * @throws Exception If failed. - */ - public void testClashingNameDifferentConfigPartial() throws Exception { - testClashingNameDifferentConfig(false); - } - - /** - * @param allOrNone Failing strategy. - * @throws Exception If failed. - */ - private void testClashingNameDifferentConfig(boolean allOrNone) throws Exception { + public void testClashingNameDifferentConfig() throws Exception { Ignite client = grid(CLIENT_NODE_NAME); List cfgs = getConfigs(client.cluster().forServers().predicate(), NUM_SERVICES); - int numDepSvcs; - - if (allOrNone) - numDepSvcs = NUM_SERVICES / 2; - else - numDepSvcs = NUM_SERVICES - 1; - + int numDepSvcs = NUM_SERVICES - 1; CountDownLatch latch = new CountDownLatch(numDepSvcs); @@ -454,7 +389,7 @@ private void testClashingNameDifferentConfig(boolean allOrNone) throws Exception subscribeExeLatch(cfgs, latch); - client.services().deployAll(fstBatch, false); + client.services().deployAll(fstBatch); ServiceConfiguration failingCfg = copyService(cfgs.get(NUM_SERVICES - 1)); @@ -464,7 +399,7 @@ private void testClashingNameDifferentConfig(boolean allOrNone) throws Exception sndBatch.add(failingCfg); - assertFailingDeploy(client, false, allOrNone, sndBatch, failingCfg); + assertFailingDeploy(client, false, sndBatch, failingCfg); assertTrue("Waiting for services deployment timed out.", latch.await(30, TimeUnit.SECONDS)); @@ -483,7 +418,7 @@ public void testCancelAll() throws Exception { subscribeExeLatch(cfgs, latch); - client.services().deployAll(cfgs, true); + client.services().deployAll(cfgs); latch.await(30, TimeUnit.SECONDS); @@ -504,7 +439,7 @@ public void testCancelAllAsync() throws Exception { subscribeExeLatch(cfgs, latch); - client.services().deployAll(cfgs, true); + client.services().deployAll(cfgs); latch.await(30, TimeUnit.SECONDS); @@ -518,7 +453,7 @@ public void testCancelAllAsync() throws Exception { } /** - * TODO: enable when IGNITE-6259 is fixed + * TODO: enable when IGNITE-6259 is fixed. * * @throws Exception If failed. */ @@ -533,7 +468,7 @@ public void _testCancelAllTopologyChange() throws Exception { subscribeExeLatch(cfgs, latch); - client.services().deployAll(cfgs, true); + client.services().deployAll(cfgs); latch.await(30, TimeUnit.SECONDS); @@ -585,7 +520,7 @@ public void testCancelAllClashingNames() throws Exception { subscribeExeLatch(cfgs, latch); - client.services().deployAll(cfgs, true); + client.services().deployAll(cfgs); latch.await(30, TimeUnit.SECONDS); @@ -616,11 +551,10 @@ public void testCancelAllClashingNames() throws Exception { * @param client Client. * @param async If {@code true}, then async version of deploy method will be used. * @param cfgs Service configurations. - * @param allOrNone Failing policy. * @param failingCfg Configuration of the failing service. * @throws Exception If failed. */ - private void assertFailingDeploy(Ignite client, boolean async, boolean allOrNone, List cfgs, + private void assertFailingDeploy(Ignite client, boolean async, List cfgs, ServiceConfiguration failingCfg) throws Exception { IgniteServices services = client.services(); @@ -628,26 +562,21 @@ private void assertFailingDeploy(Ignite client, boolean async, boolean allOrNone if (async) { services = services.withAsync(); - services.deployAll(cfgs, allOrNone); + services.deployAll(cfgs); } try { if (async) services.future().get(); else - services.deployAll(cfgs, allOrNone); + services.deployAll(cfgs); fail("Should never reach here."); } catch (ServiceDeploymentException e) { info("Expected exception: " + e.getMessage()); - Collection expFails; - - if (allOrNone) - expFails = cfgs; - else - expFails = Collections.singleton(failingCfg); + Collection expFails = Collections.singleton(failingCfg); Collection actFails = e.getFailedConfigurations(); From 565ee846490834b74a9cc0f8cd0a28a5494392cd Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Fri, 8 Sep 2017 11:20:28 +0300 Subject: [PATCH 277/357] GG-12690 - Deregister local continuous query handlers on cache stop. --- .../CacheContinuousQueryHandler.java | 7 ++++++ .../CacheContinuousQueryManager.java | 2 +- .../continuous/GridContinuousProcessor.java | 24 +++++++++++++++++++ 3 files changed, 32 insertions(+), 1 deletion(-) 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 50d29d1f8de6e..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; @@ -568,6 +569,29 @@ 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())) { + it2.remove(); + + assert hnd instanceof CacheContinuousQueryHandler : hnd; + + CacheContinuousQueryHandler hnd0 = (CacheContinuousQueryHandler)hnd; + + unregisterHandler(entry.getKey(), hnd, true); + + CacheContinuousQueryManager qryMgr = ctx.continuousQueries(); + + qryMgr.unregisterListener(hnd0.internal(), entry.getKey()); + } + } } /** From 1c07cc9babf2e77736cd1caa44ddf944eeea3cf1 Mon Sep 17 00:00:00 2001 From: Slava Koptilin Date: Fri, 8 Sep 2017 14:37:03 +0300 Subject: [PATCH 278/357] ignite-12754 fix for IgniteCacheCreatePutTest.testUpdatesAndCacheStart. fix is based on the following commit: 05dd08b993e2d7f88176c051463b178431714f85 --- .../processors/cache/distributed/dht/GridDhtCacheAdapter.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 2fa934b79db71..417eb3536536f 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 @@ -219,7 +219,8 @@ protected GridDhtCacheAdapter(GridCacheContext ctx, GridCacheConcurrentMap @Override public void onKernalStart() throws IgniteCheckedException { super.onKernalStart(); - preldr.onKernalStart(); + if (preldr != null) + preldr.onKernalStart(); } /** {@inheritDoc} */ From 9e3e1149173787f7e5501f937907f3fc6ffd6ffe Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Mon, 5 Sep 2016 13:43:54 +0300 Subject: [PATCH 279/357] Minor fix for IGNITE-6256: Add localnode to topology snapshot. (cherry picked from commit bb40e57) --- .../ignite/spi/discovery/tcp/ServerImpl.java | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 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 cfa36f1599590..b1fd7e23f308f 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 @@ -458,34 +458,36 @@ else if (log.isInfoEnabled()) { U.interrupt(statsPrinter); U.join(statsPrinter, log); - Collection rmts = null; + Collection nodes = null; if (!disconnect) spi.printStopInfo(); else { spi.getSpiContext().deregisterPorts(); - rmts = ring.visibleRemoteNodes(); + nodes = ring.visibleNodes(); } long topVer = ring.topologyVersion(); ring.clear(); - if (rmts != null && !rmts.isEmpty()) { - // This is restart/disconnection and remote nodes are not empty. - // We need to fire FAIL event for each. + if (nodes != null) { + // This is restart/disconnection and we need to fire FAIL event for each remote node. DiscoverySpiListener lsnr = spi.lsnr; if (lsnr != null) { - Collection processed = new HashSet<>(); + Collection processed = new HashSet<>(nodes.size()); + + for (TcpDiscoveryNode n : nodes) { + if(n.isLocal()) + continue; - for (TcpDiscoveryNode n : rmts) { assert n.visible(); processed.add(n); - List top = U.arrayList(rmts, F.notIn(processed)); + List top = U.arrayList(nodes, F.notIn(processed)); topVer++; From aa264b1474dd964dc46287bd80d9e6e1ba3fb758 Mon Sep 17 00:00:00 2001 From: Denis Mekhanikov Date: Mon, 11 Sep 2017 15:58:15 +0300 Subject: [PATCH 280/357] GG-12708 fix IgniteCacheTxStoreSessionWriteBehindCoalescingTest.testSession --- .../IgniteCacheStoreSessionWriteBehindAbstractTest.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheStoreSessionWriteBehindAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheStoreSessionWriteBehindAbstractTest.java index 4b0f7c0bb22c0..f8665373c4cb4 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheStoreSessionWriteBehindAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheStoreSessionWriteBehindAbstractTest.java @@ -104,8 +104,6 @@ protected CacheConfiguration cacheConfiguration(String igniteInstanceName) throw ccfg1.setName(CACHE_NAME1); - ccfg1.setCacheStoreFactory(singletonFactory(new TestStore())); - cfg.setCacheConfiguration(ccfg0, ccfg1); return cfg; From fbd1f2d201a4c6f95ddc485c3d7ff4f8fd5adf8f Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Tue, 13 Jun 2017 19:41:55 +0300 Subject: [PATCH 281/357] Backport test fix: ignite-2.1.1 Extract Ignite updates checker to separate class. Fixed GridUpdateNotifier test. (cherry picked from commit 2b030c0) --- .../cluster/GridUpdateNotifier.java | 50 +++++++------- .../cluster/HttpIgniteUpdatesChecker.java | 65 +++++++++++++++++++ .../cluster/GridUpdateNotifierSelfTest.java | 10 ++- 3 files changed, 97 insertions(+), 28 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 5b2edcd94bab3..df956b6e62972 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,6 +103,9 @@ 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. * @@ -111,14 +114,16 @@ class GridUpdateNotifier { * @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 { + boolean reportOnlyNew, HttpIgniteUpdatesChecker updatesChecker) throws IgniteCheckedException { try { this.ver = ver; this.gridName = gridName == null ? "null" : gridName; 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 168655820615765817b05d8e8dbe9cbae5fd41dd Mon Sep 17 00:00:00 2001 From: Konstantin Dudkov Date: Fri, 7 Apr 2017 17:17:04 +0300 Subject: [PATCH 282/357] IGNITE-4876 - Tests should wait for topology change on all nodes (cherry picked from commit 4e80ddf) --- .../junits/GridAbstractTest.java | 53 ++++++++++++++++++- 1 file changed, 51 insertions(+), 2 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java index 5c43ecbfc9bb2..a9815483ee453 100755 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java @@ -57,16 +57,20 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.configuration.NearCacheConfiguration; import org.apache.ignite.events.EventType; +import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.IgniteKernal; import org.apache.ignite.internal.IgnitionEx; import org.apache.ignite.internal.binary.BinaryEnumCache; import org.apache.ignite.internal.binary.BinaryMarshaller; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.resource.GridSpringResourceContext; import org.apache.ignite.internal.util.GridClassLoaderCache; import org.apache.ignite.internal.util.GridTestClockTimer; import org.apache.ignite.internal.util.GridUnsafe; +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.internal.LT; @@ -135,6 +139,9 @@ public abstract class GridAbstractTest extends TestCase { /** */ private static final long DFLT_TEST_TIMEOUT = 5 * 60 * 1000; + /** */ + private static final int DFLT_TOP_WAIT_TIMEOUT = 2000; + /** */ private static final transient Map, TestCounters> tests = new ConcurrentHashMap<>(); @@ -862,6 +869,16 @@ protected void stopGrid(@Nullable String gridName) { */ @SuppressWarnings({"deprecation"}) protected void stopGrid(@Nullable String gridName, boolean cancel) { + stopGrid(gridName, cancel, true); + } + + /** + * @param gridName Ignite instance name. + * @param cancel Cancel flag. + * @param awaitTop Await topology change flag. + */ + @SuppressWarnings({"deprecation"}) + protected void stopGrid(@Nullable String gridName, boolean cancel, boolean awaitTop) { try { Ignite ignite = grid(gridName); @@ -874,6 +891,9 @@ protected void stopGrid(@Nullable String gridName, boolean cancel) { G.stop(gridName, cancel); else IgniteProcessProxy.stop(gridName, cancel); + + if (awaitTop) + awaitTopologyChange(); } catch (IllegalStateException ignored) { // Ignore error if grid already stopped. @@ -908,10 +928,10 @@ protected void stopAllGrids(boolean cancel) { } for (Ignite g : clients) - stopGrid(g.name(), cancel); + stopGrid(g.name(), cancel, false); for (Ignite g : srvs) - stopGrid(g.name(), cancel); + stopGrid(g.name(), cancel, false); assert G.allGrids().isEmpty(); } @@ -1894,6 +1914,35 @@ public static IgniteEx grid(String name, boolean remote, boolean thisRemote) { } } + /** + * + * @throws IgniteInterruptedCheckedException + */ + public void awaitTopologyChange() throws IgniteInterruptedCheckedException { + for (Ignite g : G.allGrids()) { + final GridKernalContext ctx = ((IgniteKernal)g).context(); + + if (ctx.isStopping()) + continue; + + AffinityTopologyVersion topVer = ctx.discovery().topologyVersionEx(); + AffinityTopologyVersion exchVer = ctx.cache().context().exchange().readyAffinityVersion(); + + if (! topVer.equals(exchVer)) { + info("topology version mismatch: node " + g.name() + " " + exchVer + ", " + topVer); + + GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + AffinityTopologyVersion topVer = ctx.discovery().topologyVersionEx(); + AffinityTopologyVersion exchVer = ctx.cache().context().exchange().readyAffinityVersion(); + + return exchVer.equals(topVer); + } + }, DFLT_TOP_WAIT_TIMEOUT); + } + } + } + /** * */ From 8ca38cdfb6edfb84ef4e0b2c83853776d8bf3b31 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Tue, 12 Sep 2017 16:53:46 +0300 Subject: [PATCH 283/357] GG-12695: Fixed test IgniteCacheNearRestartRollbackSelfTest.testRestarts(). --- .../apache/ignite/internal/IgniteKernal.java | 66 +++++++++++-------- ...gniteCacheNearRestartRollbackSelfTest.java | 2 - 2 files changed, 37 insertions(+), 31 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 a1997b1adfed7..d41075f012315 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 @@ -887,37 +887,45 @@ public void start(final IgniteConfiguration cfg, ctx.add(discoMgr, false); - // Start processors before discovery manager, so they will - // be able to start receiving messages once discovery completes. - startProcessor(createComponent(DiscoveryNodeValidationProcessor.class, ctx)); - startProcessor(new GridClockSyncProcessor(ctx)); - startProcessor(new GridAffinityProcessor(ctx)); - startProcessor(createComponent(GridSegmentationProcessor.class, ctx)); - startProcessor(createComponent(IgniteCacheObjectProcessor.class, ctx)); - startProcessor(new GridCacheProcessor(ctx)); - startProcessor(new GridQueryProcessor(ctx)); - startProcessor(new OdbcProcessor(ctx)); - startProcessor(new GridServiceProcessor(ctx)); - startProcessor(new GridTaskSessionProcessor(ctx)); - startProcessor(new GridJobProcessor(ctx)); - startProcessor(new GridTaskProcessor(ctx)); - startProcessor((GridProcessor)SCHEDULE.createOptional(ctx)); - startProcessor(new GridRestProcessor(ctx)); - startProcessor(new DataStreamProcessor(ctx)); - startProcessor((GridProcessor)IGFS.create(ctx, F.isEmpty(cfg.getFileSystemConfiguration()))); - startProcessor(new GridContinuousProcessor(ctx)); - startProcessor(createHadoopComponent()); - startProcessor(new DataStructuresProcessor(ctx)); - startProcessor(createComponent(PlatformProcessor.class, ctx)); - - // Start plugins. - for (PluginProvider provider : ctx.plugins().allProviders()) { - ctx.add(new GridPluginComponent(provider)); - - provider.start(ctx.plugins().pluginContextForProvider(provider)); + // Start processors before discovery manager, so they will + // be able to start receiving messages once discovery completes. + try { + startProcessor(createComponent(DiscoveryNodeValidationProcessor.class, ctx)); + startProcessor(new GridClockSyncProcessor(ctx)); + startProcessor(new GridAffinityProcessor(ctx)); + startProcessor(createComponent(GridSegmentationProcessor.class, ctx)); + startProcessor(createComponent(IgniteCacheObjectProcessor.class, ctx)); + startProcessor(new GridCacheProcessor(ctx)); + startProcessor(new GridQueryProcessor(ctx)); + startProcessor(new OdbcProcessor(ctx)); + startProcessor(new GridServiceProcessor(ctx)); + startProcessor(new GridTaskSessionProcessor(ctx)); + startProcessor(new GridJobProcessor(ctx)); + startProcessor(new GridTaskProcessor(ctx)); + startProcessor((GridProcessor)SCHEDULE.createOptional(ctx)); + startProcessor(new GridRestProcessor(ctx)); + startProcessor(new DataStreamProcessor(ctx)); + startProcessor((GridProcessor)IGFS.create(ctx, F.isEmpty(cfg.getFileSystemConfiguration()))); + startProcessor(new GridContinuousProcessor(ctx)); + startProcessor(createHadoopComponent()); + startProcessor(new DataStructuresProcessor(ctx)); + startProcessor(createComponent(PlatformProcessor.class, ctx)); + + // Start plugins. + for (PluginProvider provider : ctx.plugins().allProviders()) { + ctx.add(new GridPluginComponent(provider)); + + provider.start(ctx.plugins().pluginContextForProvider(provider)); + } + + fillNodeAttributes(clusterProc.updateNotifierEnabled()); } + catch (Throwable e) { + // Stop discovery spi to close tcp socket. + ctx.discovery().stop(true); - fillNodeAttributes(clusterProc.updateNotifierEnabled()); + throw e; + } gw.writeLock(); 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 aea4d7782d858..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 @@ -132,8 +132,6 @@ 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 b954d26ee388416baadeeae597efd2d6060038f7 Mon Sep 17 00:00:00 2001 From: Ilya Lantukh Date: Mon, 1 Aug 2016 10:10:51 +0300 Subject: [PATCH 284/357] IGNITE-1690: Re-enabled IgniteCacheCreateRestartSelfTest.testStopOriginatingNode. This closes #753. (cherry picked from commit 0763de8) --- .../processors/cache/IgniteCacheCreateRestartSelfTest.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheCreateRestartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheCreateRestartSelfTest.java index e8e66c4155391..681636a34860e 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheCreateRestartSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheCreateRestartSelfTest.java @@ -71,8 +71,6 @@ public class IgniteCacheCreateRestartSelfTest extends GridCommonAbstractTest { * @throws Exception If failed. */ public void testStopOriginatingNode() throws Exception { - fail("https://issues.apache.org/jira/browse/IGNITE-1690"); - startGrids(NODES); ThreadLocalRandom rnd = ThreadLocalRandom.current(); From 88a7c0239bffcc2a3a47e67d11df360542dd0be5 Mon Sep 17 00:00:00 2001 From: devozerov Date: Wed, 14 Jun 2017 14:55:48 +0300 Subject: [PATCH 285/357] Fixed missing Apache header in HttpIgniteUpdatesChecker. (cherry picked from commit 69aa299) --- .../cluster/HttpIgniteUpdatesChecker.java | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) 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 index c052c0997fe2a..2b93ceb1e44e4 100644 --- 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 @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.ignite.internal.processors.cluster; import java.io.BufferedReader; From 77723e433561fd908efda9abd067536494137ac4 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Wed, 13 Sep 2017 12:07:55 +0300 Subject: [PATCH 286/357] GG-12767: fixed flaky test IgniteCacheGetRestartTest. Try avoid timeout in IgniteCacheGetRestartTest. (cherry picked from commit b41ecd1) --- .../cache/distributed/IgniteCacheGetRestartTest.java | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheGetRestartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheGetRestartTest.java index 3b0c2fac32947..eef7750b05bf0 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheGetRestartTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheGetRestartTest.java @@ -26,6 +26,7 @@ import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteDataStreamer; +import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.cache.CacheMode; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; @@ -90,6 +91,8 @@ public class IgniteCacheGetRestartTest extends GridCommonAbstractTest { /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { + System.setProperty(IgniteSystemProperties.IGNITE_ENABLE_FORCIBLE_NODE_KILL, "true"); + super.beforeTestsStarted(); startGrids(SRVS); @@ -108,11 +111,13 @@ public class IgniteCacheGetRestartTest extends GridCommonAbstractTest { super.afterTestsStopped(); stopAllGrids(); + + System.clearProperty(IgniteSystemProperties.IGNITE_ENABLE_FORCIBLE_NODE_KILL); } /** {@inheritDoc} */ @Override protected long getTestTimeout() { - return TEST_TIME + 60_000; + return TEST_TIME + 3 * 60_000; } /** @@ -198,6 +203,8 @@ private void checkRestart(final CacheConfiguration ccfg, final int restartCnt) t @Override public Void call() throws Exception { int nodeIdx = restartNodeIdx.getAndIncrement(); + Thread.currentThread().setName("restart-thread-" + nodeIdx); + boolean clientMode = clientNode.compareAndSet(false, true); while (U.currentTimeMillis() < stopTime) { @@ -219,7 +226,7 @@ private void checkRestart(final CacheConfiguration ccfg, final int restartCnt) t IgniteInternalFuture syncFut = ((IgniteCacheProxy)cache).context().preloader().syncFuture(); - while (!syncFut.isDone()) + while (!syncFut.isDone() && U.currentTimeMillis() < stopTime) checkGet(cache); checkGet(cache); From 395e78cf92621df23c8173323031f655cfe53518 Mon Sep 17 00:00:00 2001 From: Alexey Goncharuk Date: Thu, 13 Apr 2017 16:36:54 +0300 Subject: [PATCH 287/357] IGNITE-3477 - Fixing flaky full API suite (cherry picked from commit 8d2b020) --- .../cache/H2CacheStoreStrategy.java | 35 +++++++++++++++---- 1 file changed, 28 insertions(+), 7 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/H2CacheStoreStrategy.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/H2CacheStoreStrategy.java index ccb299438ce86..6261366e10569 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/H2CacheStoreStrategy.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/H2CacheStoreStrategy.java @@ -54,6 +54,9 @@ public class H2CacheStoreStrategy implements TestCacheStoreStrategy { /** Pool to get {@link Connection}s from. */ private final JdbcConnectionPool dataSrc; + /** */ + private final int port; + /** Script that creates CACHE table. */ private static final String CREATE_CACHE_TABLE = "create table if not exists CACHE(k binary not null, v binary not null, PRIMARY KEY(k));"; @@ -75,9 +78,14 @@ public class H2CacheStoreStrategy implements TestCacheStoreStrategy { * @throws IgniteCheckedException If failed. */ public H2CacheStoreStrategy() throws IgniteCheckedException { + Server srv = null; + try { - Server.createTcpServer().start(); - dataSrc = H2CacheStoreSessionListenerFactory.createDataSource(); + srv = Server.createTcpServer().start(); + + port = srv.getPort(); + + dataSrc = H2CacheStoreSessionListenerFactory.createDataSource(port); try (Connection conn = connection()) { RunScript.execute(conn, new StringReader(CREATE_CACHE_TABLE)); @@ -86,7 +94,8 @@ public H2CacheStoreStrategy() throws IgniteCheckedException { } } catch (SQLException e) { - throw new IgniteCheckedException(e); + throw new IgniteCheckedException("Failed to set up cache store strategy" + + (srv == null ? "" : ": " + srv.getStatus()), e); } } @@ -242,7 +251,7 @@ private int querySingleInt(String qry, String errorMsg) { /** {@inheritDoc} */ @Override public void updateCacheConfiguration(CacheConfiguration cfg) { - cfg.setCacheStoreSessionListenerFactories(new H2CacheStoreSessionListenerFactory()); + cfg.setCacheStoreSessionListenerFactories(new H2CacheStoreSessionListenerFactory(port)); } /** {@inheritDoc} */ @@ -260,11 +269,23 @@ public static class H2StoreFactory implements Factory /** Serializable {@link Factory} producing H2 backed {@link CacheStoreSessionListener}s. */ public static class H2CacheStoreSessionListenerFactory implements Factory { + /** */ + private int port; + + /** + * @param port Port. + */ + public H2CacheStoreSessionListenerFactory(int port) { + this.port = port; + } + /** * @return Connection pool */ - static JdbcConnectionPool createDataSource() { - JdbcConnectionPool pool = JdbcConnectionPool.create("jdbc:h2:tcp://localhost/mem:TestDb;LOCK_MODE=0", "sa", ""); + static JdbcConnectionPool createDataSource(int port) { + JdbcConnectionPool pool = JdbcConnectionPool.create("jdbc:h2:tcp://localhost:" + port + + "/mem:TestDb;LOCK_MODE=0", "sa", ""); + pool.setMaxConnections(100); return pool; } @@ -272,7 +293,7 @@ static JdbcConnectionPool createDataSource() { /** {@inheritDoc} */ @Override public CacheStoreSessionListener create() { CacheJdbcStoreSessionListener lsnr = new CacheJdbcStoreSessionListener(); - lsnr.setDataSource(createDataSource()); + lsnr.setDataSource(createDataSource(port)); return lsnr; } } From 9ad68c735f76c6f5ce16c787e5cc07d2638d116d Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Thu, 13 Apr 2017 16:36:54 +0300 Subject: [PATCH 288/357] GG-12773: Fixed potential deadlock when service is being deployed while node is stopping. --- .../service/GridServiceProcessor.java | 171 ++++++++++-------- .../GridServiceProcessorStopSelfTest.java | 137 +++++++++++++- 2 files changed, 228 insertions(+), 80 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 c452da31086d0..106aa7f678010 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 @@ -493,124 +493,137 @@ public IgniteInternalFuture deployKeyAffinitySingleton(String name, Service s public IgniteInternalFuture deploy(ServiceConfiguration cfg) { A.notNull(cfg, "cfg"); - ServicesCompatibilityState state = markCompatibilityStateAsUsed(); + if (!busyLock.enterBusy()) { + IgniteCheckedException e = new IgniteCheckedException("Service deployment has been cancelled (node is stopping): " + + cfg.getName()); - validate(cfg); - - ctx.security().authorize(cfg.getName(), SecurityPermission.SERVICE_DEPLOY, null); + return new GridFinishedFuture<>(e); + } - if (!state.srvcCompatibility) { - Marshaller marsh = ctx.config().getMarshaller(); + try { - LazyServiceConfiguration cfg0; + ServicesCompatibilityState state = markCompatibilityStateAsUsed(); - try { - byte[] srvcBytes = U.marshal(marsh, cfg.getService()); + validate(cfg); - cfg0 = new LazyServiceConfiguration(cfg, srvcBytes); - } - catch (IgniteCheckedException e) { - U.error(log, "Failed to marshal service with configured marshaller [srvc=" + cfg.getService() - + ", marsh=" + marsh + "]", e); + ctx.security().authorize(cfg.getName(), SecurityPermission.SERVICE_DEPLOY, null); - return new GridFinishedFuture<>(e); - } + if (!state.srvcCompatibility) { + Marshaller marsh = ctx.config().getMarshaller(); - cfg = cfg0; - } + LazyServiceConfiguration cfg0; - GridServiceDeploymentFuture fut = new GridServiceDeploymentFuture(cfg); + try { + byte[] srvcBytes = U.marshal(marsh, cfg.getService()); - GridServiceDeploymentFuture old = depFuts.putIfAbsent(cfg.getName(), fut); + cfg0 = new LazyServiceConfiguration(cfg, srvcBytes); + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to marshal service with configured marshaller [srvc=" + cfg.getService() + + ", marsh=" + marsh + "]", e); - if (old != null) { - if (!old.configuration().equalsIgnoreNodeFilter(cfg)) { - fut.onDone(new IgniteCheckedException("Failed to deploy service (service already exists with " + - "different configuration) [deployed=" + old.configuration() + ", new=" + cfg + ']')); + return new GridFinishedFuture<>(e); + } - return fut; + cfg = cfg0; } - return old; - } + GridServiceDeploymentFuture fut = new GridServiceDeploymentFuture(cfg); - if (ctx.clientDisconnected()) { - fut.onDone(new IgniteClientDisconnectedCheckedException(ctx.cluster().clientReconnectFuture(), - "Failed to deploy service, client node disconnected.")); + GridServiceDeploymentFuture old = depFuts.putIfAbsent(cfg.getName(), fut); - depFuts.remove(cfg.getName(), fut); - } + if (old != null) { + if (!old.configuration().equalsIgnoreNodeFilter(cfg)) { + fut.onDone(new IgniteCheckedException("Failed to deploy service (service already exists with " + + "different configuration) [deployed=" + old.configuration() + ", new=" + cfg + ']')); - while (true) { - try { - GridServiceDeploymentKey key = new GridServiceDeploymentKey(cfg.getName()); + return fut; + } + + return old; + } - if (ctx.deploy().enabled()) - ctx.cache().context().deploy().ignoreOwnership(true); + if (ctx.clientDisconnected()) { + fut.onDone(new IgniteClientDisconnectedCheckedException(ctx.cluster().clientReconnectFuture(), + "Failed to deploy service, client node disconnected.")); + depFuts.remove(cfg.getName(), fut); + } + + while (true) { try { - GridServiceDeployment dep = (GridServiceDeployment)cache.getAndPutIfAbsent(key, - new GridServiceDeployment(ctx.localNodeId(), cfg)); + GridServiceDeploymentKey key = new GridServiceDeploymentKey(cfg.getName()); - if (dep != null) { - if (!dep.configuration().equalsIgnoreNodeFilter(cfg)) { - // Remove future from local map. - depFuts.remove(cfg.getName(), fut); + if (ctx.deploy().enabled()) + ctx.cache().context().deploy().ignoreOwnership(true); - fut.onDone(new IgniteCheckedException("Failed to deploy service (service already exists with " + - "different configuration) [deployed=" + dep.configuration() + ", new=" + cfg + ']')); - } - else { - Iterator> it = serviceEntries( - ServiceAssignmentsPredicate.INSTANCE); + try { + GridServiceDeployment dep = (GridServiceDeployment)cache.getAndPutIfAbsent(key, + new GridServiceDeployment(ctx.localNodeId(), cfg)); + + if (dep != null) { + if (!dep.configuration().equalsIgnoreNodeFilter(cfg)) { + // Remove future from local map. + depFuts.remove(cfg.getName(), fut); + + fut.onDone(new IgniteCheckedException("Failed to deploy service (service already exists with " + + "different configuration) [deployed=" + dep.configuration() + ", new=" + cfg + ']')); + } + else { + Iterator> it = serviceEntries( + ServiceAssignmentsPredicate.INSTANCE); - while (it.hasNext()) { - Cache.Entry e = it.next(); + while (it.hasNext()) { + Cache.Entry e = it.next(); - if (e.getKey() instanceof GridServiceAssignmentsKey) { - GridServiceAssignments assigns = (GridServiceAssignments)e.getValue(); + if (e.getKey() instanceof GridServiceAssignmentsKey) { + GridServiceAssignments assigns = (GridServiceAssignments)e.getValue(); - if (assigns.name().equals(cfg.getName())) { - // Remove future from local map. - depFuts.remove(cfg.getName(), fut); + if (assigns.name().equals(cfg.getName())) { + // Remove future from local map. + depFuts.remove(cfg.getName(), fut); - fut.onDone(); + fut.onDone(); - break; + break; + } } } - } - if (!dep.configuration().equalsIgnoreNodeFilter(cfg)) - U.warn(log, "Service already deployed with different configuration (will ignore) " + - "[deployed=" + dep.configuration() + ", new=" + cfg + ']'); + if (!dep.configuration().equalsIgnoreNodeFilter(cfg)) + U.warn(log, "Service already deployed with different configuration (will ignore) " + + "[deployed=" + dep.configuration() + ", new=" + cfg + ']'); + } } } - } - finally { - if (ctx.deploy().enabled()) - ctx.cache().context().deploy().ignoreOwnership(false); - } + finally { + if (ctx.deploy().enabled()) + ctx.cache().context().deploy().ignoreOwnership(false); + } - return fut; - } - catch (ClusterTopologyCheckedException e) { - if (log.isDebugEnabled()) - log.debug("Topology changed while deploying service (will retry): " + e.getMessage()); - } - catch (IgniteCheckedException e) { - if (e.hasCause(ClusterTopologyCheckedException.class)) { + return fut; + } + catch (ClusterTopologyCheckedException e) { if (log.isDebugEnabled()) log.debug("Topology changed while deploying service (will retry): " + e.getMessage()); - - continue; } + catch (IgniteCheckedException e) { + if (e.hasCause(ClusterTopologyCheckedException.class)) { + if (log.isDebugEnabled()) + log.debug("Topology changed while deploying service (will retry): " + e.getMessage()); - U.error(log, "Failed to deploy service: " + cfg.getName(), e); + continue; + } - return new GridFinishedFuture<>(e); + U.error(log, "Failed to deploy service: " + cfg.getName(), e); + + return new GridFinishedFuture<>(e); + } } } + finally { + busyLock.leaveBusy(); + } } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorStopSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorStopSelfTest.java index ea0ba51b60954..563c3219f28da 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorStopSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorStopSelfTest.java @@ -21,6 +21,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.Lock; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; @@ -31,8 +32,12 @@ import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.util.lang.GridAbsPredicate; +import org.apache.ignite.internal.util.lang.GridPlainCallable; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.services.Service; +import org.apache.ignite.services.ServiceConfiguration; import org.apache.ignite.services.ServiceContext; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; @@ -161,8 +166,138 @@ public void testStopDuringHangedDeployment() throws Exception { assertTrue("Deploy future isn't completed", wait); fut.get(); + } - Ignition.stopAll(true); + /** + * @throws Exception If fails + */ + public void testServiceDeploymentCancelationOnStop() throws Exception { + final Ignite node = startGrid(0); + + final IgniteServices services = node.services(); + // Deploy some service. + services.deploy(getServiceConfiguration("myService1")); + + //Stop node async, this will cancel the service #1. + final IgniteInternalFuture stopAsync = GridTestUtils.runAsync(new GridPlainCallable() { + @Override public Boolean call() throws Exception { + node.close(); + + return true; + } + }, "node-stopping-thread"); + + // Wait for the service #1 cancellation during node stopping. + // At this point node.stop process will be paused until svcCancelFinishLatch released. + ServiceImpl.svcCancelStartLatch.await(); + + final AtomicReference queuedFuture = new AtomicReference<>(); + + // Try to deploy another service. + final IgniteInternalFuture deployAsync = GridTestUtils.runAsync(new GridPlainCallable() { + @Override public Boolean call() throws Exception { + IgniteServices async = services.withAsync(); + + async.deploy(getServiceConfiguration("myService2")); + + IgniteFuture future = async.future(); + + queuedFuture.set(future); + + // Here, deployment future is added to queue and + // then it will be cancelled when svcCancelFinishLatch be released. + // So, we'll wait for queue cleaning and try to deploy one more service. + try { + future.get(); + } + catch (Exception ignore) { + // Noop. + } + + // Normally, this should fail with some Exception as node is stopping right now. + // But we face a deadlock here. + for (int i = 0; i < 5; i++) { + try { + services.deploy(getServiceConfiguration("service3")); + } + catch (Exception ignore) { + // Noop. + } + } + + return true; + } + }, "svc-deploy-thread"); + + GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + return queuedFuture.get() != null; + } + }, 3000); + + // Allow node to be stopped. + ServiceImpl.svcCancelFinishLatch.countDown(); + + // Wait for all service deployments have finished. + boolean deployDone = GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + IgniteFuture fut = queuedFuture.get(); + + return fut != null && fut.isDone() && deployAsync.isDone(); + + } + }, 5000); + + assertTrue("Node stopping and service deployment processes falls into a deadlock.", deployDone); + + if (!deployDone) + deployAsync.cancel(); + + if (!stopAsync.isDone()) + stopAsync.cancel(); + } + + /** */ + private ServiceConfiguration getServiceConfiguration(String svcName) { + ServiceConfiguration svcCfg = new ServiceConfiguration(); + svcCfg.setName(svcName); + svcCfg.setService(new ServiceImpl()); + svcCfg.setTotalCount(1); + + return svcCfg; + } + + /** Dummy Implementation. */ + static class ServiceImpl implements Service { + /** */ + static final CountDownLatch svcCancelStartLatch = new CountDownLatch(1); + + /** */ + static final CountDownLatch svcCancelFinishLatch = new CountDownLatch(1); + + /** {@inheritDoc} */ + @Override public void cancel(ServiceContext ctx) { + System.out.println("cancel service: " + ctx.executionId()); + try { + svcCancelStartLatch.countDown(); + + svcCancelFinishLatch.await(); + } + catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + + /** {@inheritDoc} */ + @Override public void init(ServiceContext ctx) throws Exception { + System.out.println("init service: " + ctx.executionId()); + // No-op + } + + /** {@inheritDoc} */ + @Override public void execute(ServiceContext ctx) throws Exception { + // No-op + } } /** From 64c337c58501f2786298cc5b92a76b8d85f6f030 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Thu, 14 Sep 2017 20:05:43 +0300 Subject: [PATCH 289/357] Fixed flaky test IgniteCachePeekModesAbstractTest.testNonLocalPartitionSize. Fixed wrong swap entries counting. --- .../IgniteCachePeekModesAbstractTest.java | 37 ++++++++++++++----- 1 file changed, 27 insertions(+), 10 deletions(-) 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 fac24ccd00045..75485cdeb0d5b 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 @@ -996,23 +996,40 @@ private T2 swapKeysCount(int nodeIdx) { * @param part Cache partition * @return Tuple with number of primary and backup keys (one or both will be zero). */ - private T2 swapKeysCount(int nodeIdx, int part) throws IgniteCheckedException { + private T2 swapKeysCount(final int nodeIdx, final int part) throws IgniteCheckedException { GridCacheContext ctx = ((IgniteEx)ignite(nodeIdx)).context().cache().internalCache().context(); // Swap and offheap are disabled for near cache. GridCacheSwapManager swapMgr = ctx.isNear() ? ctx.near().dht().context().swap() : ctx.swap(); - //First count entries... - int cnt = (int)swapMgr.swapEntriesCount(part); - GridCacheAffinityManager affinity = ctx.affinity(); - AffinityTopologyVersion topVer = affinity.affinityTopologyVersion(); + AffinityTopologyVersion topVer = ctx.affinity().affinityTopologyVersion(); + + Affinity aff = ignite(nodeIdx).affinity(null); + + ClusterNode node = ignite(nodeIdx).cluster().localNode(); + + Iterator it = swapMgr.swapKeyIterator(true, true, topVer); + + CacheObjectContext coctx = ((IgniteEx)ignite(nodeIdx)).context().cache().internalCache() + .context().cacheObjectContext(); - //And then find out whether they are primary or backup ones. int primaryCnt = 0; int backupCnt = 0; - if (affinity.primaryByPartition(ctx.localNode(), part, topVer)) - primaryCnt = cnt; - else if (affinity.primaryByPartition(ctx.localNode(), part, topVer)) - backupCnt = cnt; + + while (it.hasNext()) { + Integer key = it.next().value(coctx, false); + + if (part != aff.partition(key)) + continue; + + if (aff.isPrimary(node, key)) + primaryCnt++; + else { + assertTrue(aff.isBackup(node, key)); + + backupCnt++; + } + } + return new T2<>(primaryCnt, backupCnt); } From 46296cf8150987c40346474f352c87d1e0fb1be0 Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Fri, 15 Sep 2017 11:17:53 +0700 Subject: [PATCH 290/357] IGNITE-6367 Restrict files search by IGNITE_HOME. --- .../visor/file/VisorLatestTextFilesTask.java | 15 +++--- .../visor/log/VisorLogSearchTask.java | 28 +++++------ .../internal/visor/util/VisorTaskUtils.java | 47 ++++++++++++++++--- 3 files changed, 61 insertions(+), 29 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/file/VisorLatestTextFilesTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/file/VisorLatestTextFilesTask.java index 55d358c78dd6e..63b526e1a1b77 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/file/VisorLatestTextFilesTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/file/VisorLatestTextFilesTask.java @@ -18,13 +18,11 @@ package org.apache.ignite.internal.visor.file; import java.io.File; -import java.net.URL; import java.util.ArrayList; import java.util.Collection; import java.util.List; import org.apache.ignite.internal.processors.task.GridInternal; import org.apache.ignite.internal.util.typedef.internal.S; -import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.internal.visor.VisorJob; import org.apache.ignite.internal.visor.VisorOneNodeTask; import org.apache.ignite.internal.visor.log.VisorLogFile; @@ -33,6 +31,7 @@ import static org.apache.ignite.internal.visor.util.VisorTaskUtils.LOG_FILES_COUNT_LIMIT; import static org.apache.ignite.internal.visor.util.VisorTaskUtils.matchedFiles; +import static org.apache.ignite.internal.visor.util.VisorTaskUtils.resolveIgnitePath; /** * Get list files matching filter. @@ -70,13 +69,11 @@ private VisorLatestTextFilesJob(IgniteBiTuple arg, boolean debug assert path != null; assert regexp != null; - URL url = U.resolveIgniteUrl(path); - - if (url == null) - return null; - try { - File folder = new File(url.toURI()); + File folder = resolveIgnitePath(path); + + if (folder == null) + return null; List files = matchedFiles(folder, regexp); @@ -98,4 +95,4 @@ private VisorLatestTextFilesJob(IgniteBiTuple arg, boolean debug return S.toString(VisorLatestTextFilesJob.class, this); } } -} \ No newline at end of file +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/log/VisorLogSearchTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/log/VisorLogSearchTask.java index b6552b2fca5f6..44fb0a90a5b0a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/log/VisorLogSearchTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/log/VisorLogSearchTask.java @@ -18,10 +18,8 @@ package org.apache.ignite.internal.visor.log; import java.io.File; -import java.io.FileNotFoundException; import java.io.IOException; import java.io.Serializable; -import java.net.URL; import java.nio.charset.Charset; import java.util.ArrayList; import java.util.Collection; @@ -35,7 +33,6 @@ import org.apache.ignite.internal.util.io.GridReversedLinesFileReader; import org.apache.ignite.internal.util.lang.GridTuple3; import org.apache.ignite.internal.util.typedef.internal.S; -import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.internal.visor.VisorJob; import org.apache.ignite.internal.visor.VisorMultiNodeTask; import org.apache.ignite.lang.IgniteBiTuple; @@ -43,6 +40,8 @@ import static org.apache.ignite.internal.visor.util.VisorTaskUtils.decode; import static org.apache.ignite.internal.visor.util.VisorTaskUtils.matchedFiles; +import static org.apache.ignite.internal.visor.util.VisorTaskUtils.resolveIgnitePath; +import static org.apache.ignite.internal.visor.util.VisorTaskUtils.resolveSymbolicLink; import static org.apache.ignite.internal.visor.util.VisorTaskUtils.textFile; /** @@ -216,21 +215,22 @@ private List> searchInFile(File f, Charse /** {@inheritDoc} */ @Override protected Collection run(VisorLogSearchArg arg) { - URL url = U.resolveIgniteUrl(arg.folder); + try { + File folder = resolveIgnitePath(arg.folder); - if (url == null) - throw new IgniteException(new FileNotFoundException("Log folder not found: " + arg.folder)); + if (folder == null) + return null; - UUID uuid = ignite.localNode().id(); - String nid = uuid.toString().toLowerCase(); + folder = resolveSymbolicLink(folder); - String filePtrn = arg.filePtrn.replace("@nid8", nid.substring(0, 8)).replace("@nid", nid); + UUID uuid = ignite.localNode().id(); + String nid = uuid.toString().toLowerCase(); - try { - File fld = new File(url.toURI()); - int pathIdx = (fld.isDirectory() ? fld : fld.getParentFile()).getAbsolutePath().length() + 1; + String filePtrn = arg.filePtrn.replace("@nid8", nid.substring(0, 8)).replace("@nid", nid); + + int pathIdx = (folder.isDirectory() ? folder : folder.getParentFile()).getAbsolutePath().length() + 1; - List matchingFiles = matchedFiles(fld, filePtrn); + List matchingFiles = matchedFiles(folder, filePtrn); Collection results = new ArrayList<>(arg.limit); @@ -276,4 +276,4 @@ private List> searchInFile(File f, Charse return S.toString(VisorLogSearchJob.class, this); } } -} \ No newline at end of file +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java index 3f5003a03d401..7518d93e1aa8e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java @@ -30,7 +30,10 @@ import java.nio.charset.CharacterCodingException; import java.nio.charset.Charset; import java.nio.charset.CharsetDecoder; +import java.nio.file.Files; +import java.nio.file.LinkOption; import java.nio.file.Path; +import java.nio.file.Paths; import java.text.SimpleDateFormat; import java.util.ArrayList; import java.util.Arrays; @@ -494,6 +497,34 @@ public static Collection collectEvents(Ignite ignite, String evt return res; } + /** + * @param path Path to resolve only relative to IGNITE_HOME. + * @return Resolved path as file, or {@code null} if path cannot be resolved. + * @throws IOException If failed to resolve path. + */ + public static File resolveIgnitePath(String path) throws IOException { + File folder = U.resolveIgnitePath(path); + + if (folder == null) + return null; + + if (!folder.toPath().toRealPath(LinkOption.NOFOLLOW_LINKS).startsWith(Paths.get(U.getIgniteHome()))) + return null; + + return folder; + } + + /** + * @param file File to resolve. + * @return Resolved file if it is a symbolic link or original file. + * @throws IOException If failed to resolve symlink. + */ + public static File resolveSymbolicLink(File file) throws IOException { + Path path = file.toPath(); + + return Files.isSymbolicLink(path) ? Files.readSymbolicLink(path).toFile() : file; + } + /** * Finds all files in folder and in it's sub-tree of specified depth. * @@ -501,8 +532,11 @@ public static Collection collectEvents(Ignite ignite, String evt * @param maxDepth Depth of the tree. If 1 - just look in the folder, no sub-folders. * @param filter file filter. * @return List of found files. + * @throws IOException If failed to list files. */ - public static List fileTree(File file, int maxDepth, @Nullable FileFilter filter) { + public static List fileTree(File file, int maxDepth, @Nullable FileFilter filter) throws IOException { + file = resolveSymbolicLink(file); + if (file.isDirectory()) { File[] files = (filter == null) ? file.listFiles() : file.listFiles(filter); @@ -525,12 +559,13 @@ else if (maxDepth > 1) } /** - * @param fld Folder with files to match. + * @param file Folder with files to match. * @param ptrn Pattern to match against file name. * @return Collection of matched files. + * @throws IOException If failed to filter files. */ - public static List matchedFiles(File fld, final String ptrn) { - List files = fileTree(fld, MAX_FOLDER_DEPTH, + public static List matchedFiles(File file, final String ptrn) throws IOException { + List files = fileTree(file, MAX_FOLDER_DEPTH, new FileFilter() { @Override public boolean accept(File f) { return !f.isHidden() && (f.isDirectory() || f.isFile() && f.getName().matches(ptrn)); @@ -873,8 +908,6 @@ public static List startLocalNode(@Nullable IgniteLogger log, String cf if (cmdFilePath == null || !cmdFilePath.exists()) throw new FileNotFoundException(String.format("File not found: %s", cmdFile)); - String ignite = cmdFilePath.getCanonicalPath(); - File nodesCfgPath = U.resolveIgnitePath(cfgPath); if (nodesCfgPath == null || !nodesCfgPath.exists()) @@ -887,6 +920,8 @@ public static List startLocalNode(@Nullable IgniteLogger log, String cf List run = new ArrayList<>(); try { + String ignite = cmdFilePath.getCanonicalPath(); + for (int i = 0; i < nodesToStart; i++) { if (U.isMacOs()) { Map macEnv = new HashMap<>(System.getenv()); From 86cb71985b6f6407264b3dbd40d2feed4d446a79 Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Fri, 15 Sep 2017 11:17:53 +0700 Subject: [PATCH 291/357] IGNITE-6367 Restrict files search by IGNITE_HOME. (cherry picked from commit 46296cf) --- .../visor/file/VisorLatestTextFilesTask.java | 15 +++--- .../visor/log/VisorLogSearchTask.java | 28 +++++------ .../internal/visor/util/VisorTaskUtils.java | 47 ++++++++++++++++--- 3 files changed, 61 insertions(+), 29 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/file/VisorLatestTextFilesTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/file/VisorLatestTextFilesTask.java index 55d358c78dd6e..63b526e1a1b77 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/file/VisorLatestTextFilesTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/file/VisorLatestTextFilesTask.java @@ -18,13 +18,11 @@ package org.apache.ignite.internal.visor.file; import java.io.File; -import java.net.URL; import java.util.ArrayList; import java.util.Collection; import java.util.List; import org.apache.ignite.internal.processors.task.GridInternal; import org.apache.ignite.internal.util.typedef.internal.S; -import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.internal.visor.VisorJob; import org.apache.ignite.internal.visor.VisorOneNodeTask; import org.apache.ignite.internal.visor.log.VisorLogFile; @@ -33,6 +31,7 @@ import static org.apache.ignite.internal.visor.util.VisorTaskUtils.LOG_FILES_COUNT_LIMIT; import static org.apache.ignite.internal.visor.util.VisorTaskUtils.matchedFiles; +import static org.apache.ignite.internal.visor.util.VisorTaskUtils.resolveIgnitePath; /** * Get list files matching filter. @@ -70,13 +69,11 @@ private VisorLatestTextFilesJob(IgniteBiTuple arg, boolean debug assert path != null; assert regexp != null; - URL url = U.resolveIgniteUrl(path); - - if (url == null) - return null; - try { - File folder = new File(url.toURI()); + File folder = resolveIgnitePath(path); + + if (folder == null) + return null; List files = matchedFiles(folder, regexp); @@ -98,4 +95,4 @@ private VisorLatestTextFilesJob(IgniteBiTuple arg, boolean debug return S.toString(VisorLatestTextFilesJob.class, this); } } -} \ No newline at end of file +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/log/VisorLogSearchTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/log/VisorLogSearchTask.java index b6552b2fca5f6..44fb0a90a5b0a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/log/VisorLogSearchTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/log/VisorLogSearchTask.java @@ -18,10 +18,8 @@ package org.apache.ignite.internal.visor.log; import java.io.File; -import java.io.FileNotFoundException; import java.io.IOException; import java.io.Serializable; -import java.net.URL; import java.nio.charset.Charset; import java.util.ArrayList; import java.util.Collection; @@ -35,7 +33,6 @@ import org.apache.ignite.internal.util.io.GridReversedLinesFileReader; import org.apache.ignite.internal.util.lang.GridTuple3; import org.apache.ignite.internal.util.typedef.internal.S; -import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.internal.visor.VisorJob; import org.apache.ignite.internal.visor.VisorMultiNodeTask; import org.apache.ignite.lang.IgniteBiTuple; @@ -43,6 +40,8 @@ import static org.apache.ignite.internal.visor.util.VisorTaskUtils.decode; import static org.apache.ignite.internal.visor.util.VisorTaskUtils.matchedFiles; +import static org.apache.ignite.internal.visor.util.VisorTaskUtils.resolveIgnitePath; +import static org.apache.ignite.internal.visor.util.VisorTaskUtils.resolveSymbolicLink; import static org.apache.ignite.internal.visor.util.VisorTaskUtils.textFile; /** @@ -216,21 +215,22 @@ private List> searchInFile(File f, Charse /** {@inheritDoc} */ @Override protected Collection run(VisorLogSearchArg arg) { - URL url = U.resolveIgniteUrl(arg.folder); + try { + File folder = resolveIgnitePath(arg.folder); - if (url == null) - throw new IgniteException(new FileNotFoundException("Log folder not found: " + arg.folder)); + if (folder == null) + return null; - UUID uuid = ignite.localNode().id(); - String nid = uuid.toString().toLowerCase(); + folder = resolveSymbolicLink(folder); - String filePtrn = arg.filePtrn.replace("@nid8", nid.substring(0, 8)).replace("@nid", nid); + UUID uuid = ignite.localNode().id(); + String nid = uuid.toString().toLowerCase(); - try { - File fld = new File(url.toURI()); - int pathIdx = (fld.isDirectory() ? fld : fld.getParentFile()).getAbsolutePath().length() + 1; + String filePtrn = arg.filePtrn.replace("@nid8", nid.substring(0, 8)).replace("@nid", nid); + + int pathIdx = (folder.isDirectory() ? folder : folder.getParentFile()).getAbsolutePath().length() + 1; - List matchingFiles = matchedFiles(fld, filePtrn); + List matchingFiles = matchedFiles(folder, filePtrn); Collection results = new ArrayList<>(arg.limit); @@ -276,4 +276,4 @@ private List> searchInFile(File f, Charse return S.toString(VisorLogSearchJob.class, this); } } -} \ No newline at end of file +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java index 3f5003a03d401..7518d93e1aa8e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java @@ -30,7 +30,10 @@ import java.nio.charset.CharacterCodingException; import java.nio.charset.Charset; import java.nio.charset.CharsetDecoder; +import java.nio.file.Files; +import java.nio.file.LinkOption; import java.nio.file.Path; +import java.nio.file.Paths; import java.text.SimpleDateFormat; import java.util.ArrayList; import java.util.Arrays; @@ -494,6 +497,34 @@ public static Collection collectEvents(Ignite ignite, String evt return res; } + /** + * @param path Path to resolve only relative to IGNITE_HOME. + * @return Resolved path as file, or {@code null} if path cannot be resolved. + * @throws IOException If failed to resolve path. + */ + public static File resolveIgnitePath(String path) throws IOException { + File folder = U.resolveIgnitePath(path); + + if (folder == null) + return null; + + if (!folder.toPath().toRealPath(LinkOption.NOFOLLOW_LINKS).startsWith(Paths.get(U.getIgniteHome()))) + return null; + + return folder; + } + + /** + * @param file File to resolve. + * @return Resolved file if it is a symbolic link or original file. + * @throws IOException If failed to resolve symlink. + */ + public static File resolveSymbolicLink(File file) throws IOException { + Path path = file.toPath(); + + return Files.isSymbolicLink(path) ? Files.readSymbolicLink(path).toFile() : file; + } + /** * Finds all files in folder and in it's sub-tree of specified depth. * @@ -501,8 +532,11 @@ public static Collection collectEvents(Ignite ignite, String evt * @param maxDepth Depth of the tree. If 1 - just look in the folder, no sub-folders. * @param filter file filter. * @return List of found files. + * @throws IOException If failed to list files. */ - public static List fileTree(File file, int maxDepth, @Nullable FileFilter filter) { + public static List fileTree(File file, int maxDepth, @Nullable FileFilter filter) throws IOException { + file = resolveSymbolicLink(file); + if (file.isDirectory()) { File[] files = (filter == null) ? file.listFiles() : file.listFiles(filter); @@ -525,12 +559,13 @@ else if (maxDepth > 1) } /** - * @param fld Folder with files to match. + * @param file Folder with files to match. * @param ptrn Pattern to match against file name. * @return Collection of matched files. + * @throws IOException If failed to filter files. */ - public static List matchedFiles(File fld, final String ptrn) { - List files = fileTree(fld, MAX_FOLDER_DEPTH, + public static List matchedFiles(File file, final String ptrn) throws IOException { + List files = fileTree(file, MAX_FOLDER_DEPTH, new FileFilter() { @Override public boolean accept(File f) { return !f.isHidden() && (f.isDirectory() || f.isFile() && f.getName().matches(ptrn)); @@ -873,8 +908,6 @@ public static List startLocalNode(@Nullable IgniteLogger log, String cf if (cmdFilePath == null || !cmdFilePath.exists()) throw new FileNotFoundException(String.format("File not found: %s", cmdFile)); - String ignite = cmdFilePath.getCanonicalPath(); - File nodesCfgPath = U.resolveIgnitePath(cfgPath); if (nodesCfgPath == null || !nodesCfgPath.exists()) @@ -887,6 +920,8 @@ public static List startLocalNode(@Nullable IgniteLogger log, String cf List run = new ArrayList<>(); try { + String ignite = cmdFilePath.getCanonicalPath(); + for (int i = 0; i < nodesToStart; i++) { if (U.isMacOs()) { Map macEnv = new HashMap<>(System.getenv()); From 1d165d9c07fc8e13d6ee8d4e7b9cf1a83144e75e Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Fri, 15 Sep 2017 11:17:53 +0700 Subject: [PATCH 292/357] IGNITE-6367 Restrict files search by IGNITE_HOME. (cherry picked from commit 46296cf) --- .../visor/file/VisorLatestTextFilesTask.java | 15 +++--- .../visor/log/VisorLogSearchTask.java | 28 +++++------ .../internal/visor/util/VisorTaskUtils.java | 47 ++++++++++++++++--- 3 files changed, 61 insertions(+), 29 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/file/VisorLatestTextFilesTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/file/VisorLatestTextFilesTask.java index 55d358c78dd6e..63b526e1a1b77 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/file/VisorLatestTextFilesTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/file/VisorLatestTextFilesTask.java @@ -18,13 +18,11 @@ package org.apache.ignite.internal.visor.file; import java.io.File; -import java.net.URL; import java.util.ArrayList; import java.util.Collection; import java.util.List; import org.apache.ignite.internal.processors.task.GridInternal; import org.apache.ignite.internal.util.typedef.internal.S; -import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.internal.visor.VisorJob; import org.apache.ignite.internal.visor.VisorOneNodeTask; import org.apache.ignite.internal.visor.log.VisorLogFile; @@ -33,6 +31,7 @@ import static org.apache.ignite.internal.visor.util.VisorTaskUtils.LOG_FILES_COUNT_LIMIT; import static org.apache.ignite.internal.visor.util.VisorTaskUtils.matchedFiles; +import static org.apache.ignite.internal.visor.util.VisorTaskUtils.resolveIgnitePath; /** * Get list files matching filter. @@ -70,13 +69,11 @@ private VisorLatestTextFilesJob(IgniteBiTuple arg, boolean debug assert path != null; assert regexp != null; - URL url = U.resolveIgniteUrl(path); - - if (url == null) - return null; - try { - File folder = new File(url.toURI()); + File folder = resolveIgnitePath(path); + + if (folder == null) + return null; List files = matchedFiles(folder, regexp); @@ -98,4 +95,4 @@ private VisorLatestTextFilesJob(IgniteBiTuple arg, boolean debug return S.toString(VisorLatestTextFilesJob.class, this); } } -} \ No newline at end of file +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/log/VisorLogSearchTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/log/VisorLogSearchTask.java index b6552b2fca5f6..44fb0a90a5b0a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/log/VisorLogSearchTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/log/VisorLogSearchTask.java @@ -18,10 +18,8 @@ package org.apache.ignite.internal.visor.log; import java.io.File; -import java.io.FileNotFoundException; import java.io.IOException; import java.io.Serializable; -import java.net.URL; import java.nio.charset.Charset; import java.util.ArrayList; import java.util.Collection; @@ -35,7 +33,6 @@ import org.apache.ignite.internal.util.io.GridReversedLinesFileReader; import org.apache.ignite.internal.util.lang.GridTuple3; import org.apache.ignite.internal.util.typedef.internal.S; -import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.internal.visor.VisorJob; import org.apache.ignite.internal.visor.VisorMultiNodeTask; import org.apache.ignite.lang.IgniteBiTuple; @@ -43,6 +40,8 @@ import static org.apache.ignite.internal.visor.util.VisorTaskUtils.decode; import static org.apache.ignite.internal.visor.util.VisorTaskUtils.matchedFiles; +import static org.apache.ignite.internal.visor.util.VisorTaskUtils.resolveIgnitePath; +import static org.apache.ignite.internal.visor.util.VisorTaskUtils.resolveSymbolicLink; import static org.apache.ignite.internal.visor.util.VisorTaskUtils.textFile; /** @@ -216,21 +215,22 @@ private List> searchInFile(File f, Charse /** {@inheritDoc} */ @Override protected Collection run(VisorLogSearchArg arg) { - URL url = U.resolveIgniteUrl(arg.folder); + try { + File folder = resolveIgnitePath(arg.folder); - if (url == null) - throw new IgniteException(new FileNotFoundException("Log folder not found: " + arg.folder)); + if (folder == null) + return null; - UUID uuid = ignite.localNode().id(); - String nid = uuid.toString().toLowerCase(); + folder = resolveSymbolicLink(folder); - String filePtrn = arg.filePtrn.replace("@nid8", nid.substring(0, 8)).replace("@nid", nid); + UUID uuid = ignite.localNode().id(); + String nid = uuid.toString().toLowerCase(); - try { - File fld = new File(url.toURI()); - int pathIdx = (fld.isDirectory() ? fld : fld.getParentFile()).getAbsolutePath().length() + 1; + String filePtrn = arg.filePtrn.replace("@nid8", nid.substring(0, 8)).replace("@nid", nid); + + int pathIdx = (folder.isDirectory() ? folder : folder.getParentFile()).getAbsolutePath().length() + 1; - List matchingFiles = matchedFiles(fld, filePtrn); + List matchingFiles = matchedFiles(folder, filePtrn); Collection results = new ArrayList<>(arg.limit); @@ -276,4 +276,4 @@ private List> searchInFile(File f, Charse return S.toString(VisorLogSearchJob.class, this); } } -} \ No newline at end of file +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java index 3f5003a03d401..7518d93e1aa8e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java @@ -30,7 +30,10 @@ import java.nio.charset.CharacterCodingException; import java.nio.charset.Charset; import java.nio.charset.CharsetDecoder; +import java.nio.file.Files; +import java.nio.file.LinkOption; import java.nio.file.Path; +import java.nio.file.Paths; import java.text.SimpleDateFormat; import java.util.ArrayList; import java.util.Arrays; @@ -494,6 +497,34 @@ public static Collection collectEvents(Ignite ignite, String evt return res; } + /** + * @param path Path to resolve only relative to IGNITE_HOME. + * @return Resolved path as file, or {@code null} if path cannot be resolved. + * @throws IOException If failed to resolve path. + */ + public static File resolveIgnitePath(String path) throws IOException { + File folder = U.resolveIgnitePath(path); + + if (folder == null) + return null; + + if (!folder.toPath().toRealPath(LinkOption.NOFOLLOW_LINKS).startsWith(Paths.get(U.getIgniteHome()))) + return null; + + return folder; + } + + /** + * @param file File to resolve. + * @return Resolved file if it is a symbolic link or original file. + * @throws IOException If failed to resolve symlink. + */ + public static File resolveSymbolicLink(File file) throws IOException { + Path path = file.toPath(); + + return Files.isSymbolicLink(path) ? Files.readSymbolicLink(path).toFile() : file; + } + /** * Finds all files in folder and in it's sub-tree of specified depth. * @@ -501,8 +532,11 @@ public static Collection collectEvents(Ignite ignite, String evt * @param maxDepth Depth of the tree. If 1 - just look in the folder, no sub-folders. * @param filter file filter. * @return List of found files. + * @throws IOException If failed to list files. */ - public static List fileTree(File file, int maxDepth, @Nullable FileFilter filter) { + public static List fileTree(File file, int maxDepth, @Nullable FileFilter filter) throws IOException { + file = resolveSymbolicLink(file); + if (file.isDirectory()) { File[] files = (filter == null) ? file.listFiles() : file.listFiles(filter); @@ -525,12 +559,13 @@ else if (maxDepth > 1) } /** - * @param fld Folder with files to match. + * @param file Folder with files to match. * @param ptrn Pattern to match against file name. * @return Collection of matched files. + * @throws IOException If failed to filter files. */ - public static List matchedFiles(File fld, final String ptrn) { - List files = fileTree(fld, MAX_FOLDER_DEPTH, + public static List matchedFiles(File file, final String ptrn) throws IOException { + List files = fileTree(file, MAX_FOLDER_DEPTH, new FileFilter() { @Override public boolean accept(File f) { return !f.isHidden() && (f.isDirectory() || f.isFile() && f.getName().matches(ptrn)); @@ -873,8 +908,6 @@ public static List startLocalNode(@Nullable IgniteLogger log, String cf if (cmdFilePath == null || !cmdFilePath.exists()) throw new FileNotFoundException(String.format("File not found: %s", cmdFile)); - String ignite = cmdFilePath.getCanonicalPath(); - File nodesCfgPath = U.resolveIgnitePath(cfgPath); if (nodesCfgPath == null || !nodesCfgPath.exists()) @@ -887,6 +920,8 @@ public static List startLocalNode(@Nullable IgniteLogger log, String cf List run = new ArrayList<>(); try { + String ignite = cmdFilePath.getCanonicalPath(); + for (int i = 0; i < nodesToStart; i++) { if (U.isMacOs()) { Map macEnv = new HashMap<>(System.getenv()); From 6dfdaedc0b3cb9c7ea617d78753859d0e8b77595 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Thu, 14 Sep 2017 20:05:43 +0300 Subject: [PATCH 293/357] GG-12692: Fixed flaky test IgniteCachePeekModesAbstractTest.testNonLocalPartitionSize. (cherry picked from commit 64c337c) --- .../IgniteCachePeekModesAbstractTest.java | 37 ++++++++++++++----- 1 file changed, 27 insertions(+), 10 deletions(-) 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 fac24ccd00045..75485cdeb0d5b 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 @@ -996,23 +996,40 @@ private T2 swapKeysCount(int nodeIdx) { * @param part Cache partition * @return Tuple with number of primary and backup keys (one or both will be zero). */ - private T2 swapKeysCount(int nodeIdx, int part) throws IgniteCheckedException { + private T2 swapKeysCount(final int nodeIdx, final int part) throws IgniteCheckedException { GridCacheContext ctx = ((IgniteEx)ignite(nodeIdx)).context().cache().internalCache().context(); // Swap and offheap are disabled for near cache. GridCacheSwapManager swapMgr = ctx.isNear() ? ctx.near().dht().context().swap() : ctx.swap(); - //First count entries... - int cnt = (int)swapMgr.swapEntriesCount(part); - GridCacheAffinityManager affinity = ctx.affinity(); - AffinityTopologyVersion topVer = affinity.affinityTopologyVersion(); + AffinityTopologyVersion topVer = ctx.affinity().affinityTopologyVersion(); + + Affinity aff = ignite(nodeIdx).affinity(null); + + ClusterNode node = ignite(nodeIdx).cluster().localNode(); + + Iterator it = swapMgr.swapKeyIterator(true, true, topVer); + + CacheObjectContext coctx = ((IgniteEx)ignite(nodeIdx)).context().cache().internalCache() + .context().cacheObjectContext(); - //And then find out whether they are primary or backup ones. int primaryCnt = 0; int backupCnt = 0; - if (affinity.primaryByPartition(ctx.localNode(), part, topVer)) - primaryCnt = cnt; - else if (affinity.primaryByPartition(ctx.localNode(), part, topVer)) - backupCnt = cnt; + + while (it.hasNext()) { + Integer key = it.next().value(coctx, false); + + if (part != aff.partition(key)) + continue; + + if (aff.isPrimary(node, key)) + primaryCnt++; + else { + assertTrue(aff.isBackup(node, key)); + + backupCnt++; + } + } + return new T2<>(primaryCnt, backupCnt); } From a85a42d495bf449b7786fd9a8362be54b1fd2eea Mon Sep 17 00:00:00 2001 From: Dmitriy Govorukhin Date: Mon, 23 Jan 2017 20:35:02 +0300 Subject: [PATCH 294/357] Bakport fix for flacky test GridServiceProcessorProxySelfTest.testLocalProxyInvocation. (cherry picked from commit 3587ac6) --- .../GridServiceProcessorProxySelfTest.java | 23 ++++++++++++++++--- 1 file changed, 20 insertions(+), 3 deletions(-) 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 850cb5702b5b0..1418973940b41 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 @@ -20,9 +20,11 @@ import java.util.Map; import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicReference; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteException; import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.util.typedef.PA; import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.services.Service; import org.apache.ignite.services.ServiceContext; @@ -240,12 +242,27 @@ public void testLocalProxyInvocation() throws Exception { ignite.services().deployNodeSingleton(name, new MapServiceImpl()); for (int i = 0; i < nodeCount(); i++) { - MapService svc = grid(i).services().serviceProxy(name, MapService.class, false); + final int idx = i; + + final AtomicReference< MapService> ref = new AtomicReference<>(); + + //wait because after deployNodeSingleton we don't have guarantees what service was deploy. + boolean wait = GridTestUtils.waitForCondition(new PA() { + @Override public boolean apply() { + MapService svc = grid(idx) + .services() + .serviceProxy(name, MapService.class, false); + + ref.set(svc); + + return svc instanceof Service; + } + }, 2000); // Make sure service is a local instance. - assertTrue("Invalid service instance [srv=" + svc + ", node=" + i + ']', svc instanceof Service); + assertTrue("Invalid service instance [srv=" + ref.get() + ", node=" + i + ']', wait); - svc.put(i, Integer.toString(i)); + ref.get().put(i, Integer.toString(i)); } MapService map = ignite.services().serviceProxy(name, MapService.class, false); From 238c7723e95dfed8bbd69bf26c98011490780a68 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Mon, 18 Sep 2017 12:54:33 +0300 Subject: [PATCH 295/357] GG-12717: Backport fix CacheJdbcPojoStoreTest from master. --- .../store/jdbc/CacheAbstractJdbcStore.java | 67 +++++---- .../cache/store/CacheOsStoreManager.java | 3 + .../store/GridCacheStoreManagerAdapter.java | 8 ++ .../CacheJdbcPojoStoreAbstractSelfTest.java | 19 ++- ...naryMarshallerStoreKeepBinarySelfTest.java | 28 ++++ ...rStoreKeepBinaryWithSqlEscapeSelfTest.java | 28 ++++ .../store/jdbc/CacheJdbcPojoStoreTest.java | 136 ++++++++++++++---- .../testframework/junits/IgniteMock.java | 4 + .../testsuites/IgniteCacheTestSuite.java | 4 + 9 files changed, 237 insertions(+), 60 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerStoreKeepBinarySelfTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerStoreKeepBinaryWithSqlEscapeSelfTest.java 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 817b1a5da8531..64bdec90f5296 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 @@ -61,6 +61,7 @@ import org.apache.ignite.cache.store.jdbc.dialect.OracleDialect; import org.apache.ignite.cache.store.jdbc.dialect.SQLServerDialect; import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.binary.BinaryEnumObjectImpl; import org.apache.ignite.internal.binary.BinaryMarshaller; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.typedef.C1; @@ -560,22 +561,35 @@ private JdbcTypeField[] translateFields(Collection oldFl /** * @param type Type name to check. + * @param binarySupported True if binary marshaller enable. * @return {@code True} if class not found. */ - protected TypeKind kindForName(String type) { + protected TypeKind kindForName(String type, boolean binarySupported) { if (BUILT_IN_TYPES.contains(type)) return TypeKind.BUILT_IN; + if (binarySupported) + return TypeKind.BINARY; + try { Class.forName(type); return TypeKind.POJO; } - catch(ClassNotFoundException ignored) { - return TypeKind.BINARY; + catch (ClassNotFoundException e) { + throw new CacheException("Failed to find class " + type + + " (make sure the class is present in classPath or use BinaryMarshaller)", e); } } + /** + * @param type Type name to check. + * @return {@code True} if class not found. + */ + protected TypeKind kindForName(String type) { + return kindForName(type, ignite.configuration().getMarshaller() instanceof BinaryMarshaller); + } + /** * @param cacheName Cache name to check mappings for. * @return Type mappings for specified cache name. @@ -640,11 +654,7 @@ private Map getOrCreateCacheMappings(@Nullable String cach String keyType = type.getKeyType(); String valType = type.getValueType(); - TypeKind keyKind = kindForName(keyType); - - if (!binarySupported && keyKind == TypeKind.BINARY) - throw new CacheException("Key type has no class [cache=" + U.maskName(cacheName) + - ", type=" + keyType + "]"); + TypeKind keyKind = kindForName(keyType, binarySupported); checkTypeConfiguration(cacheName, keyKind, keyType, type.getKeyFields()); @@ -654,21 +664,11 @@ private Map getOrCreateCacheMappings(@Nullable String cach throw new CacheException("Key type must be unique in type metadata [cache=" + U.maskName(cacheName) + ", type=" + keyType + "]"); - TypeKind valKind = kindForName(valType); + TypeKind valKind = kindForName(valType, binarySupported); checkTypeConfiguration(cacheName, valKind, valType, type.getValueFields()); entryMappings.put(keyTypeId, new EntryMapping(cacheName, dialect, type, keyKind, valKind, sqlEscapeAll)); - - // Add one more binding to binary typeId for POJOs, - // because object could be passed to store in binary format. - if (binarySupported && keyKind == TypeKind.POJO) { - keyTypeId = typeIdForTypeName(TypeKind.BINARY, keyType); - - valKind = valKind == TypeKind.POJO ? TypeKind.BINARY : valKind; - - entryMappings.put(keyTypeId, new EntryMapping(cacheName, dialect, type, TypeKind.BINARY, valKind, sqlEscapeAll)); - } } Map> mappings = new HashMap<>(cacheMappings); @@ -1419,10 +1419,17 @@ protected void fillParameter(PreparedStatement stmt, int idx, JdbcTypeField fiel // No-op. } } - else if (field.getJavaFieldType().isEnum() && fieldVal instanceof Enum) { - Enum val = (Enum)fieldVal; - - fieldVal = NUMERIC_TYPES.contains(field.getDatabaseFieldType()) ? val.ordinal() : val.name(); + else if (field.getJavaFieldType().isEnum()) { + if (fieldVal instanceof Enum) { + Enum val = (Enum)fieldVal; + + fieldVal = NUMERIC_TYPES.contains(field.getDatabaseFieldType()) ? val.ordinal() : val.name(); + } + else if (fieldVal instanceof BinaryEnumObjectImpl) { + BinaryEnumObjectImpl val = (BinaryEnumObjectImpl)fieldVal; + + fieldVal = val.enumOrdinal(); + } } stmt.setObject(idx, fieldVal); @@ -1474,14 +1481,14 @@ protected int fillKeyParameters(PreparedStatement stmt, EntryMapping m, Object k */ protected int fillValueParameters(PreparedStatement stmt, int idx, EntryMapping em, Object val) throws CacheWriterException { - TypeKind valKind = em.valueKind(); - - // Object could be passed by cache in binary format in case of cache configured with setStoreKeepBinary(true). - if (valKind == TypeKind.POJO && val instanceof BinaryObject) - valKind = TypeKind.BINARY; - for (JdbcTypeField field : em.uniqValFlds) { - Object fieldVal = extractParameter(em.cacheName, em.valueType(), valKind, field.getJavaFieldName(), val); + Object fieldVal = extractParameter( + em.cacheName, + em.valueType(), + em.valueKind(), + field.getJavaFieldName(), + val + ); fillParameter(stmt, idx++, field, fieldVal); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java index 27771ff3e629a..2e23d04f735d8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java @@ -77,6 +77,9 @@ public CacheOsStoreManager(GridKernalContext ctx, CacheConfiguration cfg) { /** {@inheritDoc} */ @Override public boolean convertBinary() { + if (alwaysKeepBinary) + return false; + return configuredConvertBinary() && !(cfgStore instanceof PlatformCacheStore); } 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 6ab8c56578593..5d5c7d667c94e 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 @@ -33,9 +33,11 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.cache.store.CacheStore; +import org.apache.ignite.cache.store.CacheStore; import org.apache.ignite.cache.store.CacheStoreSession; import org.apache.ignite.cache.store.CacheStoreSessionListener; import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStore; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.processors.cache.CacheEntryImpl; @@ -113,6 +115,9 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt /** */ private boolean globalSesLsnrs; + /** Always keep binary. */ + protected boolean alwaysKeepBinary; + /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public void initialize(@Nullable CacheStore cfgStore, Map sesHolders) throws IgniteCheckedException { @@ -148,6 +153,9 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt sesHolder = sesHolder0; locStore = U.hasAnnotation(cfgStore, CacheLocalStore.class); + + if (cfgStore instanceof CacheJdbcPojoStore) + alwaysKeepBinary = true; } /** {@inheritDoc} */ 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 9e59769503f53..5ca6ca45364f8 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 @@ -35,6 +35,7 @@ import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.ConnectorConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.binary.BinaryMarshaller; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; @@ -221,6 +222,7 @@ protected CacheConfiguration cacheConfiguration() throws Exception { cc.setAtomicityMode(transactional ? TRANSACTIONAL : ATOMIC); cc.setSwapEnabled(false); cc.setWriteBehindEnabled(false); + cc.setStoreKeepBinary(storeKeepBinary()); CacheJdbcPojoStoreFactory storeFactory = new CacheJdbcPojoStoreFactory<>(); storeFactory.setDialect(new H2Dialect()); @@ -237,6 +239,13 @@ protected CacheConfiguration cacheConfiguration() throws Exception { return cc; } + /** + * @return Flag indicate keep value in binary format or not. + */ + protected boolean storeKeepBinary(){ + return false; + } + /** * Fill in-memory database with sample data. * @@ -446,6 +455,8 @@ public void testLoadCachePrimitiveKeysTx() throws Exception { * @throws Exception If failed. */ private void checkPutRemove() throws Exception { + boolean binaryMarshaller = marshaller() instanceof BinaryMarshaller || marshaller() == null; + IgniteCache c1 = grid().cache(CACHE_NAME); Connection conn = getConnection(); @@ -480,7 +491,9 @@ private void checkPutRemove() throws Exception { assertEquals(-2, rs.getInt(2)); assertEquals(testDate, rs.getDate(3)); assertEquals("Person-to-test-put-insert", rs.getString(4)); - assertEquals(testGender.toString(), rs.getString(5)); + + assertEquals(testGender.toString(), + binaryMarshaller ? Gender.values()[rs.getInt(5)].toString(): rs.getString(5)); assertFalse("Unexpected more data in result set", rs.next()); @@ -499,7 +512,9 @@ private void checkPutRemove() throws Exception { assertEquals(-3, rs.getInt(2)); assertEquals(testDate, rs.getDate(3)); assertEquals("Person-to-test-put-update", rs.getString(4)); - assertEquals(testGender.toString(), rs.getString(5)); + + assertEquals(testGender.toString(), + binaryMarshaller ? Gender.values()[rs.getInt(5)].toString(): rs.getString(5)); assertFalse("Unexpected more data in result set", rs.next()); diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerStoreKeepBinarySelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerStoreKeepBinarySelfTest.java new file mode 100644 index 0000000000000..dfca8647e2840 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerStoreKeepBinarySelfTest.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.cache.store.jdbc; + +/** + * + */ +public class CacheJdbcPojoStoreBinaryMarshallerStoreKeepBinarySelfTest extends CacheJdbcPojoStoreBinaryMarshallerSelfTest { + /** {@inheritDoc} */ + @Override protected boolean storeKeepBinary() { + return true; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerStoreKeepBinaryWithSqlEscapeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerStoreKeepBinaryWithSqlEscapeSelfTest.java new file mode 100644 index 0000000000000..c7e1f794ab208 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerStoreKeepBinaryWithSqlEscapeSelfTest.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.cache.store.jdbc; + +/** + * + */ +public class CacheJdbcPojoStoreBinaryMarshallerStoreKeepBinaryWithSqlEscapeSelfTest extends CacheJdbcPojoStoreBinaryMarshallerWithSqlEscapeSelfTest { + /** {@inheritDoc} */ + @Override protected boolean storeKeepBinary() { + return true; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreTest.java index 849cab79a03be..f17ece4e59686 100644 --- a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreTest.java +++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreTest.java @@ -17,6 +17,7 @@ package org.apache.ignite.cache.store.jdbc; +import java.lang.reflect.Field; import java.sql.Connection; import java.sql.PreparedStatement; import java.sql.SQLException; @@ -29,18 +30,22 @@ import java.util.UUID; import java.util.concurrent.ConcurrentLinkedQueue; import javax.cache.integration.CacheWriterException; - +import org.apache.ignite.Ignite; +import org.apache.ignite.binary.BinaryObject; +import org.apache.ignite.binary.BinaryObjectBuilder; import org.apache.ignite.cache.store.jdbc.dialect.H2Dialect; import org.apache.ignite.cache.store.jdbc.model.Organization; import org.apache.ignite.cache.store.jdbc.model.OrganizationKey; import org.apache.ignite.cache.store.jdbc.model.Person; import org.apache.ignite.cache.store.jdbc.model.PersonComplexKey; import org.apache.ignite.cache.store.jdbc.model.PersonKey; +import org.apache.ignite.internal.binary.BinaryMarshaller; import org.apache.ignite.internal.processors.cache.CacheEntryImpl; import org.apache.ignite.internal.util.typedef.CI2; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteBiInClosure; import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.config.GridTestProperties; import org.apache.ignite.testframework.junits.cache.GridAbstractCacheStoreSelfTest; import org.h2.jdbcx.JdbcConnectionPool; @@ -57,6 +62,12 @@ public class CacheJdbcPojoStoreTest extends GridAbstractCacheStoreSelfTest orgKeys = new ConcurrentLinkedQueue<>(); - final Collection prnKeys = new ConcurrentLinkedQueue<>(); - final Collection prnComplexKeys = new ConcurrentLinkedQueue<>(); + final Collection orgKeys = new ConcurrentLinkedQueue<>(); + final Collection prnKeys = new ConcurrentLinkedQueue<>(); + final Collection prnComplexKeys = new ConcurrentLinkedQueue<>(); IgniteBiInClosure c = new CI2() { @Override public void apply(Object k, Object v) { - if (k instanceof OrganizationKey && v instanceof Organization) - orgKeys.add((OrganizationKey)k); - else if (k instanceof PersonKey && v instanceof Person) - prnKeys.add((PersonKey)k); - else if (k instanceof PersonComplexKey && v instanceof Person) { - PersonComplexKey key = (PersonComplexKey)k; - - Person val = (Person)v; - - assertTrue("Key ID should be the same as value ID", key.getId() == val.getId()); - assertTrue("Key orgID should be the same as value orgID", key.getOrgId() == val.getOrgId()); - assertEquals("name" + key.getId(), val.getName()); - - prnComplexKeys.add((PersonComplexKey)k); + if (binaryEnable){ + if (k instanceof BinaryObject && v instanceof BinaryObject) { + BinaryObject key = (BinaryObject)k; + BinaryObject val = (BinaryObject)v; + + String keyType = key.type().typeName(); + String valType = val.type().typeName(); + + if (OrganizationKey.class.getName().equals(keyType) + && Organization.class.getName().equals(valType)) + orgKeys.add(key); + + if (PersonKey.class.getName().equals(keyType) + && Person.class.getName().equals(valType)) + prnKeys.add(key); + + if (PersonComplexKey.class.getName().equals(keyType) + && Person.class.getName().equals(valType)) + prnComplexKeys.add(key); + } + }else { + if (k instanceof OrganizationKey && v instanceof Organization) + orgKeys.add(k); + else if (k instanceof PersonKey && v instanceof Person) + prnKeys.add(k); + else if (k instanceof PersonComplexKey && v instanceof Person) { + PersonComplexKey key = (PersonComplexKey)k; + + Person val = (Person)v; + + assertTrue("Key ID should be the same as value ID", key.getId() == val.getId()); + assertTrue("Key orgID should be the same as value orgID", key.getOrgId() == val.getOrgId()); + assertEquals("name" + key.getId(), val.getName()); + + prnComplexKeys.add(k); + } } } }; @@ -322,15 +361,16 @@ else if (k instanceof PersonComplexKey && v instanceof Person) { assertEquals(PERSON_CNT, prnKeys.size()); assertEquals(PERSON_CNT, prnComplexKeys.size()); - Collection tmpOrgKeys = new ArrayList<>(orgKeys); - Collection tmpPrnKeys = new ArrayList<>(prnKeys); - Collection tmpPrnComplexKeys = new ArrayList<>(prnComplexKeys); + Collection tmpOrgKeys = new ArrayList<>(orgKeys); + Collection tmpPrnKeys = new ArrayList<>(prnKeys); + Collection tmpPrnComplexKeys = new ArrayList<>(prnComplexKeys); orgKeys.clear(); prnKeys.clear(); prnComplexKeys.clear(); - store.loadCache(c, OrganizationKey.class.getName(), "SELECT name, city, id FROM ORGANIZATION", + store.loadCache( + c, OrganizationKey.class.getName(), "SELECT name, city, id FROM ORGANIZATION", PersonKey.class.getName(), "SELECT org_id, id, name FROM Person WHERE id < 1000"); assertEquals(ORGANIZATION_CNT, orgKeys.size()); @@ -380,14 +420,29 @@ public void testParallelLoad() throws Exception { U.closeQuiet(conn); - final Collection prnComplexKeys = new ConcurrentLinkedQueue<>(); + final Collection prnComplexKeys = new ConcurrentLinkedQueue<>(); IgniteBiInClosure c = new CI2() { @Override public void apply(Object k, Object v) { - if (k instanceof PersonComplexKey && v instanceof Person) - prnComplexKeys.add((PersonComplexKey)k); - else - fail("Unexpected entry [key=" + k + ", value=" + v + "]"); + if (binaryEnable) { + if (k instanceof BinaryObject && v instanceof BinaryObject) { + BinaryObject key = (BinaryObject)k; + BinaryObject val = (BinaryObject)v; + + String keyType = key.type().typeName(); + String valType = val.type().typeName(); + + if (PersonComplexKey.class.getName().equals(keyType) + && Person.class.getName().equals(valType)) + prnComplexKeys.add(key); + } + } + else { + if (k instanceof PersonComplexKey && v instanceof Person) + prnComplexKeys.add(k); + else + fail("Unexpected entry [key=" + k + ", value=" + v + "]"); + } } }; @@ -440,7 +495,7 @@ public void testWriteRetry() throws Exception { ses.newSession(null); try { - store.write(new CacheEntryImpl<>(k1, v1)); + store.write(new CacheEntryImpl<>(wrap(k1), wrap(v1))); fail("CacheWriterException wasn't thrown."); } @@ -469,4 +524,29 @@ public void testTimestamp() throws Exception { assertNull(store.load(k)); } + + /** + * @param obj Object. + */ + private Object wrap(Object obj) throws IllegalAccessException { + if (binaryEnable) { + Class cls = obj.getClass(); + + BinaryObjectBuilder builder = ig.binary().builder(cls.getName()); + + for (Field f : cls.getDeclaredFields()) { + if (f.getName().contains("serialVersionUID")) + continue; + + f.setAccessible(true); + + builder.setField(f.getName(), f.get(obj)); + } + + return builder.build(); + } + + return obj; + } + } diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java index 5722fa3d6056a..14ba030e24590 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java @@ -55,6 +55,7 @@ import org.apache.ignite.configuration.NearCacheConfiguration; import org.apache.ignite.internal.binary.BinaryCachingMetadataHandler; import org.apache.ignite.internal.binary.BinaryContext; +import org.apache.ignite.internal.binary.BinaryMarshaller; import org.apache.ignite.internal.binary.builder.BinaryObjectBuilderImpl; import org.apache.ignite.internal.processors.cacheobject.NoOpBinary; import org.apache.ignite.internal.util.typedef.internal.U; @@ -327,6 +328,9 @@ public IgniteMock( return typeName.hashCode(); } }; + + if (marshaller instanceof BinaryMarshaller) + ctx.configure((BinaryMarshaller)marshaller, configuration()); } binaryMock = new NoOpBinary() { diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java index deec72af95cd8..dce93747f12bd 100755 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java @@ -32,6 +32,8 @@ import org.apache.ignite.cache.store.GridCacheLoadOnlyStoreAdapterSelfTest; import org.apache.ignite.cache.store.StoreResourceInjectionSelfTest; import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreBinaryMarshallerSelfTest; +import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreBinaryMarshallerStoreKeepBinarySelfTest; +import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreBinaryMarshallerStoreKeepBinaryWithSqlEscapeSelfTest; import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreBinaryMarshallerWithSqlEscapeSelfTest; import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreMultitreadedSelfTest; import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreOptimizedMarshallerSelfTest; @@ -252,7 +254,9 @@ public static TestSuite suite(Set ignoredTests) throws Exception { suite.addTestSuite(CacheJdbcPojoStoreOptimizedMarshallerSelfTest.class); suite.addTestSuite(CacheJdbcPojoStoreOptimizedMarshallerWithSqlEscapeSelfTest.class); suite.addTestSuite(CacheJdbcPojoStoreBinaryMarshallerSelfTest.class); + suite.addTestSuite(CacheJdbcPojoStoreBinaryMarshallerStoreKeepBinarySelfTest.class); suite.addTestSuite(CacheJdbcPojoStoreBinaryMarshallerWithSqlEscapeSelfTest.class); + suite.addTestSuite(CacheJdbcPojoStoreBinaryMarshallerStoreKeepBinaryWithSqlEscapeSelfTest.class); suite.addTestSuite(CacheJdbcPojoStoreMultitreadedSelfTest.class); suite.addTestSuite(GridCacheBalancingStoreSelfTest.class); suite.addTestSuite(GridCacheAffinityApiSelfTest.class); From 8aa662dbed2f9ce27931920ace6efafbf0fc6671 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Thu, 13 Apr 2017 16:36:54 +0300 Subject: [PATCH 296/357] GG-12773: Fixed potential deadlock when service is being deployed while node is stopping. (cherry picked from commit 9ad68c7) --- .../service/GridServiceProcessor.java | 171 ++++++++++-------- .../GridServiceProcessorStopSelfTest.java | 137 +++++++++++++- 2 files changed, 228 insertions(+), 80 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 c961eb0443cdc..7b6311c4018a0 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 @@ -493,124 +493,137 @@ public IgniteInternalFuture deployKeyAffinitySingleton(String name, Service s public IgniteInternalFuture deploy(ServiceConfiguration cfg) { A.notNull(cfg, "cfg"); - ServicesCompatibilityState state = markCompatibilityStateAsUsed(); + if (!busyLock.enterBusy()) { + IgniteCheckedException e = new IgniteCheckedException("Service deployment has been cancelled (node is stopping): " + + cfg.getName()); - validate(cfg); - - ctx.security().authorize(cfg.getName(), SecurityPermission.SERVICE_DEPLOY, null); + return new GridFinishedFuture<>(e); + } - if (!state.srvcCompatibility) { - Marshaller marsh = ctx.config().getMarshaller(); + try { - LazyServiceConfiguration cfg0; + ServicesCompatibilityState state = markCompatibilityStateAsUsed(); - try { - byte[] srvcBytes = U.marshal(marsh, cfg.getService()); + validate(cfg); - cfg0 = new LazyServiceConfiguration(cfg, srvcBytes); - } - catch (IgniteCheckedException e) { - U.error(log, "Failed to marshal service with configured marshaller [srvc=" + cfg.getService() - + ", marsh=" + marsh + "]", e); + ctx.security().authorize(cfg.getName(), SecurityPermission.SERVICE_DEPLOY, null); - return new GridFinishedFuture<>(e); - } + if (!state.srvcCompatibility) { + Marshaller marsh = ctx.config().getMarshaller(); - cfg = cfg0; - } + LazyServiceConfiguration cfg0; - GridServiceDeploymentFuture fut = new GridServiceDeploymentFuture(cfg); + try { + byte[] srvcBytes = U.marshal(marsh, cfg.getService()); - GridServiceDeploymentFuture old = depFuts.putIfAbsent(cfg.getName(), fut); + cfg0 = new LazyServiceConfiguration(cfg, srvcBytes); + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to marshal service with configured marshaller [srvc=" + cfg.getService() + + ", marsh=" + marsh + "]", e); - if (old != null) { - if (!old.configuration().equalsIgnoreNodeFilter(cfg)) { - fut.onDone(new IgniteCheckedException("Failed to deploy service (service already exists with " + - "different configuration) [deployed=" + old.configuration() + ", new=" + cfg + ']')); + return new GridFinishedFuture<>(e); + } - return fut; + cfg = cfg0; } - return old; - } + GridServiceDeploymentFuture fut = new GridServiceDeploymentFuture(cfg); - if (ctx.clientDisconnected()) { - fut.onDone(new IgniteClientDisconnectedCheckedException(ctx.cluster().clientReconnectFuture(), - "Failed to deploy service, client node disconnected.")); + GridServiceDeploymentFuture old = depFuts.putIfAbsent(cfg.getName(), fut); - depFuts.remove(cfg.getName(), fut); - } + if (old != null) { + if (!old.configuration().equalsIgnoreNodeFilter(cfg)) { + fut.onDone(new IgniteCheckedException("Failed to deploy service (service already exists with " + + "different configuration) [deployed=" + old.configuration() + ", new=" + cfg + ']')); - while (true) { - try { - GridServiceDeploymentKey key = new GridServiceDeploymentKey(cfg.getName()); + return fut; + } + + return old; + } - if (ctx.deploy().enabled()) - ctx.cache().context().deploy().ignoreOwnership(true); + if (ctx.clientDisconnected()) { + fut.onDone(new IgniteClientDisconnectedCheckedException(ctx.cluster().clientReconnectFuture(), + "Failed to deploy service, client node disconnected.")); + depFuts.remove(cfg.getName(), fut); + } + + while (true) { try { - GridServiceDeployment dep = (GridServiceDeployment)cache.getAndPutIfAbsent(key, - new GridServiceDeployment(ctx.localNodeId(), cfg)); + GridServiceDeploymentKey key = new GridServiceDeploymentKey(cfg.getName()); - if (dep != null) { - if (!dep.configuration().equalsIgnoreNodeFilter(cfg)) { - // Remove future from local map. - depFuts.remove(cfg.getName(), fut); + if (ctx.deploy().enabled()) + ctx.cache().context().deploy().ignoreOwnership(true); - fut.onDone(new IgniteCheckedException("Failed to deploy service (service already exists with " + - "different configuration) [deployed=" + dep.configuration() + ", new=" + cfg + ']')); - } - else { - Iterator> it = serviceEntries( - ServiceAssignmentsPredicate.INSTANCE); + try { + GridServiceDeployment dep = (GridServiceDeployment)cache.getAndPutIfAbsent(key, + new GridServiceDeployment(ctx.localNodeId(), cfg)); + + if (dep != null) { + if (!dep.configuration().equalsIgnoreNodeFilter(cfg)) { + // Remove future from local map. + depFuts.remove(cfg.getName(), fut); + + fut.onDone(new IgniteCheckedException("Failed to deploy service (service already exists with " + + "different configuration) [deployed=" + dep.configuration() + ", new=" + cfg + ']')); + } + else { + Iterator> it = serviceEntries( + ServiceAssignmentsPredicate.INSTANCE); - while (it.hasNext()) { - Cache.Entry e = it.next(); + while (it.hasNext()) { + Cache.Entry e = it.next(); - if (e.getKey() instanceof GridServiceAssignmentsKey) { - GridServiceAssignments assigns = (GridServiceAssignments)e.getValue(); + if (e.getKey() instanceof GridServiceAssignmentsKey) { + GridServiceAssignments assigns = (GridServiceAssignments)e.getValue(); - if (assigns.name().equals(cfg.getName())) { - // Remove future from local map. - depFuts.remove(cfg.getName(), fut); + if (assigns.name().equals(cfg.getName())) { + // Remove future from local map. + depFuts.remove(cfg.getName(), fut); - fut.onDone(); + fut.onDone(); - break; + break; + } } } - } - if (!dep.configuration().equalsIgnoreNodeFilter(cfg)) - U.warn(log, "Service already deployed with different configuration (will ignore) " + - "[deployed=" + dep.configuration() + ", new=" + cfg + ']'); + if (!dep.configuration().equalsIgnoreNodeFilter(cfg)) + U.warn(log, "Service already deployed with different configuration (will ignore) " + + "[deployed=" + dep.configuration() + ", new=" + cfg + ']'); + } } } - } - finally { - if (ctx.deploy().enabled()) - ctx.cache().context().deploy().ignoreOwnership(false); - } + finally { + if (ctx.deploy().enabled()) + ctx.cache().context().deploy().ignoreOwnership(false); + } - return fut; - } - catch (ClusterTopologyCheckedException e) { - if (log.isDebugEnabled()) - log.debug("Topology changed while deploying service (will retry): " + e.getMessage()); - } - catch (IgniteCheckedException e) { - if (e.hasCause(ClusterTopologyCheckedException.class)) { + return fut; + } + catch (ClusterTopologyCheckedException e) { if (log.isDebugEnabled()) log.debug("Topology changed while deploying service (will retry): " + e.getMessage()); - - continue; } + catch (IgniteCheckedException e) { + if (e.hasCause(ClusterTopologyCheckedException.class)) { + if (log.isDebugEnabled()) + log.debug("Topology changed while deploying service (will retry): " + e.getMessage()); - U.error(log, "Failed to deploy service: " + cfg.getName(), e); + continue; + } - return new GridFinishedFuture<>(e); + U.error(log, "Failed to deploy service: " + cfg.getName(), e); + + return new GridFinishedFuture<>(e); + } } } + finally { + busyLock.leaveBusy(); + } } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorStopSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorStopSelfTest.java index 03b00f47e6c0d..dc2dd1262c775 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorStopSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorStopSelfTest.java @@ -21,6 +21,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.Lock; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; @@ -31,8 +32,12 @@ import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.util.lang.GridAbsPredicate; +import org.apache.ignite.internal.util.lang.GridPlainCallable; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.services.Service; +import org.apache.ignite.services.ServiceConfiguration; import org.apache.ignite.services.ServiceContext; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; @@ -161,8 +166,138 @@ public void testStopDuringHangedDeployment() throws Exception { assertTrue("Deploy future isn't completed", wait); fut.get(); + } - Ignition.stopAll(true); + /** + * @throws Exception If fails + */ + public void testServiceDeploymentCancelationOnStop() throws Exception { + final Ignite node = startGrid(0); + + final IgniteServices services = node.services(); + // Deploy some service. + services.deploy(getServiceConfiguration("myService1")); + + //Stop node async, this will cancel the service #1. + final IgniteInternalFuture stopAsync = GridTestUtils.runAsync(new GridPlainCallable() { + @Override public Boolean call() throws Exception { + node.close(); + + return true; + } + }, "node-stopping-thread"); + + // Wait for the service #1 cancellation during node stopping. + // At this point node.stop process will be paused until svcCancelFinishLatch released. + ServiceImpl.svcCancelStartLatch.await(); + + final AtomicReference queuedFuture = new AtomicReference<>(); + + // Try to deploy another service. + final IgniteInternalFuture deployAsync = GridTestUtils.runAsync(new GridPlainCallable() { + @Override public Boolean call() throws Exception { + IgniteServices async = services.withAsync(); + + async.deploy(getServiceConfiguration("myService2")); + + IgniteFuture future = async.future(); + + queuedFuture.set(future); + + // Here, deployment future is added to queue and + // then it will be cancelled when svcCancelFinishLatch be released. + // So, we'll wait for queue cleaning and try to deploy one more service. + try { + future.get(); + } + catch (Exception ignore) { + // Noop. + } + + // Normally, this should fail with some Exception as node is stopping right now. + // But we face a deadlock here. + for (int i = 0; i < 5; i++) { + try { + services.deploy(getServiceConfiguration("service3")); + } + catch (Exception ignore) { + // Noop. + } + } + + return true; + } + }, "svc-deploy-thread"); + + GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + return queuedFuture.get() != null; + } + }, 3000); + + // Allow node to be stopped. + ServiceImpl.svcCancelFinishLatch.countDown(); + + // Wait for all service deployments have finished. + boolean deployDone = GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + IgniteFuture fut = queuedFuture.get(); + + return fut != null && fut.isDone() && deployAsync.isDone(); + + } + }, 5000); + + assertTrue("Node stopping and service deployment processes falls into a deadlock.", deployDone); + + if (!deployDone) + deployAsync.cancel(); + + if (!stopAsync.isDone()) + stopAsync.cancel(); + } + + /** */ + private ServiceConfiguration getServiceConfiguration(String svcName) { + ServiceConfiguration svcCfg = new ServiceConfiguration(); + svcCfg.setName(svcName); + svcCfg.setService(new ServiceImpl()); + svcCfg.setTotalCount(1); + + return svcCfg; + } + + /** Dummy Implementation. */ + static class ServiceImpl implements Service { + /** */ + static final CountDownLatch svcCancelStartLatch = new CountDownLatch(1); + + /** */ + static final CountDownLatch svcCancelFinishLatch = new CountDownLatch(1); + + /** {@inheritDoc} */ + @Override public void cancel(ServiceContext ctx) { + System.out.println("cancel service: " + ctx.executionId()); + try { + svcCancelStartLatch.countDown(); + + svcCancelFinishLatch.await(); + } + catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + + /** {@inheritDoc} */ + @Override public void init(ServiceContext ctx) throws Exception { + System.out.println("init service: " + ctx.executionId()); + // No-op + } + + /** {@inheritDoc} */ + @Override public void execute(ServiceContext ctx) throws Exception { + // No-op + } } /** From 87ddd3f87ba6746307a7ba54947ffd1d4b5c6076 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Fri, 22 Sep 2017 16:50:27 +0300 Subject: [PATCH 297/357] GG-12755 - Backport ExchangeActions from master --- .../cache/CacheAffinitySharedManager.java | 215 +++++++++------- .../cache/DynamicCacheChangeBatch.java | 26 +- .../processors/cache/ExchangeActions.java | 240 ++++++++++++++++++ .../GridCachePartitionExchangeManager.java | 20 +- .../processors/cache/GridCacheProcessor.java | 94 ++++--- .../dht/GridDhtAssignmentFetchFuture.java | 11 +- .../GridDhtPartitionsExchangeFuture.java | 156 +++++------- 7 files changed, 522 insertions(+), 240 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java index 7cbd333907058..de2390ece1703 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java @@ -381,36 +381,17 @@ public void forceCloseCache(final GridDhtPartitionsExchangeFuture fut, boolean c * * @param fut Exchange future. * @param crd Coordinator flag. - * @param reqs Cache change requests. + * @param exchActions Exchange actions. * @throws IgniteCheckedException If failed. * @return {@code True} if client-only exchange is needed. */ public boolean onCacheChangeRequest(final GridDhtPartitionsExchangeFuture fut, boolean crd, - Collection reqs) + final ExchangeActions exchActions) throws IgniteCheckedException { - assert !F.isEmpty(reqs) : fut; - - for (DynamicCacheChangeRequest req : reqs) { - Integer cacheId = CU.cacheId(req.cacheName()); - - if (req.stop()) { - DynamicCacheDescriptor desc = registeredCaches.remove(cacheId); - - assert desc != null : cacheId; - } - else if (req.start() && !req.clientStartOnly()) { - DynamicCacheDescriptor desc = new DynamicCacheDescriptor(cctx.kernalContext(), - req.startCacheConfiguration(), - req.cacheType(), - false, - req.deploymentId()); + assert exchActions != null && !exchActions.empty() : exchActions; - DynamicCacheDescriptor old = registeredCaches.put(cacheId, desc); - - assert old == null : old; - } - } + updateCachesInfo(exchActions); boolean clientOnly = true; @@ -426,82 +407,115 @@ else if (req.start() && !req.clientStartOnly()) { Set stoppedCaches = null; - for (DynamicCacheChangeRequest req : reqs) { - if (!(req.clientStartOnly() || req.close())) + for (ExchangeActions.CacheActionData action : exchActions.cacheStartRequests()) { + DynamicCacheDescriptor cacheDesc = action.descriptor(); + + DynamicCacheChangeRequest req = action.request(); + + Integer cacheId = cacheDesc.cacheId(); + + cctx.cache().prepareCacheStart(cacheDesc, req, fut.topologyVersion()); + + if (!req.clientStartOnly()) clientOnly = false; - Integer cacheId = CU.cacheId(req.cacheName()); + if (fut.isCacheAdded(cacheId, fut.topologyVersion())) { + if (fut.discoCache().cacheAffinityNodes(req.cacheName()).isEmpty()) + U.quietAndWarn(log, "No server nodes found for cache client: " + req.cacheName()); + } + + if (!crd || !lateAffAssign) { + GridCacheContext cacheCtx = cctx.cacheContext(cacheId); + + if (cacheCtx != null && !cacheCtx.isLocal()) { + boolean clientCacheStarted = + req.clientStartOnly() && req.initiatingNodeId().equals(cctx.localNodeId()); - if (req.start()) { - cctx.cache().prepareCacheStart(req, fut.topologyVersion()); + if (clientCacheStarted) + initAffinity(cacheCtx.affinity().affinityCache(), fut, lateAffAssign); + else if (!req.clientStartOnly()) { + assert fut.topologyVersion().equals(cacheCtx.startTopologyVersion()); - if (fut.isCacheAdded(cacheId, fut.topologyVersion())) { - if (fut.discoCache().cacheAffinityNodes(req.cacheName()).isEmpty()) - U.quietAndWarn(log, "No server nodes found for cache client: " + req.cacheName()); + GridAffinityAssignmentCache aff = cacheCtx.affinity().affinityCache(); + + assert aff.lastVersion().equals(AffinityTopologyVersion.NONE) : aff.lastVersion(); + + List> assignment = aff.calculate(fut.topologyVersion(), + fut.discoveryEvent(), fut.discoCache()); + + aff.initialize(fut.topologyVersion(), assignment); + } } + } + else + initStartedCacheOnCoordinator(fut, action.descriptor()); + } - if (!crd || !lateAffAssign) { - GridCacheContext cacheCtx = cctx.cacheContext(cacheId); + for (DynamicCacheChangeRequest req : exchActions.cacheCloseRequests(cctx.localNodeId())) { + Integer cacheId = CU.cacheId(req.cacheName()); - if (cacheCtx != null && !cacheCtx.isLocal()) { - boolean clientCacheStarted = - req.clientStartOnly() && req.initiatingNodeId().equals(cctx.localNodeId()); + cctx.cache().blockGateway(req); - if (clientCacheStarted) - initAffinity(cacheCtx.affinity().affinityCache(), fut, lateAffAssign); - else if (!req.clientStartOnly()) { - assert fut.topologyVersion().equals(cacheCtx.startTopologyVersion()); + if (crd) { + GridCacheContext cacheCtx = cctx.cacheContext(cacheId); - GridAffinityAssignmentCache aff = cacheCtx.affinity().affinityCache(); + // Client cache was stopped, need create 'client' CacheHolder. + if (cacheCtx != null && !cacheCtx.affinityNode()) { + CacheHolder cache = caches.remove(cacheId); - assert aff.lastVersion().equals(AffinityTopologyVersion.NONE) : aff.lastVersion(); + assert !cache.client() : cache; - List> assignment = aff.calculate(fut.topologyVersion(), - fut.discoveryEvent(), fut.discoCache()); + cache = CacheHolder2.create(cctx, + cctx.cache().cacheDescriptor(cacheId), + fut, + cache.affinity()); - aff.initialize(fut.topologyVersion(), assignment); - } - } + caches.put(cacheId, cache); } - else - initStartedCacheOnCoordinator(fut, cacheId); } - else if (req.stop() || req.close()) { - cctx.cache().blockGateway(req); + } - if (crd) { - boolean rmvCache = false; + for (ExchangeActions.CacheActionData action : exchActions.cacheStopRequests()) { + DynamicCacheChangeRequest req = action.request(); - if (req.close() && req.initiatingNodeId().equals(cctx.localNodeId())) { - GridCacheContext cacheCtx = cctx.cacheContext(cacheId); + clientOnly = false; - rmvCache = cacheCtx != null && !cacheCtx.affinityNode(); - } - else if (req.stop()) - rmvCache = true; + Integer cacheId = CU.cacheId(req.cacheName()); - if (rmvCache) { - CacheHolder cache = caches.remove(cacheId); + cctx.cache().blockGateway(req); - if (cache != null) { - if (!req.stop()) { - assert !cache.client(); + if (crd) { + boolean rmvCache = false; - cache = CacheHolder2.create(cctx, - cctx.cache().cacheDescriptor(cacheId), - fut, - cache.affinity()); + if (req.close() && req.initiatingNodeId().equals(cctx.localNodeId())) { + GridCacheContext cacheCtx = cctx.cacheContext(cacheId); - caches.put(cacheId, cache); - } - else { - if (stoppedCaches == null) - stoppedCaches = new HashSet<>(); + rmvCache = cacheCtx != null && !cacheCtx.affinityNode(); + } + else if (req.stop()) + rmvCache = true; - stoppedCaches.add(cache.cacheId()); + if (rmvCache) { + CacheHolder cache = caches.remove(cacheId); - cctx.io().removeHandler(cacheId, GridDhtAffinityAssignmentResponse.class); - } + if (cache != null) { + if (!req.stop()) { + assert !cache.client(); + + cache = CacheHolder2.create(cctx, + cctx.cache().cacheDescriptor(cacheId), + fut, + cache.affinity()); + + caches.put(cacheId, cache); + } + else { + if (stoppedCaches == null) + stoppedCaches = new HashSet<>(); + + stoppedCaches.add(cache.cacheId()); + + cctx.io().removeHandler(cacheId, GridDhtAffinityAssignmentResponse.class); } } } @@ -539,6 +553,37 @@ else if (req.stop()) return clientOnly; } + /** + * @param exchActions Cache change requests to execute on exchange. + */ + private void updateCachesInfo(ExchangeActions exchActions) { + for (ExchangeActions.CacheActionData action : exchActions.cacheStopRequests()) { + Integer cacheId = CU.cacheId(action.request().cacheName()); + + DynamicCacheDescriptor desc = registeredCaches.remove(cacheId); + + assert desc != null : cacheId; + } + + for (ExchangeActions.CacheActionData action : exchActions.cacheStartRequests()) { + DynamicCacheChangeRequest req = action.request(); + + Integer cacheId = CU.cacheId(req.cacheName()); + + if (!req.clientStartOnly()) { + DynamicCacheDescriptor desc = new DynamicCacheDescriptor(cctx.kernalContext(), + req.startCacheConfiguration(), + req.cacheType(), + false, + req.deploymentId()); + + DynamicCacheDescriptor old = registeredCaches.put(cacheId, desc); + + assert old == null : old; + } + } + } + /** * Called when received {@link CacheAffinityChangeMessage} which should complete exchange. * @@ -784,20 +829,20 @@ private void forAllCaches(boolean crd, IgniteInClosureX initCoordinatorCaches(final GridDhtPartitionsExc assert prev.topologyVersion().compareTo(fut.topologyVersion()) < 0 : prev; GridDhtAssignmentFetchFuture fetchFut = new GridDhtAssignmentFetchFuture(cctx, - aff.cacheName(), + aff.cacheId(), prev.topologyVersion(), prev.discoCache()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java index a2500633a1ccf..d9d8b0ca9e4df 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java @@ -47,6 +47,9 @@ public class DynamicCacheChangeBatch implements DiscoveryCustomMessage { /** */ private boolean clientReconnect; + /** */ + private transient ExchangeActions exchangeActions; + /** * @param reqs Requests. */ @@ -117,14 +120,23 @@ public boolean clientReconnect() { * @return {@code True} if request should trigger partition exchange. */ public boolean exchangeNeeded() { - if (reqs != null) { - for (DynamicCacheChangeRequest req : reqs) { - if (req.exchangeNeeded()) - return true; - } - } + return exchangeActions != null; + } - return false; + /** + * @return Exchange actions. + */ + public ExchangeActions exchangeActions() { + return exchangeActions; + } + + /** + * @param exchangeActions New exchange actions. + */ + public void exchangeActions(ExchangeActions exchangeActions) { + assert exchangeActions != null && !exchangeActions.empty() : exchangeActions; + + this.exchangeActions = exchangeActions; } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java new file mode 100644 index 0000000000000..0f5f914a2c003 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java @@ -0,0 +1,240 @@ +/* + * 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.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import org.apache.ignite.internal.util.typedef.F; +import org.jetbrains.annotations.Nullable; + +/** + * Cache change requests to execute when receive {@link DynamicCacheChangeBatch} event. + */ +public class ExchangeActions { + /** */ + private Map cachesToStart; + + /** */ + private Map cachesToStop; + + /** */ + private Map cachesToClose; + + /** + * @return New caches start requests. + */ + public Collection cacheStartRequests() { + return cachesToStart != null ? cachesToStart.values() : Collections.emptyList(); + } + + /** + * @return Stop cache requests. + */ + public Collection cacheStopRequests() { + return cachesToStop != null ? cachesToStop.values() : Collections.emptyList(); + } + + /** + * @return {@code True} if have start requests. + */ + public boolean hasStart() { + return !F.isEmpty(cachesToStart); + } + + /** + * @param cacheId Cache ID. + * @return {@code True} if cache stop was requested. + */ + public boolean cacheStopped(int cacheId) { + if (cachesToStop != null) { + for (CacheActionData cache : cachesToStop.values()) { + if (cache.desc.cacheId() == cacheId) + return true; + } + } + + return false; + } + + /** + * @param cacheId Cache ID. + * @return {@code True} if cache start was requested. + */ + public boolean cacheStarted(int cacheId) { + if (cachesToStart != null) { + for (CacheActionData cache : cachesToStart.values()) { + if (cache.desc.cacheId() == cacheId) + return true; + } + } + + return false; + } + + /** + * @param map Actions map. + * @param req Request. + * @param desc Cache descriptor. + * @return Actions map. + */ + private Map add(Map map, + DynamicCacheChangeRequest req, + DynamicCacheDescriptor desc) { + assert req != null; + assert desc != null; + + if (map == null) + map = new LinkedHashMap<>(); + + CacheActionData old = map.put(req.cacheName(), new CacheActionData(req, desc)); + + assert old == null : old; + + return map; + } + + /** + * @param req Request. + * @param desc Cache descriptor. + */ + void addCacheToStart(DynamicCacheChangeRequest req, DynamicCacheDescriptor desc) { + assert req.start() : req; + + cachesToStart = add(cachesToStart, req, desc); + } + + /** + * @param req Request. + * @param desc Cache descriptor. + */ + void addCacheToStop(DynamicCacheChangeRequest req, DynamicCacheDescriptor desc) { + assert req.stop() : req; + + cachesToStop = add(cachesToStop, req, desc); + } + + /** + * @param req Request. + * @param desc Cache descriptor. + */ + void addCacheToClose(DynamicCacheChangeRequest req, DynamicCacheDescriptor desc) { + assert req.close() : req; + + cachesToClose = add(cachesToClose, req, desc); + } + + /** + * @param nodeId Local node ID. + * @return Close cache requests. + */ + List cacheCloseRequests(UUID nodeId) { + List res = null; + + if (cachesToClose != null) { + for (CacheActionData req : cachesToClose.values()) { + if (nodeId.equals(req.req.initiatingNodeId())) { + if (res == null) + res = new ArrayList<>(cachesToClose.size()); + + res.add(req.req); + } + } + } + + return res != null ? res : Collections.emptyList(); + } + + /** + * @return {@code True} if there are no cache change actions. + */ + public boolean empty() { + return F.isEmpty(cachesToStart) && + F.isEmpty(cachesToStop) && + F.isEmpty(cachesToClose); + } + + /** + * @param ctx Context. + */ + public void completeRequestFutures(GridCacheSharedContext ctx, @Nullable Throwable err) { + completeRequestFutures(cachesToStart, ctx, err); + completeRequestFutures(cachesToStop, ctx, err); + completeRequestFutures(cachesToClose, ctx, err); + } + + /** + * @param map Actions map. + * @param ctx Context. + */ + private void completeRequestFutures(Map map, GridCacheSharedContext ctx, @Nullable Throwable err) { + if (map != null) { + for (CacheActionData req : map.values()) + ctx.cache().completeStartFuture(req.req, err); + } + } + + /** + * + */ + public static class CacheActionData { + /** */ + private final DynamicCacheChangeRequest req; + + /** */ + private final DynamicCacheDescriptor desc; + + /** + * @param req Request. + * @param desc Cache descriptor. + */ + CacheActionData(DynamicCacheChangeRequest req, DynamicCacheDescriptor desc) { + assert req != null; + assert desc != null; + + this.req = req; + this.desc = desc; + } + + /** + * @return Request. + */ + public DynamicCacheChangeRequest request() { + return req; + } + + /** + * @return Cache descriptor. + */ + public DynamicCacheDescriptor descriptor() { + return desc; + } + } + + /** {@inheritDoc} */ + @Override public String toString() { + return "ExchangeActions [startCaches=" + (cachesToStart != null ? cachesToStart.keySet() : null) + + ", stopCaches=" + (cachesToStop != null ? cachesToStop.keySet() : null) + + ", closeCaches=" + (cachesToClose != null ? cachesToClose.keySet() : null) + + ']'; + } +} 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 d146099243916..a41004bc771c7 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 @@ -229,13 +229,9 @@ public class GridCachePartitionExchangeManager extends GridCacheSharedMana if (customEvt.customMessage() instanceof DynamicCacheChangeBatch) { DynamicCacheChangeBatch batch = (DynamicCacheChangeBatch)customEvt.customMessage(); - Collection valid = new ArrayList<>(batch.requests().size()); - // Validate requests to check if event should trigger partition exchange. for (final DynamicCacheChangeRequest req : batch.requests()) { - if (req.exchangeNeeded()) - valid.add(req); - else { + if (!req.exchangeNeeded()) { IgniteInternalFuture fut = null; if (req.cacheFutureTopologyVersion() != null) @@ -253,10 +249,10 @@ public class GridCachePartitionExchangeManager extends GridCacheSharedMana } } - if (!F.isEmpty(valid)) { + if (batch.exchangeNeeded()) { exchId = exchangeId(n.id(), affinityTopologyVersion(evt), evt.type()); - exchFut = exchangeFuture(exchId, evt, cache, valid, null); + exchFut = exchangeFuture(exchId, evt, cache, batch.exchangeActions(), null); } } else if (customEvt.customMessage() instanceof CacheAffinityChangeMessage) { @@ -1091,25 +1087,25 @@ private GridDhtPartitionExchangeId exchangeId(UUID nodeId, AffinityTopologyVersi * @param exchId Exchange ID. * @param discoEvt Discovery event. * @param cache Discovery data cache. - * @param reqs Cache change requests. + * @param exchActions Exchange actions. * @param affChangeMsg Affinity change message. * @return Exchange future. */ private GridDhtPartitionsExchangeFuture exchangeFuture(GridDhtPartitionExchangeId exchId, @Nullable DiscoveryEvent discoEvt, @Nullable DiscoCache cache, - @Nullable Collection reqs, + @Nullable ExchangeActions exchActions, @Nullable CacheAffinityChangeMessage affChangeMsg) { GridDhtPartitionsExchangeFuture fut; GridDhtPartitionsExchangeFuture old = exchFuts.addx( - fut = new GridDhtPartitionsExchangeFuture(cctx, busyLock, exchId, reqs, affChangeMsg)); + fut = new GridDhtPartitionsExchangeFuture(cctx, busyLock, exchId, exchActions, affChangeMsg)); if (old != null) { fut = old; - if (reqs != null) - fut.cacheChangeRequests(reqs); + if (exchActions != null) + fut.exchangeActions(exchActions); if (affChangeMsg != null) fut.affinityChangeMessage(affChangeMsg); 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 96473e1f043d2..488cc2dd37e23 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 @@ -39,7 +39,6 @@ import javax.cache.configuration.Factory; import javax.cache.integration.CacheLoader; import javax.cache.integration.CacheWriter; -import javax.management.JMException; import javax.management.MBeanServer; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; @@ -63,7 +62,6 @@ import org.apache.ignite.configuration.TransactionConfiguration; import org.apache.ignite.events.EventType; import org.apache.ignite.internal.GridKernalContext; -import org.apache.ignite.internal.suggestions.GridPerformanceSuggestions; import org.apache.ignite.internal.IgniteClientDisconnectedCheckedException; import org.apache.ignite.internal.IgniteComponentType; import org.apache.ignite.internal.IgniteInternalFuture; @@ -102,6 +100,7 @@ import org.apache.ignite.internal.processors.plugin.CachePluginManager; import org.apache.ignite.internal.processors.query.GridQueryProcessor; import org.apache.ignite.internal.processors.timeout.GridTimeoutObject; +import org.apache.ignite.internal.suggestions.GridPerformanceSuggestions; import org.apache.ignite.internal.util.F0; import org.apache.ignite.internal.util.future.GridCompoundFuture; import org.apache.ignite.internal.util.future.GridFinishedFuture; @@ -1628,13 +1627,13 @@ public CacheMode cacheMode(String cacheName) { * @param topVer Topology version. * @throws IgniteCheckedException If failed. */ - public void prepareCacheStart(DynamicCacheChangeRequest req, AffinityTopologyVersion topVer) + public void prepareCacheStart(DynamicCacheDescriptor cacheDesc, + DynamicCacheChangeRequest req, + AffinityTopologyVersion topVer) throws IgniteCheckedException { assert req.start() : req; assert req.cacheType() != null : req; - String cacheName = maskNull(req.cacheName()); - prepareCacheStart( req.startCacheConfiguration(), req.nearCacheConfiguration(), @@ -1645,10 +1644,8 @@ public void prepareCacheStart(DynamicCacheChangeRequest req, AffinityTopologyVer topVer ); - DynamicCacheDescriptor desc = registeredCaches.get(cacheName); - - if (desc != null) - desc.onStart(); + if (cacheDesc != null) + cacheDesc.onStart(); } /** @@ -1820,14 +1817,14 @@ private void prepareCacheStop(DynamicCacheChangeRequest req, boolean forceClose) * Callback invoked when first exchange future for dynamic cache is completed. * * @param topVer Completed topology version. - * @param reqs Change requests. + * @param exchActions Exchange actions. * @param err Error. */ @SuppressWarnings("unchecked") public void onExchangeDone( AffinityTopologyVersion topVer, - Collection reqs, - Throwable err + @Nullable ExchangeActions exchActions, + @Nullable Throwable err ) { for (GridCacheAdapter cache : caches.values()) { GridCacheContext cacheCtx = cache.context(); @@ -1842,36 +1839,36 @@ public void onExchangeDone( } } - if (!F.isEmpty(reqs)) { - for (DynamicCacheChangeRequest req : reqs) { - if (err == null) { - String masked = maskNull(req.cacheName()); + if (exchActions != null && err == null) { + for (ExchangeActions.CacheActionData action : exchActions.cacheStopRequests()) { + DynamicCacheChangeRequest req = action.request(); - if (req.stop()) { - stopGateway(req); + stopGateway(req); - prepareCacheStop(req, false); - } - else if (req.close() && req.initiatingNodeId().equals(ctx.localNodeId())) { - IgniteCacheProxy proxy = jCacheProxies.remove(masked); + prepareCacheStop(req, false); + } - if (proxy != null) { - if (proxy.context().affinityNode()) { - GridCacheAdapter cache = caches.get(masked); + for (DynamicCacheChangeRequest req : exchActions.cacheCloseRequests(ctx.localNodeId())) { + String cacheName = maskNull(req.cacheName()); - if (cache != null) - jCacheProxies.put(masked, new IgniteCacheProxy(cache.context(), cache, null, false)); - } - else { - proxy.context().gate().onStopped(); + IgniteCacheProxy proxy = jCacheProxies.get(cacheName); - prepareCacheStop(req, false); - } - } + if (proxy != null) { + if (proxy.context().affinityNode()) { + GridCacheAdapter cache = caches.get(cacheName); + + assert cache != null : cacheName; + + jCacheProxies.put(cacheName, new IgniteCacheProxy(cache.context(), cache, null, false)); } - } + else { + jCacheProxies.remove(cacheName); - completeStartFuture(req, err); + proxy.context().gate().onStopped(); + + prepareCacheStop(req, false); + } + } } } } @@ -1880,7 +1877,7 @@ else if (req.close() && req.initiatingNodeId().equals(ctx.localNodeId())) { * @param req Request to complete future for. * @param err Error to be passed to futures. */ - public void completeStartFuture(DynamicCacheChangeRequest req, @Nullable Throwable err) { + void completeStartFuture(DynamicCacheChangeRequest req, @Nullable Throwable err) { DynamicCacheStartFuture fut = (DynamicCacheStartFuture)pendingFuts.get(maskNull(req.cacheName())); assert req.deploymentId() != null; @@ -2648,6 +2645,8 @@ private boolean onCacheChangeRequested(DynamicCacheChangeBatch batch, boolean incMinorTopVer = false; + ExchangeActions exchangeActions = new ExchangeActions(); + for (DynamicCacheChangeRequest req : batch.requests()) { initReceivedCacheConfiguration(req); @@ -2729,6 +2728,8 @@ private boolean onCacheChangeRequested(DynamicCacheChangeBatch batch, req.nearCacheConfiguration() != null); needExchange = true; + + exchangeActions.addCacheToStart(req, startDesc); } } else { @@ -2755,15 +2756,19 @@ private boolean onCacheChangeRequested(DynamicCacheChangeBatch batch, needExchange = clientReq && ctx.discovery().addClientNode(req.cacheName(), req.initiatingNodeId(), req.nearCacheConfiguration() != null); - - if (needExchange) - req.clientStartOnly(true); } } + + if (needExchange) { + req.clientStartOnly(true); + + exchangeActions.addCacheToStart(req, desc); + } } if (!needExchange && desc != null) req.cacheFutureTopologyVersion(desc.startTopologyVersion()); + } else { assert req.stop() ^ req.close() : req; @@ -2777,11 +2782,16 @@ private boolean onCacheChangeRequested(DynamicCacheChangeBatch batch, ctx.discovery().removeCacheFilter(req.cacheName()); needExchange = true; + + exchangeActions.addCacheToStop(req, desc); } else { assert req.close() : req; needExchange = ctx.discovery().onClientCacheClose(req.cacheName(), req.initiatingNodeId()); + + if (needExchange) + exchangeActions.addCacheToClose(req, desc); } } } @@ -2791,6 +2801,12 @@ private boolean onCacheChangeRequested(DynamicCacheChangeBatch batch, incMinorTopVer |= needExchange; } + if (incMinorTopVer) { + assert !exchangeActions.empty() : exchangeActions; + + batch.exchangeActions(exchangeActions); + } + return incMinorTopVer; } 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 4377c6e4aa6a8..79970f4f29ede 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 @@ -26,9 +26,9 @@ 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.IgniteNeedReconnectException; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.managers.discovery.DiscoCache; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; @@ -37,7 +37,6 @@ 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; import org.jetbrains.annotations.Nullable; @@ -73,20 +72,20 @@ public class GridDhtAssignmentFetchFuture extends GridFutureAdapter(CU.cacheId(cacheName), topVer); + this.key = new T2<>(cacheId, topVer); - Collection availableNodes = discoCache.cacheAffinityNodes(CU.cacheId(cacheName)); + Collection availableNodes = discoCache.cacheAffinityNodes(cacheId); LinkedList tmp = new LinkedList<>(); 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 5bbcfc8d9ae80..0de932780a1ec 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 @@ -54,6 +54,7 @@ import org.apache.ignite.internal.processors.cache.DynamicCacheChangeFailureMessage; import org.apache.ignite.internal.processors.cache.DynamicCacheChangeRequest; import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor; +import org.apache.ignite.internal.processors.cache.ExchangeActions; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; @@ -160,8 +161,7 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter lastVer = new AtomicReference<>(); /** - * Messages received on non-coordinator are stored in case if this node - * becomes coordinator. + * Messages received on non-coordinator are stored in case if this node becomes coordinator. */ private final Map singleMsgs = new ConcurrentHashMap8<>(); @@ -179,9 +179,6 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter reqs; - /** */ private CacheAffinityChangeMessage affChangeMsg; @@ -212,9 +209,11 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter forcedRebFut; + /** Cache change requests. */ + private ExchangeActions exchActions; + /** - * Dummy future created to trigger reassignments if partition - * topology changed while preloading. + * Dummy future created to trigger reassignments if partition topology changed while preloading. * * @param cctx Cache context. * @param reassign Dummy reassign flag. @@ -239,8 +238,7 @@ public GridDhtPartitionsExchangeFuture( } /** - * Force preload future created to trigger reassignments if partition - * topology changed while preloading. + * Force preload future created to trigger reassignments if partition topology changed while preloading. * * @param cctx Cache context. * @param discoEvt Discovery event. @@ -266,14 +264,14 @@ public GridDhtPartitionsExchangeFuture(GridCacheSharedContext cctx, DiscoveryEve * @param cctx Cache context. * @param busyLock Busy lock. * @param exchId Exchange ID. - * @param reqs Cache change requests. + * @param exchActions Exchange actions. * @param affChangeMsg Affinity change message. */ public GridDhtPartitionsExchangeFuture( GridCacheSharedContext cctx, ReadWriteLock busyLock, GridDhtPartitionExchangeId exchId, - Collection reqs, + ExchangeActions exchActions, CacheAffinityChangeMessage affChangeMsg ) { assert busyLock != null; @@ -286,7 +284,7 @@ public GridDhtPartitionsExchangeFuture( this.cctx = cctx; this.busyLock = busyLock; this.exchId = exchId; - this.reqs = reqs; + this.exchActions = exchActions; this.affChangeMsg = affChangeMsg; log = cctx.logger(getClass()); @@ -297,13 +295,6 @@ public GridDhtPartitionsExchangeFuture( log.debug("Creating exchange future [localNode=" + cctx.localNodeId() + ", fut=" + this + ']'); } - /** - * @param reqs Cache change requests. - */ - public void cacheChangeRequests(Collection reqs) { - this.reqs = reqs; - } - /** * @param affChangeMsg Affinity change message. */ @@ -377,16 +368,7 @@ public boolean isCacheAdded(int cacheId, AffinityTopologyVersion topVer) { * @return {@code True} if non-client cache was added during this exchange. */ public boolean cacheStarted(int cacheId) { - if (!F.isEmpty(reqs)) { - for (DynamicCacheChangeRequest req : reqs) { - if (req.start() && !req.clientStartOnly()) { - if (CU.cacheId(req.cacheName()) == cacheId) - return true; - } - } - } - - return false; + return exchActions != null && exchActions.cacheStarted(cacheId); } /** @@ -486,8 +468,11 @@ public void init() throws IgniteInterruptedCheckedException { Collection receivedCaches; if (discoEvt.type() == EVT_DISCOVERY_CUSTOM_EVT) { - if (!F.isEmpty(reqs)) + if (exchActions != null) { + assert !exchActions.empty(); + exchange = onCacheChangeRequest(crdNode); + } else { assert affChangeMsg != null : this; @@ -603,7 +588,7 @@ public void init() throws IgniteInterruptedCheckedException { } /** - * @throws IgniteCheckedException If failed. + * @throws IgniteCheckedException If failed. */ private void initTopologies() throws IgniteCheckedException { if (crd != null) { @@ -648,29 +633,28 @@ private void updateTopologies(boolean crd) throws IgniteCheckedException { /** * @param crd Coordinator flag. - * @throws IgniteCheckedException If failed. * @return Exchange type. + * @throws IgniteCheckedException If failed. */ private ExchangeType onCacheChangeRequest(boolean crd) throws IgniteCheckedException { - assert !F.isEmpty(reqs) : this; + assert exchActions != null; - boolean clientOnly = cctx.affinity().onCacheChangeRequest(this, crd, reqs); + boolean clientOnly = cctx.affinity().onCacheChangeRequest(this, crd, exchActions); if (clientOnly) { boolean clientCacheStarted = false; - for (DynamicCacheChangeRequest req : reqs) { - if (req.start() && req.clientStartOnly() && req.initiatingNodeId().equals(cctx.localNodeId())) { + for (ExchangeActions.CacheActionData action :exchActions.cacheStartRequests()){ + DynamicCacheChangeRequest req = action.request(); + + if (req.clientStartOnly() && req.initiatingNodeId().equals(cctx.localNodeId())) { clientCacheStarted = true; break; } } - if (clientCacheStarted) - return ExchangeType.CLIENT; - else - return ExchangeType.NONE; + return clientCacheStarted ? ExchangeType.CLIENT : ExchangeType.NONE; } else return cctx.kernalContext().clientNode() ? ExchangeType.CLIENT : ExchangeType.ALL; @@ -678,8 +662,8 @@ private ExchangeType onCacheChangeRequest(boolean crd) throws IgniteCheckedExcep /** * @param crd Coordinator flag. - * @throws IgniteCheckedException If failed. * @return Exchange type. + * @throws IgniteCheckedException If failed. */ private ExchangeType onAffinityChangeRequest(boolean crd) throws IgniteCheckedException { assert affChangeMsg != null : this; @@ -694,8 +678,8 @@ private ExchangeType onAffinityChangeRequest(boolean crd) throws IgniteCheckedEx /** * @param crd Coordinator flag. - * @throws IgniteCheckedException If failed. * @return Exchange type. + * @throws IgniteCheckedException If failed. */ private ExchangeType onClientNodeEvent(boolean crd) throws IgniteCheckedException { assert CU.clientNode(discoEvt.eventNode()) : this; @@ -718,8 +702,8 @@ private ExchangeType onClientNodeEvent(boolean crd) throws IgniteCheckedExceptio /** * @param crd Coordinator flag. - * @throws IgniteCheckedException If failed. * @return Exchange type. + * @throws IgniteCheckedException If failed. */ private ExchangeType onServerNodeEvent(boolean crd) throws IgniteCheckedException { assert !CU.clientNode(discoEvt.eventNode()) : this; @@ -1003,19 +987,7 @@ private void dumpPendingObjects() { * @return {@code True} if cache is stopping by this exchange. */ public boolean stopping(int cacheId) { - boolean stopping = false; - - if (!F.isEmpty(reqs)) { - for (DynamicCacheChangeRequest req : reqs) { - if (cacheId == CU.cacheId(req.cacheName())) { - stopping = req.stop(); - - break; - } - } - } - - return stopping; + return exchActions != null && exchActions.cacheStopped(cacheId); } /** @@ -1150,7 +1122,10 @@ private void sendPartitions(ClusterNode oldestNode) { cctx.exchange().onExchangeDone(this, err); - cctx.cache().onExchangeDone(exchId.topologyVersion(), reqs, err); + cctx.cache().onExchangeDone(exchId.topologyVersion(), exchActions, err); + + if (exchActions != null) + exchActions.completeRequestFutures(cctx, err); if (super.onDone(res, err) && realExchange) { if (log.isDebugEnabled()) @@ -1164,7 +1139,7 @@ private void sendPartitions(ClusterNode oldestNode) { cacheCtx.config().getAffinity().removeNode(exchId.nodeId()); } - reqs = null; + exchActions = null; if (discoEvt instanceof DiscoveryCustomEvent) ((DiscoveryCustomEvent)discoEvt).customMessage(null); @@ -1358,8 +1333,8 @@ private void onAffinityInitialized(IgniteInternalFuture glob * @return {@code true} if the given {@code discoEvt} supports the rollback procedure. */ private boolean isRollbackSupported() { - boolean rollbackSupported = false; - if (!discoCache.checkAttribute(ATTR_EXCHANGE_ROLLBACK_SUPPORTED, Boolean.TRUE)) return false; // Currently the rollback process is supported for dynamically started caches. - if (discoEvt.type() == EVT_DISCOVERY_CUSTOM_EVT && !F.isEmpty(reqs)) { - for (DynamicCacheChangeRequest req : reqs) { - if (req.start()) { - rollbackSupported = true; - - break; - } - } - } - - return rollbackSupported; + return discoEvt.type() == EVT_DISCOVERY_CUSTOM_EVT && exchActions != null && exchActions.hasStart(); } /** - * Tries to revert all the changes that were done during initialization phase - * in case of the given {@code discoEvt} supports the rollback procedure. + * Tries to revert all the changes that were done during initialization phase in case of the given {@code discoEvt} + * supports the rollback procedure. */ private void rollbackExchange() { - if (discoEvt.type() == EVT_DISCOVERY_CUSTOM_EVT && !F.isEmpty(reqs)) { - for (DynamicCacheChangeRequest req : reqs) { - if (req.start()) { - DynamicCacheChangeRequest stopReq = - new DynamicCacheChangeRequest(req.cacheName(), cctx.localNodeId()); + if (discoEvt.type() == EVT_DISCOVERY_CUSTOM_EVT && exchActions != null) { + for (ExchangeActions.CacheActionData action : exchActions.cacheStartRequests()) { + DynamicCacheChangeRequest req = action.request(); - stopReq.stop(true); - stopReq.deploymentId(req.deploymentId()); + DynamicCacheChangeRequest stopReq = + new DynamicCacheChangeRequest(req.cacheName(), cctx.localNodeId()); - // Cleanup GridCacheProcessor. - cctx.cache().forceCloseCache(stopReq); + stopReq.stop(true); + stopReq.deploymentId(req.deploymentId()); - // Cleanup CacheAffinitySharedManager. - cctx.affinity().forceCloseCache(this, crd.isLocal(), Collections.singletonList(stopReq)); - } + // Cleanup GridCacheProcessor. + cctx.cache().forceCloseCache(stopReq); + + // Cleanup CacheAffinitySharedManager. + cctx.affinity().forceCloseCache(this, crd.isLocal(), Collections.singletonList(stopReq)); } } } @@ -1452,10 +1415,11 @@ private void onAllReceived() { if (!F.isEmpty(exchangeGlobalExceptions) && isRollbackSupported()) { IgniteCheckedException err = createExchangeException(exchangeGlobalExceptions); - List cacheNames = new ArrayList<>(reqs.size()); + List cacheNames = new ArrayList<>(); - for (DynamicCacheChangeRequest req : reqs) - cacheNames.add(req.cacheName()); + for (ExchangeActions.CacheActionData action : F.concat(false, + exchActions.cacheStartRequests(), exchActions.cacheStopRequests())) + cacheNames.add(action.request().cacheName()); DynamicCacheChangeFailureMessage msg = new DynamicCacheChangeFailureMessage( cctx.localNode(), exchId, err, cacheNames); @@ -1621,7 +1585,7 @@ private void processMessage(ClusterNode node, GridDhtPartitionsFullMessage msg) if (!crd.equals(node)) { if (log.isDebugEnabled()) log.debug("Received full partition map from unexpected node [oldest=" + crd.id() + - ", nodeId=" + node.id() + ']'); + ", nodeId=" + node.id() + ']'); if (node.order() > crd.order()) fullMsgs.put(node, msg); @@ -1950,6 +1914,16 @@ public boolean reconnectOnError(Throwable e) { return exchId.compareTo(fut.exchId); } + /** + * @param exchActions New cache change requests. + */ + public void exchangeActions(ExchangeActions exchActions) { + assert exchActions == null || !exchActions.empty() : exchActions; + assert evtLatch != null && evtLatch.getCount() == 1L : this; + + this.exchActions = exchActions; + } + /** {@inheritDoc} */ @Override public boolean equals(Object o) { if (this == o) From ec6c1194f941c7818656a777d9fe99bb70db58d6 Mon Sep 17 00:00:00 2001 From: Slava Koptilin Date: Fri, 15 Sep 2017 14:54:07 +0300 Subject: [PATCH 298/357] ignite-3495: metrics for atomic caches --- .../processors/cache/GridCacheAdapter.java | 258 ++++------ .../distributed/dht/GridDhtCacheAdapter.java | 26 + .../dht/GridPartitionedGetFuture.java | 21 +- .../dht/GridPartitionedSingleGetFuture.java | 8 + .../dht/atomic/GridDhtAtomicCache.java | 33 +- .../GridNearAtomicAbstractUpdateFuture.java | 25 + .../atomic/GridNearAtomicUpdateFuture.java | 25 + .../dht/colocated/GridDhtColocatedCache.java | 19 +- .../distributed/near/GridNearGetFuture.java | 8 + .../cache/local/GridLocalCache.java | 42 ++ .../local/atomic/GridLocalAtomicCache.java | 78 ++- ...acheAtomicClientServerMetricsSelfTest.java | 452 ++++++++++++++++++ .../IgniteCacheMetricsSelfTestSuite.java | 4 +- 13 files changed, 816 insertions(+), 183 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicClientServerMetricsSelfTest.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 ff1850dbcfe39..74415d2733f43 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 @@ -1406,10 +1406,6 @@ public final V getTopologySafe(K key) throws IgniteCheckedException { @Nullable @Override public V get(K key) throws IgniteCheckedException { A.notNull(key, "key"); - boolean statsEnabled = ctx.config().isStatisticsEnabled(); - - long start = statsEnabled ? System.nanoTime() : 0L; - boolean keepBinary = ctx.keepBinary(); if (keepBinary) @@ -1423,9 +1419,6 @@ public final V getTopologySafe(K key) throws IgniteCheckedException { val = (V)ctx.config().getInterceptor().onGet(key, val); } - if (statsEnabled) - metrics0().addGetTimeNanos(System.nanoTime() - start); - return val; } @@ -1433,10 +1426,6 @@ public final V getTopologySafe(K key) throws IgniteCheckedException { @Nullable @Override public CacheEntry getEntry(K key) throws IgniteCheckedException { A.notNull(key, "key"); - boolean statsEnabled = ctx.config().isStatisticsEnabled(); - - long start = statsEnabled ? System.nanoTime() : 0L; - boolean keepBinary = ctx.keepBinary(); if (keepBinary) @@ -1459,9 +1448,6 @@ public final V getTopologySafe(K key) throws IgniteCheckedException { val = (val0 != null) ? new CacheEntryImplEx<>(key, val0, t != null ? t.version() : null) : null; } - if (statsEnabled) - metrics0().addGetTimeNanos(System.nanoTime() - start); - return val; } @@ -1469,10 +1455,6 @@ public final V getTopologySafe(K key) throws IgniteCheckedException { @Override public IgniteInternalFuture getAsync(final K key) { A.notNull(key, "key"); - final boolean statsEnabled = ctx.config().isStatisticsEnabled(); - - final long start = statsEnabled ? System.nanoTime() : 0L; - final boolean keepBinary = ctx.keepBinary(); final K key0 = keepBinary ? (K)ctx.toCacheKeyObject(key) : key; @@ -1488,9 +1470,6 @@ public final V getTopologySafe(K key) throws IgniteCheckedException { } }); - if (statsEnabled) - fut.listen(new UpdateGetTimeStatClosure(metrics0(), start)); - return fut; } @@ -1498,10 +1477,6 @@ public final V getTopologySafe(K key) throws IgniteCheckedException { @Override public IgniteInternalFuture> getEntryAsync(final K key) { A.notNull(key, "key"); - final boolean statsEnabled = ctx.config().isStatisticsEnabled(); - - final long start = statsEnabled ? System.nanoTime() : 0L; - final boolean keepBinary = ctx.keepBinary(); final K key0 = keepBinary ? (K)ctx.toCacheKeyObject(key) : key; @@ -1535,9 +1510,6 @@ public final V getTopologySafe(K key) throws IgniteCheckedException { } }); - if (statsEnabled) - fut.listen(new UpdateGetTimeStatClosure(metrics0(), start)); - return fr; } @@ -1545,18 +1517,11 @@ public final V getTopologySafe(K key) 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 = getAll0(keys, !ctx.keepBinary(), false); if (ctx.config().getInterceptor() != null) map = interceptGet(keys, map); - if (statsEnabled) - metrics0().addGetTimeNanos(System.nanoTime() - start); - return map; } @@ -1565,22 +1530,16 @@ public final V getTopologySafe(K key) throws IgniteCheckedException { throws IgniteCheckedException { A.notNull(keys, "keys"); - boolean statsEnabled = ctx.config().isStatisticsEnabled(); - - long start = statsEnabled ? System.nanoTime() : 0L; - Map map = (Map)getAll0(keys, !ctx.keepBinary(), true); Collection> res = new HashSet<>(); if (ctx.config().getInterceptor() != null) res = interceptGetEntries(keys, map); - else + else { for (Map.Entry e : map.entrySet()) res.add(new CacheEntryImplEx<>(e.getKey(), (V)e.getValue().value(), e.getValue().version())); - - if (statsEnabled) - metrics0().addGetTimeNanos(System.nanoTime() - start); + } return res; } @@ -1589,10 +1548,6 @@ public final V getTopologySafe(K key) throws IgniteCheckedException { @Override public IgniteInternalFuture> getAllAsync(@Nullable final Collection keys) { A.notNull(keys, "keys"); - final boolean statsEnabled = ctx.config().isStatisticsEnabled(); - - final long start = statsEnabled ? System.nanoTime() : 0L; - IgniteInternalFuture> fut = getAllAsync(keys, !ctx.keepBinary(), false); if (ctx.config().getInterceptor() != null) @@ -1602,9 +1557,6 @@ public final V getTopologySafe(K key) throws IgniteCheckedException { } }); - if (statsEnabled) - fut.listen(new UpdateGetTimeStatClosure>(metrics0(), start)); - return fut; } @@ -1613,10 +1565,6 @@ public final V getTopologySafe(K key) throws IgniteCheckedException { @Nullable final Collection keys) { A.notNull(keys, "keys"); - final boolean statsEnabled = ctx.config().isStatisticsEnabled(); - - final long start = statsEnabled ? System.nanoTime() : 0L; - IgniteInternalFuture> fut = (IgniteInternalFuture>) ((IgniteInternalFuture)getAllAsync(keys, !ctx.keepBinary(), true)); @@ -1641,9 +1589,6 @@ public final V getTopologySafe(K key) throws IgniteCheckedException { } }); - if (statsEnabled) - fut.listen(new UpdateGetTimeStatClosure>(metrics0(), start)); - return rf; } @@ -2212,21 +2157,12 @@ private void clearReservationsIfNeeded( */ @Nullable public V getAndPut(final K key, final V val, @Nullable final CacheEntryPredicate filter) throws IgniteCheckedException { - boolean statsEnabled = ctx.config().isStatisticsEnabled(); - - long start = statsEnabled ? System.nanoTime() : 0L; - A.notNull(key, "key", val, "val"); if (keyCheck) validateCacheKey(key); - V prevVal = getAndPut0(key, val, filter); - - if (statsEnabled) - metrics0().addPutAndGetTimeNanos(System.nanoTime() - start); - - return prevVal; + return getAndPut0(key, val, filter); } /** @@ -2261,21 +2197,12 @@ protected V getAndPut0(final K key, final V val, @Nullable final CacheEntryPredi * @return Put operation future. */ 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) - fut.listen(new UpdatePutAndGetTimeStatClosure(metrics0(), start)); - - return fut; + return getAndPutAsync0(key, val, filter); } /** @@ -2288,7 +2215,11 @@ public IgniteInternalFuture getAndPutAsync0(final K key, final V val, @Nullable final CacheEntryPredicate filter) { - return asyncOp(new AsyncOp() { + boolean statsEnabled = ctx.config().isStatisticsEnabled(); + + long start = (statsEnabled)? System.nanoTime() : 0L; + + IgniteInternalFuture fut = asyncOp(new AsyncOp() { @Override public IgniteInternalFuture op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) { return tx.putAsync(ctx, readyTopVer, key, val, true, filter) .chain((IgniteClosure, V>)RET2VAL); @@ -2298,6 +2229,11 @@ public IgniteInternalFuture getAndPutAsync0(final K key, return "putAsync [key=" + key + ", val=" + val + ", filter=" + filter + ']'; } }); + + if (statsEnabled) + fut.listen(new UpdatePutAndGetTimeStatClosure(metrics0(), start)); + + return fut; } /** {@inheritDoc} */ @@ -2316,21 +2252,12 @@ public IgniteInternalFuture getAndPutAsync0(final K key, */ public boolean put(final K key, final V val, final CacheEntryPredicate filter) 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 = put0(key, val, filter); - - if (statsEnabled && stored) - metrics0().addPutTimeNanos(System.nanoTime() - start); - - return stored; + return put0(key, val, filter); } /** @@ -2344,6 +2271,10 @@ public boolean put(final K key, final V val, final CacheEntryPredicate filter) */ protected boolean put0(final K key, final V val, final CacheEntryPredicate filter) throws IgniteCheckedException { + boolean statsEnabled = ctx.config().isStatisticsEnabled(); + + long start = statsEnabled ? System.nanoTime() : 0L; + 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(); @@ -2356,6 +2287,9 @@ protected boolean put0(final K key, final V val, final CacheEntryPredicate filte assert res != null; + if (statsEnabled && Boolean.TRUE.equals(res)) + metrics0().addPutTimeNanos(System.nanoTime() - start); + return res; } @@ -2647,16 +2581,7 @@ public final IgniteInternalFuture putAsync(K key, V val, @Nullable Cach if (keyCheck) validateCacheKey(key); - final boolean statsEnabled = ctx.config().isStatisticsEnabled(); - - final long start = statsEnabled ? System.nanoTime() : 0L; - - IgniteInternalFuture fut = putAsync0(key, val, filter); - - if (statsEnabled) - fut.listen(new UpdatePutTimeStatClosure(metrics0(), start)); - - return fut; + return putAsync0(key, val, filter); } /** @@ -2665,9 +2590,12 @@ public final IgniteInternalFuture putAsync(K key, V val, @Nullable Cach * @param filter Optional filter. * @return Putx operation future. */ - public IgniteInternalFuture putAsync0(final K key, final V val, - @Nullable final CacheEntryPredicate filter) { - return asyncOp(new AsyncOp() { + public IgniteInternalFuture putAsync0(final K key, final V val, @Nullable final CacheEntryPredicate filter) { + boolean statsEnabled = ctx.config().isStatisticsEnabled(); + + long start = statsEnabled ? System.nanoTime() : 0L; + + IgniteInternalFuture fut = asyncOp(new AsyncOp() { @Override public IgniteInternalFuture op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) { return tx.putAsync(ctx, readyTopVer, @@ -2685,6 +2613,11 @@ public IgniteInternalFuture putAsync0(final K key, final V val, "filter", filter, false); } }); + + if (statsEnabled) + fut.listen(new UpdatePutTimeStatClosure(metrics0(), start)); + + return fut; } /** {@inheritDoc} */ @@ -2754,17 +2687,10 @@ public IgniteInternalFuture putAsync0(final K key, final V val, if (F.isEmpty(m)) return; - boolean statsEnabled = ctx.config().isStatisticsEnabled(); - - long start = statsEnabled ? System.nanoTime() : 0L; - if (keyCheck) validateCacheKeys(m.keySet()); putAll0(m); - - if (statsEnabled) - metrics0().addPutTimeNanos(System.nanoTime() - start); } /** @@ -2772,6 +2698,10 @@ public IgniteInternalFuture putAsync0(final K key, final V val, * @throws IgniteCheckedException If failed. */ protected void putAll0(final Map m) throws IgniteCheckedException { + boolean statsEnabled = ctx.config().isStatisticsEnabled(); + + long start = (statsEnabled)? System.nanoTime() : 0L; + syncOp(new SyncInOp(m.size() == 1) { @Override public void inOp(IgniteTxLocalAdapter tx) throws IgniteCheckedException { tx.putAllAsync(ctx, null, m, false).get(); @@ -2781,6 +2711,9 @@ protected void putAll0(final Map m) throws IgniteCheck return "putAll [map=" + m + ']'; } }); + + if (statsEnabled) + metrics0().addPutTimeNanos(System.nanoTime() - start); } /** {@inheritDoc} */ @@ -2815,21 +2748,12 @@ protected IgniteInternalFuture putAllAsync0(final Map getAndRemoveAsync(final K key) { - final boolean statsEnabled = ctx.config().isStatisticsEnabled(); - - final long start = statsEnabled ? System.nanoTime() : 0L; - A.notNull(key, "key"); if (keyCheck) validateCacheKey(key); - IgniteInternalFuture fut = getAndRemoveAsync0(key); - - if (statsEnabled) - fut.listen(new UpdateRemoveTimeStatClosure(metrics0(), start)); - - return fut; + return getAndRemoveAsync0(key); } /** @@ -2890,7 +2805,11 @@ protected V getAndRemove0(final K key) throws IgniteCheckedException { * @return Future. */ protected IgniteInternalFuture getAndRemoveAsync0(final K key) { - return asyncOp(new AsyncOp() { + boolean statsEnabled = ctx.config().isStatisticsEnabled(); + + long start = (statsEnabled)? System.nanoTime() : 0L; + + IgniteInternalFuture fut = asyncOp(new AsyncOp() { @Override public IgniteInternalFuture op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) { // TODO should we invoke interceptor here? return tx.removeAllAsync(ctx, @@ -2905,6 +2824,11 @@ protected IgniteInternalFuture getAndRemoveAsync0(final K key) { return "removeAsync [key=" + key + ']'; } }); + + if (statsEnabled) + fut.listen(new UpdateRemoveTimeStatClosure(metrics0(), start)); + + return fut; } /** {@inheritDoc} */ @@ -2925,10 +2849,6 @@ protected IgniteInternalFuture getAndRemoveAsync0(final K key) { /** {@inheritDoc} */ @Override public void removeAll(final Collection keys) throws IgniteCheckedException { - boolean statsEnabled = ctx.config().isStatisticsEnabled(); - - long start = statsEnabled ? System.nanoTime() : 0L; - A.notNull(keys, "keys"); if (F.isEmpty(keys)) @@ -2938,9 +2858,6 @@ protected IgniteInternalFuture getAndRemoveAsync0(final K key) { validateCacheKeys(keys); removeAll0(keys); - - if (statsEnabled) - metrics0().addRemoveTimeNanos(System.nanoTime() - start); } /** @@ -2948,6 +2865,10 @@ protected IgniteInternalFuture getAndRemoveAsync0(final K key) { * @throws IgniteCheckedException If failed. */ protected void removeAll0(final Collection keys) throws IgniteCheckedException { + boolean statsEnabled = ctx.config().isStatisticsEnabled(); + + long start = (statsEnabled)? System.nanoTime() : 0L; + syncOp(new SyncInOp(keys.size() == 1) { @Override public void inOp(IgniteTxLocalAdapter tx) throws IgniteCheckedException { tx.removeAllAsync(ctx, @@ -2962,6 +2883,9 @@ protected void removeAll0(final Collection keys) throws IgniteCheck return "removeAll [keys=" + keys + ']'; } }); + + if (statsEnabled) + metrics0().addRemoveTimeNanos(System.nanoTime() - start); } /** {@inheritDoc} */ @@ -2969,19 +2893,10 @@ protected void removeAll0(final Collection keys) throws IgniteCheck if (F.isEmpty(keys)) return new GridFinishedFuture(); - final boolean statsEnabled = ctx.config().isStatisticsEnabled(); - - final long start = statsEnabled ? System.nanoTime() : 0L; - if (keyCheck) validateCacheKeys(keys); - IgniteInternalFuture fut = removeAllAsync0(keys); - - if (statsEnabled) - fut.listen(new UpdateRemoveTimeStatClosure<>(metrics0(), start)); - - return fut; + return removeAllAsync0(keys); } /** @@ -2989,7 +2904,11 @@ protected void removeAll0(final Collection keys) throws IgniteCheck * @return Future. */ protected IgniteInternalFuture removeAllAsync0(final Collection keys) { - return asyncOp(new AsyncOp(keys) { + boolean statsEnabled = ctx.config().isStatisticsEnabled(); + + long start = (statsEnabled)? System.nanoTime() : 0L; + + IgniteInternalFuture fut = asyncOp(new AsyncOp(keys) { @Override public IgniteInternalFuture op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) { return tx.removeAllAsync(ctx, readyTopVer, @@ -3003,6 +2922,11 @@ protected IgniteInternalFuture removeAllAsync0(final Collection(metrics0(), start)); + + return fut; } /** {@inheritDoc} */ @@ -3017,21 +2941,12 @@ protected IgniteInternalFuture removeAllAsync0(final Collection(true) { @Override public Boolean op(IgniteTxLocalAdapter tx) throws IgniteCheckedException { return tx.removeAllAsync(ctx, @@ -3058,6 +2977,9 @@ protected boolean remove0(final K key, final CacheEntryPredicate filter) throws assert res != null; + if (statsEnabled && Boolean.TRUE.equals(res)) + metrics0().addRemoveTimeNanos(System.nanoTime() - start); + return res; } @@ -3074,21 +2996,12 @@ protected boolean remove0(final K key, final CacheEntryPredicate filter) throws * @return Putx operation future. */ public IgniteInternalFuture removeAsync(final K key, @Nullable final CacheEntryPredicate filter) { - final boolean statsEnabled = ctx.config().isStatisticsEnabled(); - - final long start = statsEnabled ? System.nanoTime() : 0L; - A.notNull(key, "key"); if (keyCheck) validateCacheKey(key); - IgniteInternalFuture fut = removeAsync0(key, filter); - - if (statsEnabled) - fut.listen(new UpdateRemoveTimeStatClosure(metrics0(), start)); - - return fut; + return removeAsync0(key, filter); } /** @@ -3097,7 +3010,11 @@ public IgniteInternalFuture removeAsync(final K key, @Nullable final Ca * @return Future. */ protected IgniteInternalFuture removeAsync0(final K key, @Nullable final CacheEntryPredicate filter) { - return asyncOp(new AsyncOp() { + boolean statsEnabled = ctx.config().isStatisticsEnabled(); + + long start = (statsEnabled)? System.nanoTime() : 0L; + + IgniteInternalFuture fut = asyncOp(new AsyncOp() { @Override public IgniteInternalFuture op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) { return tx.removeAllAsync(ctx, readyTopVer, @@ -3112,6 +3029,11 @@ protected IgniteInternalFuture removeAsync0(final K key, @Nullable fina return "removeAsync [key=" + key + ", filter=" + filter + ']'; } }); + + if (statsEnabled) + fut.listen(new UpdateRemoveTimeStatClosure(metrics0(), start)); + + return fut; } /** {@inheritDoc} */ 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 417eb3536536f..ac73907bdf175 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 @@ -761,6 +761,10 @@ private IgniteInternalFuture getDhtSingleAsync( protected void processNearSingleGetRequest(final UUID nodeId, final GridNearSingleGetRequest req) { assert ctx.affinityNode(); + boolean statsEnabled = ctx.config().isStatisticsEnabled(); + + final long start = statsEnabled ? System.nanoTime() : 0L; + final CacheExpiryPolicy expiryPlc = CacheExpiryPolicy.fromRemote(req.createTtl(), req.accessTtl()); IgniteInternalFuture fut = @@ -851,6 +855,15 @@ else if (req.needVersion()) sendTtlUpdateRequest(expiryPlc); } }); + + if (statsEnabled) { + fut.listen(new CI1>() { + @Override public void apply(IgniteInternalFuture f) { + if (!req.skipValues() && f.error() == null) + metrics0().addGetTimeNanos(System.nanoTime() - start); + } + }); + } } /** @@ -861,6 +874,10 @@ protected void processNearGetRequest(final UUID nodeId, final GridNearGetRequest assert ctx.affinityNode(); assert !req.reload() : req; + boolean statsEnabled = ctx.config().isStatisticsEnabled(); + + final long start = statsEnabled ? System.nanoTime() : 0L; + final CacheExpiryPolicy expiryPlc = CacheExpiryPolicy.fromRemote(req.createTtl(), req.accessTtl()); IgniteInternalFuture> fut = @@ -915,6 +932,15 @@ protected void processNearGetRequest(final UUID nodeId, final GridNearGetRequest sendTtlUpdateRequest(expiryPlc); } }); + + if (statsEnabled) { + fut.listen(new CI1>>() { + @Override public void apply(IgniteInternalFuture> f) { + if (!req.skipValues() && f.error() == null) + metrics0().addGetTimeNanos(System.nanoTime() - start); + } + }); + } } /** 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 519239aecb532..9b7c8dc8de284 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 @@ -53,7 +53,6 @@ import org.apache.ignite.internal.util.typedef.CIX1; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.P1; -import org.apache.ignite.internal.util.typedef.T2; 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; @@ -281,6 +280,10 @@ private void map( // If this is the primary or backup node for the keys. if (n.isLocal()) { + final boolean statsEnabled = cctx.config().isStatisticsEnabled(); + + final long start = (statsEnabled)? System.nanoTime() : 0L; + final GridDhtFuture> fut = cache().getDhtAsync(n.id(), -1, @@ -316,7 +319,12 @@ private void map( add(fut.chain(new C1>, Map>() { @Override public Map apply(IgniteInternalFuture> fut) { try { - return createResultMap(fut.get()); + Map result = createResultMap(fut.get()); + + if (!skipVals && statsEnabled && fut.error() == null) + cctx.cache().metrics0().addGetTimeNanos(System.nanoTime() - start); + + return result; } catch (Exception e) { U.error(log, "Failed to get values from dht cache [fut=" + fut + "]", e); @@ -435,6 +443,10 @@ private boolean map( private boolean localGet(KeyCacheObject key, int part, Map locVals) { assert cctx.affinityNode() : this; + boolean statsEnabled = cctx.config().isStatisticsEnabled(); + + long start = (statsEnabled)? System.nanoTime() : 0L; + GridDhtCacheAdapter cache = cache(); while (true) { @@ -516,9 +528,12 @@ private boolean localGet(KeyCacheObject key, int part, Map locVals) { // Entry not found, do not continue search if topology did not change and there is no store. if (!cctx.readThroughConfigured() && (topStable || partitionOwned(part))) { - if (!skipVals && cctx.config().isStatisticsEnabled()) + if (!skipVals && statsEnabled) { cache.metrics0().onRead(false); + cache.metrics0().addGetTimeNanos(System.nanoTime() - start); + } + return true; } 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 a3f6b72376e8f..25e9f0042672a 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 @@ -122,6 +122,9 @@ public class GridPartitionedSingleGetFuture extends GridFutureAdapter im /** */ private final boolean keepCacheObjects; + /** Future start time in nanoseconds. */ + private final long startTimeNanos; + /** */ @GridToStringInclude private ClusterNode node; @@ -182,6 +185,8 @@ public GridPartitionedSingleGetFuture( futId = IgniteUuid.randomUuid(); + startTimeNanos = cctx.config().isStatisticsEnabled() ? System.nanoTime() : 0L; + if (log == null) log = U.logger(cctx.kernalContext(), logRef, GridPartitionedSingleGetFuture.class); } @@ -735,6 +740,9 @@ private void remap(final AffinityTopologyVersion topVer) { cctx.dht().sendTtlUpdateRequest(expiryPlc); + if (cctx.config().isStatisticsEnabled() && !trackable && !skipVals && err == null) + cctx.cache().metrics0().addGetTimeNanos(System.nanoTime() - startTimeNanos); + return true; } 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 ffe68ed2cfee7..527b4a7d80926 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 @@ -1573,6 +1573,10 @@ private IgniteInternalFuture> getAllAsync0(@Nullable Collection locVals = U.newHashMap(keys.size()); boolean success = true; @@ -1599,7 +1603,7 @@ private IgniteInternalFuture> getAllAsync0(@Nullable Collection> getAllAsync0(@Nullable Collection(locVals); } } @@ -3203,9 +3210,31 @@ private void processNearAtomicUpdateRequest(UUID nodeId, GridNearAtomicAbstractU ", node=" + nodeId + ']'); } + long start = (ctx.config().isStatisticsEnabled())? System.nanoTime() : 0L; + req.nodeId(ctx.localNodeId()); updateAllAsyncInternal(nodeId, req, updateReplyClos); + + if (ctx.config().isStatisticsEnabled()) { + boolean retVal = req.returnValue(); + + long duration = System.nanoTime() - start; + + if (req.operation() == UPDATE) { + if (retVal) + metrics0().addPutAndGetTimeNanos(duration); + else { + metrics0().addPutTimeNanos(duration); + } + } + else if (req.operation() == DELETE) { + if (retVal) + metrics0().addRemoveAndGetTimeNanos(duration); + else + metrics0().addRemoveTimeNanos(duration); + } + } } /** 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 936c8a3bd3c87..536d0abafb792 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 @@ -42,6 +42,8 @@ import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.CLOCK; import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_ASYNC; +import static org.apache.ignite.internal.processors.cache.GridCacheOperation.DELETE; +import static org.apache.ignite.internal.processors.cache.GridCacheOperation.UPDATE; /** * Base for near atomic update futures. @@ -260,11 +262,34 @@ protected boolean storeFuture() { * @param req Request. */ protected void mapSingle(UUID nodeId, GridNearAtomicAbstractUpdateRequest req) { + final boolean statsEnabled = cctx.config().isStatisticsEnabled(); + + final long start = (statsEnabled)? System.nanoTime() : 0L; + if (cctx.localNodeId().equals(nodeId)) { cache.updateAllAsyncInternal(nodeId, req, new CI2() { @Override public void apply(GridNearAtomicAbstractUpdateRequest req, GridNearAtomicUpdateResponse res) { onResult(res.nodeId(), res, false); + + if (statsEnabled && res.error() == null) { + boolean retVal = req.returnValue(); + + long duration = System.nanoTime() - start; + + if (req.operation() == UPDATE) { + if (retVal) + cache.metrics0().addPutAndGetTimeNanos(duration); + else + cache.metrics0().addPutTimeNanos(duration); + } + else if (req.operation() == DELETE) { + if (retVal) + cache.metrics0().addRemoveAndGetTimeNanos(duration); + else + cache.metrics0().addRemoveTimeNanos(duration); + } + } } }); } 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 9bdd1becb7600..798b3889a8d6f 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 @@ -61,7 +61,9 @@ import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.CLOCK; import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_ASYNC; import static org.apache.ignite.cache.CacheWriteSynchronizationMode.PRIMARY_SYNC; +import static org.apache.ignite.internal.processors.cache.GridCacheOperation.DELETE; import static org.apache.ignite.internal.processors.cache.GridCacheOperation.TRANSFORM; +import static org.apache.ignite.internal.processors.cache.GridCacheOperation.UPDATE; /** * DHT atomic cache near update future. @@ -588,10 +590,33 @@ private void doUpdate(Map mappings) { } if (locUpdate != null) { + final boolean statsEnabled = cctx.config().isStatisticsEnabled(); + + final long start = (statsEnabled)? System.nanoTime() : 0L; + cache.updateAllAsyncInternal(cctx.localNodeId(), locUpdate, new CI2() { @Override public void apply(GridNearAtomicFullUpdateRequest req, GridNearAtomicUpdateResponse res) { onResult(res.nodeId(), res, false); + + if (statsEnabled && res.error() == null) { + boolean retVal = req.returnValue(); + + long duration = System.nanoTime() - start; + + if (req.operation() == UPDATE) { + if (retVal) + cache.metrics0().addPutAndGetTimeNanos(duration); + else + cache.metrics0().addPutTimeNanos(duration); + } + else if (req.operation() == DELETE) { + if (retVal) + cache.metrics0().addRemoveAndGetTimeNanos(duration); + else + cache.metrics0().addRemoveTimeNanos(duration); + } + } } }); } 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 e1e0ec2bf087e..fc36f6102316c 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 @@ -72,7 +72,6 @@ import org.apache.ignite.internal.util.typedef.CI2; import org.apache.ignite.internal.util.typedef.CX1; import org.apache.ignite.internal.util.typedef.F; -import org.apache.ignite.internal.util.typedef.T2; 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; @@ -289,12 +288,16 @@ public GridDistributedCacheEntry entryExx( if (keyCheck) validateCacheKeys(keys); + boolean statsEnabled = ctx.config().isStatisticsEnabled(); + + long start = statsEnabled ? System.nanoTime() : 0L; + IgniteTxLocalAdapter tx = ctx.tm().threadLocalTx(ctx); final CacheOperationContext opCtx = ctx.operationContextPerCall(); if (tx != null && !tx.implicit() && !skipTx) { - return asyncOp(tx, new AsyncOp>(keys) { + IgniteInternalFuture> fut = asyncOp(tx, new AsyncOp>(keys) { @Override public IgniteInternalFuture> op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) { return tx.getAllAsync(ctx, readyTopVer, @@ -306,6 +309,11 @@ public GridDistributedCacheEntry entryExx( needVer); } }, opCtx); + + if (statsEnabled) + fut.listen(new UpdateGetTimeStatClosure>(metrics0(), start)); + + return fut; } AffinityTopologyVersion topVer = tx == null ? @@ -314,7 +322,7 @@ public GridDistributedCacheEntry entryExx( subjId = ctx.subjectIdPerCall(subjId, opCtx); - return loadAsync( + IgniteInternalFuture> fut = loadAsync( ctx.cacheKeysView(keys), opCtx == null || !opCtx.skipStore(), forcePrimary, @@ -326,6 +334,11 @@ public GridDistributedCacheEntry entryExx( skipVals, canRemap, needVer); + + if (statsEnabled) + fut.listen(new UpdateGetTimeStatClosure>(metrics0(), start)); + + return fut; } /** 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 fb2843c02eec9..8e1c8ef79964f 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 @@ -81,6 +81,9 @@ public final class GridNearGetFuture extends CacheDistributedGetFutureAdap /** */ private GridCacheVersion ver; + /** Future start time in nanoseconds. */ + private final long startTimeNanos; + /** * @param cctx Context. * @param keys Keys. @@ -133,6 +136,8 @@ public GridNearGetFuture( ver = tx == null ? cctx.versions().next() : tx.xidVersion(); + startTimeNanos = cctx.config().isStatisticsEnabled() ? System.nanoTime() : 0L; + if (log == null) log = U.logger(cctx.kernalContext(), logRef, GridNearGetFuture.class); } @@ -224,6 +229,9 @@ public void onResult(UUID nodeId, GridNearGetResponse res) { cache().dht().sendTtlUpdateRequest(expiryPlc); + if (cctx.config().isStatisticsEnabled() && !trackable && !skipVals && err == null) + cctx.cache().metrics0().addGetTimeNanos(System.nanoTime() - startTimeNanos); + return true; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java index 5b44d759f9080..a7ada9db19228 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java @@ -19,12 +19,15 @@ import java.io.Externalizable; import java.util.Collection; +import java.util.Map; +import java.util.UUID; import java.util.concurrent.Callable; import org.apache.ignite.IgniteCheckedException; 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.CacheObject; +import org.apache.ignite.internal.processors.cache.CacheOperationContext; import org.apache.ignite.internal.processors.cache.GridCacheAdapter; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; @@ -234,4 +237,43 @@ void onFutureDone(GridLocalLockFuture fut) { log().debug("Explicitly removed future from map of futures: " + fut); } } + + /** {@inheritDoc} */ + protected IgniteInternalFuture> getAllAsync( + @Nullable Collection keys, + boolean forcePrimary, + boolean skipTx, + @Nullable UUID subjId, + String taskName, + boolean deserializeBinary, + boolean skipVals, + boolean canRemap, + final boolean needVer + ) { + boolean statsEnabled = ctx.config().isStatisticsEnabled(); + + long start = statsEnabled ? System.nanoTime() : 0L; + + CacheOperationContext opCtx = ctx.operationContextPerCall(); + + subjId = ctx.subjectIdPerCall(subjId, opCtx); + + IgniteInternalFuture> fut = getAllAsync(keys, + null, + opCtx == null || !opCtx.skipStore(), + !skipTx, + subjId, + taskName, + deserializeBinary, + forcePrimary, + skipVals ? null : expiryPolicy(opCtx != null ? opCtx.expiry() : null), + skipVals, + canRemap, + needVer); + + if (statsEnabled) + fut.listen(new UpdateGetTimeStatClosure>(metrics0(), start)); + + return fut; + } } 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 f86df2f29d684..dedd84ecef957 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 @@ -142,6 +142,10 @@ public GridLocalAtomicCache(GridCacheContext ctx) { /** {@inheritDoc} */ @Override protected boolean put0(K key, V val, CacheEntryPredicate filter) throws IgniteCheckedException { + boolean statsEnabled = ctx.config().isStatisticsEnabled(); + + long start = statsEnabled ? System.nanoTime() : 0L; + CacheOperationContext opCtx = ctx.operationContextPerCall(); Boolean res = (Boolean)updateAllInternal(UPDATE, @@ -158,29 +162,50 @@ public GridLocalAtomicCache(GridCacheContext ctx) { assert res != null; + if (statsEnabled && Boolean.TRUE.equals(res)) + metrics0().addPutTimeNanos(System.nanoTime() - start); + return res; } /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public IgniteInternalFuture getAndPutAsync0(K key, V val, @Nullable CacheEntryPredicate filter) { - return updateAllAsync0(F0.asMap(key, val), + boolean statsEnabled = ctx.config().isStatisticsEnabled(); + + long start = statsEnabled ? System.nanoTime() : 0L; + + IgniteInternalFuture fut = updateAllAsync0(F0.asMap(key, val), null, null, true, false, filter); + + if (statsEnabled) + fut.listen(new UpdatePutAndGetTimeStatClosure(metrics0(), start)); + + return fut; } /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public IgniteInternalFuture putAsync0(K key, V val, @Nullable CacheEntryPredicate filter) { - return updateAllAsync0(F0.asMap(key, val), + boolean statsEnabled = ctx.config().isStatisticsEnabled(); + + long start = statsEnabled ? System.nanoTime() : 0L; + + IgniteInternalFuture fut = updateAllAsync0(F0.asMap(key, val), null, null, false, false, filter); + + if (statsEnabled) + fut.listen(new UpdatePutTimeStatClosure(metrics0(), start)); + + return fut; } /** {@inheritDoc} */ @@ -230,7 +255,16 @@ public GridLocalAtomicCache(GridCacheContext ctx) { /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public IgniteInternalFuture getAndRemoveAsync0(K key) { - return removeAllAsync0(Collections.singletonList(key), true, false, null); + boolean statsEnabled = ctx.config().isStatisticsEnabled(); + + long start = statsEnabled ? System.nanoTime() : 0L; + + IgniteInternalFuture fut = removeAllAsync0(Collections.singletonList(key), true, false, null); + + if (statsEnabled) + fut.listen(new UpdateRemoveTimeStatClosure(metrics0(), start)); + + return fut; } /** {@inheritDoc} */ @@ -259,6 +293,10 @@ public GridLocalAtomicCache(GridCacheContext ctx) { /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public boolean remove0(K key, final CacheEntryPredicate filter) throws IgniteCheckedException { + boolean statsEnabled = ctx.config().isStatisticsEnabled(); + + long start = statsEnabled ? System.nanoTime() : 0L; + CacheOperationContext opCtx = ctx.operationContextPerCall(); Boolean rmv = (Boolean)updateAllInternal(DELETE, @@ -275,6 +313,9 @@ public GridLocalAtomicCache(GridCacheContext ctx) { assert rmv != null; + if (statsEnabled && Boolean.TRUE.equals(rmv)) + metrics0().addRemoveTimeNanos(System.nanoTime() - start); + return rmv; } @@ -303,6 +344,10 @@ public GridLocalAtomicCache(GridCacheContext ctx) { boolean deserializeBinary, boolean needVer) throws IgniteCheckedException { + boolean statsEnabled = ctx.config().isStatisticsEnabled(); + + long start = statsEnabled ? System.nanoTime() : 0L; + Map m = getAllInternal(Collections.singleton(key), ctx.isSwapOrOffheapEnabled(), ctx.readThrough(), @@ -313,6 +358,9 @@ public GridLocalAtomicCache(GridCacheContext ctx) { assert m.isEmpty() || m.size() == 1 : m.size(); + if (statsEnabled) + metrics0().addGetTimeNanos(System.nanoTime() - start); + return F.firstValue(m); } @@ -322,15 +370,24 @@ public GridLocalAtomicCache(GridCacheContext ctx) { throws IgniteCheckedException { A.notNull(keys, "keys"); + boolean statsEnabled = ctx.config().isStatisticsEnabled(); + + long start = statsEnabled ? System.nanoTime() : 0L; + String taskName = ctx.kernalContext().job().currentTaskName(); - return getAllInternal(keys, + Map result = getAllInternal(keys, ctx.isSwapOrOffheapEnabled(), ctx.readThrough(), taskName, deserializeBinary, false, needVer); + + if (statsEnabled) + metrics0().addGetTimeNanos(System.nanoTime() - start); + + return result; } /** {@inheritDoc} */ @@ -348,15 +405,24 @@ public GridLocalAtomicCache(GridCacheContext ctx) { ) { A.notNull(keys, "keys"); + boolean statsEnabled = ctx.config().isStatisticsEnabled(); + + long start = statsEnabled ? System.nanoTime() : 0L; + final boolean swapOrOffheap = ctx.isSwapOrOffheapEnabled(); final boolean storeEnabled = ctx.readThrough(); - return asyncOp(new Callable>() { + IgniteInternalFuture> fut = asyncOp(new Callable>() { @Override public Map call() throws Exception { return getAllInternal(keys, swapOrOffheap, storeEnabled, taskName, deserializeBinary, skipVals, needVer); } }); - } + + if (statsEnabled) + fut.listen(new UpdateGetTimeStatClosure>(metrics0(), start)); + + return fut; + } /** * Entry point to all public API get methods. diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicClientServerMetricsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicClientServerMetricsSelfTest.java new file mode 100644 index 0000000000000..42179e194a399 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicClientServerMetricsSelfTest.java @@ -0,0 +1,452 @@ +/* + * 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.atomic; + +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.TreeSet; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.Ignition; +import org.apache.ignite.cache.CacheMetrics; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cluster.ClusterGroup; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.events.Event; +import org.apache.ignite.lang.IgnitePredicate; +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.common.GridCommonAbstractTest; + +import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC; +import static org.apache.ignite.cache.CacheMode.PARTITIONED; +import static org.apache.ignite.cache.CacheMode.REPLICATED; +import static org.apache.ignite.events.EventType.EVT_NODE_METRICS_UPDATED; + +public class GridCacheAtomicClientServerMetricsSelfTest extends GridCommonAbstractTest { + /** Number of nodes. */ + private final static int GRID_CNT = 3; + + /** Server node index. */ + private static final int SERVER_NODE = 0; + + /** Client node index. */ + private static final int CLIENT_NODE = 2; + + /** */ + private static 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); + + cfg.setDiscoverySpi(disco); + + return cfg; + } + + private int gridCount() { + return GRID_CNT; + } + + /** {@inheritDoc} */ + protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + startGrids(GRID_CNT - 1); + + Ignition.setClientMode(true); + + startGrid(CLIENT_NODE); + + Ignition.setClientMode(false); + } + + /** {@inheritDoc} */ + protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + stopAllGrids(); + } + + /** + * Creates a new Cache Configuration for the given cache mode. + * + * @param mode Cache mode. + * @return Cache configuration. + */ + private CacheConfiguration getCacheConfiguration(CacheMode mode) { + CacheConfiguration cacheCfg = new CacheConfiguration<>(); + + cacheCfg.setCacheMode(mode); + cacheCfg.setAtomicityMode(ATOMIC); + cacheCfg.setStatisticsEnabled(true); + cacheCfg.setName("metrics"); + + return cacheCfg; + } + + /** + * Wait for {@link EventType#EVT_NODE_METRICS_UPDATED} event will be receieved. + */ + private void awaitMetricsUpdate() throws InterruptedException { + final CountDownLatch latch = new CountDownLatch(GRID_CNT * 2); + + IgnitePredicate lsnr = new IgnitePredicate() { + @Override public boolean apply(Event event) { + latch.countDown(); + return true; + } + }; + + for (int i = 0; i < gridCount(); ++i) + grid(i).events().localListen(lsnr, EVT_NODE_METRICS_UPDATED); + + latch.await(); + } + + /** + * @throws Exception If failed. + */ + public void testPartitionedGetAvgTime() throws Exception { + testGetAvgTime(PARTITIONED); + } + + /** + * @throws Exception If failed. + */ + public void testReplicatedGetAvgTime() throws Exception { + testGetAvgTime(REPLICATED); + } + + /** + * @param mode Cache mode. + * @throws Exception If failed. + */ + private void testGetAvgTime(CacheMode mode) throws Exception { + IgniteCache cache = null; + + try { + cache = grid(CLIENT_NODE).getOrCreateCache(getCacheConfiguration(mode)); + + ThreadLocalRandom rand = ThreadLocalRandom.current(); + + final int numOfKeys = 500; + for (int i = 0; i < numOfKeys; ++i) + cache.put(i, rand.nextInt(12_000_000)); + + for (int i = 0; i < numOfKeys; ++i) + cache.get(i); + + awaitMetricsUpdate(); + + ClusterGroup clientGroup = grid(CLIENT_NODE).cluster().forClients(); + ClusterGroup serverGroup = grid(SERVER_NODE).cluster().forServers(); + + CacheMetrics clientMetrics = cache.metrics(clientGroup); + CacheMetrics serverMetrics = cache.metrics(serverGroup); + + assertEquals(clientMetrics.getAveragePutTime(), 0.0, 0.0); + assertEquals(clientMetrics.getAverageGetTime(), 0.0, 0.0); + + assertTrue(serverMetrics.getAveragePutTime() > 0.0); + assertTrue(serverMetrics.getAverageGetTime() > 0.0); + } + finally { + if (cache != null) + cache.destroy(); + } + } + + /** + * @throws Exception If failed. + */ + public void testPartitionedGetAndPutAvgTime() throws Exception { + testGetAndPutAvgTime(PARTITIONED); + } + + /** + * @throws Exception If failed. + */ + public void testReplicatedGetAndPutAvgTime() throws Exception { + testGetAndPutAvgTime(REPLICATED); + } + + /** + * @param mode Cache mode. + * @throws Exception If failed. + */ + private void testGetAndPutAvgTime(CacheMode mode) throws Exception { + IgniteCache cache = null; + + try { + cache = grid(CLIENT_NODE).getOrCreateCache(getCacheConfiguration(mode)); + + ThreadLocalRandom rand = ThreadLocalRandom.current(); + + final int numOfKeys = 500; + for (int i = 0; i < numOfKeys; ++i) + cache.getAndPut(i, rand.nextInt(12_000_000)); + + awaitMetricsUpdate(); + + ClusterGroup clientGroup = grid(CLIENT_NODE).cluster().forClients(); + ClusterGroup serverGroup = grid(SERVER_NODE).cluster().forServers(); + + CacheMetrics clientMetrics = cache.metrics(clientGroup); + CacheMetrics serverMetrics = cache.metrics(serverGroup); + + assertEquals(clientMetrics.getAveragePutTime(), 0.0, 0.0); + assertEquals(clientMetrics.getAverageGetTime(), 0.0, 0.0); + + assertTrue(serverMetrics.getAveragePutTime() > 0.0); + assertTrue(serverMetrics.getAverageGetTime() > 0.0); + } + finally { + if (cache != null) + cache.destroy(); + } + } + + /** + * @throws Exception If failed. + */ + public void testPartitionedRemoveAvgTime() throws Exception { + testRemoveAvgTime(PARTITIONED); + } + + /** + * @throws Exception If failed. + */ + public void testReplicatedRemoveAvgTime() throws Exception { + testRemoveAvgTime(REPLICATED); + } + + /** + * @param mode Cache mode. + * @throws Exception If failed. + */ + private void testRemoveAvgTime(CacheMode mode) throws Exception { + IgniteCache cache = null; + + try { + cache = grid(CLIENT_NODE).getOrCreateCache(getCacheConfiguration(mode)); + + ThreadLocalRandom rand = ThreadLocalRandom.current(); + + final int numOfKeys = 500; + for (int i = 0; i < numOfKeys; ++i) + cache.put(i, rand.nextInt(12_000_000)); + + for (int i = 0; i < numOfKeys; ++i) + cache.remove(i); + + awaitMetricsUpdate(); + + ClusterGroup clientGroup = grid(CLIENT_NODE).cluster().forClients(); + ClusterGroup serverGroup = grid(SERVER_NODE).cluster().forServers(); + + CacheMetrics clientMetrics = cache.metrics(clientGroup); + CacheMetrics serverMetrics = cache.metrics(serverGroup); + + assertEquals(clientMetrics.getAveragePutTime(), 0.0, 0.0); + assertEquals(clientMetrics.getAverageRemoveTime(), 0.0, 0.0); + + assertTrue(serverMetrics.getAveragePutTime() > 0.0); + assertTrue(serverMetrics.getAverageRemoveTime() > 0.0); + } + finally { + if (cache != null) + cache.destroy(); + } + } + + /** + * @throws Exception If failed. + */ + public void testPartitionedGetAllAvgTime() throws Exception { + testGetAllAvgTime(PARTITIONED); + } + + /** + * @throws Exception If failed. + */ + public void testReplicatedGetAllAvgTime() throws Exception { + testGetAllAvgTime(REPLICATED); + } + + /** + * @param mode Cache mode. + * @throws Exception If failed. + */ + private void testGetAllAvgTime(CacheMode mode) throws Exception { + IgniteCache cache = null; + + try { + cache = grid(CLIENT_NODE).getOrCreateCache(getCacheConfiguration(mode)); + + ThreadLocalRandom rand = ThreadLocalRandom.current(); + + Set keys = new TreeSet<>(); + + final int numOfKeys = 500; + for (int i = 0; i < numOfKeys; ++i) { + cache.put(i, rand.nextInt(12_000_000)); + + keys.add(i); + } + + cache.getAll(keys); + + awaitMetricsUpdate(); + + ClusterGroup clientGroup = grid(CLIENT_NODE).cluster().forClients(); + ClusterGroup serverGroup = grid(SERVER_NODE).cluster().forServers(); + + CacheMetrics clientMetrics = cache.metrics(clientGroup); + CacheMetrics serverMetrics = cache.metrics(serverGroup); + + assertEquals(clientMetrics.getAveragePutTime(), 0.0, 0.0); + assertEquals(clientMetrics.getAverageGetTime(), 0.0, 0.0); + + assertTrue(serverMetrics.getAveragePutTime() > 0.0); + assertTrue(serverMetrics.getAverageGetTime() > 0.0); + } + finally { + if (cache != null) + cache.destroy(); + } + } + + /** + * @throws Exception If failed. + */ + public void testPartitionedPutAllAvgTime() throws Exception { + testPutAllAvgTime(PARTITIONED); + } + + /** + * @throws Exception If failed. + */ + public void testReplicatedPutAllAvgTime() throws Exception { + testPutAllAvgTime(REPLICATED); + } + + /** + * @param mode Cache mode. + * @throws Exception If failed. + */ + private void testPutAllAvgTime(CacheMode mode) throws Exception { + IgniteCache cache = null; + + try { + cache = grid(CLIENT_NODE).getOrCreateCache(getCacheConfiguration(mode)); + + ThreadLocalRandom rand = ThreadLocalRandom.current(); + + Map values = new TreeMap<>(); + + final int numOfKeys = 500; + for (int i = 0; i < numOfKeys; ++i) + values .put(i, rand.nextInt(12_000_000)); + + cache.putAll(values); + + awaitMetricsUpdate(); + + ClusterGroup clientGroup = grid(CLIENT_NODE).cluster().forClients(); + ClusterGroup serverGroup = grid(SERVER_NODE).cluster().forServers(); + + CacheMetrics clientMetrics = cache.metrics(clientGroup); + CacheMetrics serverMetrics = cache.metrics(serverGroup); + + assertEquals(clientMetrics.getAveragePutTime(), 0.0, 0.0); + + assertTrue(serverMetrics.getAveragePutTime() > 0.0); + } + finally { + if (cache != null) + cache.destroy(); + } + } + + /** + * @throws Exception If failed. + */ + public void testPartitionedRemoveAllAvgTime() throws Exception { + testRemoveAllAvgTime(PARTITIONED); + } + + /** + * @throws Exception If failed. + */ + public void testReplicatedRemoveAllAvgTime() throws Exception { + testRemoveAllAvgTime(REPLICATED); + } + + /** + * @param mode Cache mode. + * @throws Exception If failed. + */ + private void testRemoveAllAvgTime(CacheMode mode) throws Exception { + IgniteCache cache = null; + + try { + cache = grid(CLIENT_NODE).getOrCreateCache(getCacheConfiguration(mode)); + + ThreadLocalRandom rand = ThreadLocalRandom.current(); + + Set keys = new TreeSet<>(); + + final int numOfKeys = 500; + for (int i = 0; i < numOfKeys; ++i) { + cache.put(i, rand.nextInt(12_000_000)); + + keys.add(i); + } + + cache.removeAll(keys); + + awaitMetricsUpdate(); + + ClusterGroup clientGroup = grid(CLIENT_NODE).cluster().forClients(); + ClusterGroup serverGroup = grid(SERVER_NODE).cluster().forServers(); + + CacheMetrics clientMetrics = cache.metrics(clientGroup); + CacheMetrics serverMetrics = cache.metrics(serverGroup); + + assertEquals(clientMetrics.getAveragePutTime(), 0.0, 0.0); + assertEquals(clientMetrics.getAverageRemoveTime(), 0.0, 0.0); + + assertTrue(serverMetrics.getAveragePutTime() > 0.0); + assertTrue(serverMetrics.getAverageRemoveTime() > 0.0); + } + finally { + if (cache != null) + cache.destroy(); + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMetricsSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMetricsSelfTestSuite.java index f3dd1c83d5924..5785237a103a2 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMetricsSelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMetricsSelfTestSuite.java @@ -20,6 +20,7 @@ import junit.framework.TestSuite; import org.apache.ignite.internal.processors.cache.CacheMetricsForClusterGroupSelfTest; import org.apache.ignite.internal.processors.cache.OffheapCacheMetricsForClusterGroupSelfTest; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridCacheAtomicClientServerMetricsSelfTest; import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicPartitionedMetricsSelfTest; import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicPartitionedTckMetricsSelfTestImpl; import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheNearMetricsSelfTest; @@ -58,6 +59,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(GridCacheAtomicPartitionedMetricsSelfTest.class); suite.addTestSuite(GridCacheAtomicPartitionedTckMetricsSelfTestImpl.class); suite.addTestSuite(GridCacheAtomicLocalTckMetricsSelfTestImpl.class); + suite.addTestSuite(GridCacheAtomicClientServerMetricsSelfTest.class); // Cluster wide metrics. suite.addTestSuite(CacheMetricsForClusterGroupSelfTest.class); @@ -65,4 +67,4 @@ public static TestSuite suite() throws Exception { return suite; } -} \ No newline at end of file +} From 0a65037ee01e6f1a552c3771fd3940b6b23827bc Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Tue, 26 Sep 2017 17:38:21 +0300 Subject: [PATCH 299/357] Backport for IGNITE-2779: BinaryMarshaller caches must be cleaned during client reconnect. --- .../ignite/internal/binary/BinaryContext.java | 15 ++ .../CacheObjectBinaryProcessorImpl.java | 4 +- ...IgniteClientReconnectBinaryContexTest.java | 185 ++++++++++++++++++ .../IgniteClientReconnectTestSuite.java | 2 + 4 files changed, 205 insertions(+), 1 deletion(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectBinaryContexTest.java 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 b0fa68c30f0ff..bbf15bbcc4ecb 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 @@ -17,6 +17,7 @@ package org.apache.ignite.internal.binary; +import java.util.Iterator; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; @@ -1303,6 +1304,20 @@ public void unregisterBinarySchemas() { schemas = null; } + /** + * Unregisters the user types descriptors. + **/ + public void unregisterUserTypeDescriptors() { + Iterator, BinaryClassDescriptor>> it = descByCls.entrySet().iterator(); + + while (it.hasNext()) { + Map.Entry, BinaryClassDescriptor> e = it.next(); + + if (e.getValue().userType()) + it.remove(); + } + } + /** * Returns instance of {@link OptimizedMarshaller}. * 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 6b691c2236e52..5939d643aac48 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 @@ -155,9 +155,11 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm @GridToStringExclude private IgniteBinary binaries; - /** Listener removes all registred binary schemas after the local client reconnected. */ + /** Listener removes all registred binary schemas and user type descriptors after the local client reconnected. */ private final GridLocalEventListener clientDisconLsnr = new GridLocalEventListener() { @Override public void onEvent(Event evt) { + binaryContext().unregisterUserTypeDescriptors(); + binaryContext().unregisterBinarySchemas(); } }; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectBinaryContexTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectBinaryContexTest.java new file mode 100644 index 0000000000000..7401799629fac --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectBinaryContexTest.java @@ -0,0 +1,185 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal; + +import java.util.Collection; +import java.util.Collections; +import java.util.concurrent.Callable; +import javax.cache.processor.EntryProcessor; +import javax.cache.processor.EntryProcessorException; +import javax.cache.processor.MutableEntry; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.binary.BinaryMarshaller; + +/** + * + */ +public class IgniteClientReconnectBinaryContexTest extends IgniteClientReconnectAbstractTest { + /** */ + public static final String DEFAULT_CACHE_NAME = "myCache"; + + /** {@inheritDoc} */ + @Override protected int serverCount() { + return 1; + } + + /** {@inheritDoc} */ + @Override protected int clientCount() { + return 1; + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + cfg.setPeerClassLoadingEnabled(false); + + cfg.setMarshaller(new BinaryMarshaller()); + + return cfg; + } + + /** + * @throws Exception If failed. + */ + public void testReconnectCleaningUsersMetadata() throws Exception { + Ignite client = grid(serverCount()); + + assertTrue(client.cluster().localNode().isClient()); + + Ignite srv = grid(0); + + srv.destroyCache(DEFAULT_CACHE_NAME); + + CacheConfiguration cacheCfg = new CacheConfiguration() + .setName(DEFAULT_CACHE_NAME) + .setCacheMode(CacheMode.REPLICATED); + + IgniteCache cache = client.createCache(cacheCfg); + + Integer key = 1; + UserClass val = new UserClass(1); + + cache.put(key, val); // For registering user types binary metadata + + reconnectServersRestart(log, client, Collections.singleton(srv), new Callable>() { + /** {@inheritDoc} */ + @Override public Collection call() throws Exception { + return Collections.singleton((Ignite)startGrid(0)); + } + }); + + cache = client.createCache(cacheCfg); + + cache.put(key, val); + + assertEquals(val, cache.get(key)); + } + + /** + * @throws Exception If failed. + */ + public void testReconnectCleaningUsersMetadata2() throws Exception { + Ignite client = grid(serverCount()); + + assertTrue(client.cluster().localNode().isClient()); + + Ignite srv = grid(0); + + srv.destroyCache(DEFAULT_CACHE_NAME); + + CacheConfiguration cacheCfg = new CacheConfiguration() + .setName(DEFAULT_CACHE_NAME) + .setCacheMode(CacheMode.REPLICATED); + + IgniteCache cache = client.createCache(cacheCfg); + + Integer key = 1; + final UserClass val = new UserClass(1); + + EntryProcessor ep = new DummyEntryProccessor(val); + + cache.invoke(key, ep); // For registering user types binary metadata + + reconnectServersRestart(log, client, Collections.singleton(srv), new Callable>() { + /** {@inheritDoc} */ + @Override public Collection call() throws Exception { + return Collections.singleton((Ignite)startGrid(0)); + } + }); + + cache = client.createCache(cacheCfg); + + cache.invoke(key, ep); + + assertEquals(val, cache.get(key)); + } + + /** */ + private static class UserClass { + /** */ + private final int field; + + /** + * @param field Value. + */ + private UserClass(int field) { + this.field = field; + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + UserClass val = (UserClass)o; + + return field == val.field; + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return field; + } + } + + /** */ + private static class DummyEntryProccessor implements EntryProcessor { + /** */ + private final UserClass val; + + /** Constructor. */ + public DummyEntryProccessor(UserClass val) { + this.val = val; + } + + /** {@inheritDoc} */ + @Override public UserClass process(MutableEntry entry, Object... arguments) + throws EntryProcessorException { + entry.setValue(val); + + return null; + } + } +} 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 67d88e1e29eac..03d3fe230c296 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 @@ -20,6 +20,7 @@ import junit.framework.TestSuite; import org.apache.ignite.internal.IgniteClientReconnectApiExceptionTest; import org.apache.ignite.internal.IgniteClientReconnectAtomicsTest; +import org.apache.ignite.internal.IgniteClientReconnectBinaryContexTest; import org.apache.ignite.internal.IgniteClientReconnectCacheTest; import org.apache.ignite.internal.IgniteClientReconnectCollectionsTest; import org.apache.ignite.internal.IgniteClientReconnectComputeTest; @@ -46,6 +47,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(IgniteClientReconnectApiExceptionTest.class); suite.addTestSuite(IgniteClientReconnectDiscoveryStateTest.class); suite.addTestSuite(IgniteClientReconnectCacheTest.class); + suite.addTestSuite(IgniteClientReconnectBinaryContexTest.class); suite.addTestSuite(IgniteClientReconnectContinuousProcessorTest.class); suite.addTestSuite(IgniteClientReconnectComputeTest.class); suite.addTestSuite(IgniteClientReconnectAtomicsTest.class); From 39ee8af17f5cbe2db83b5ce127ac6e85c9fb4bf7 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Tue, 26 Sep 2017 17:38:21 +0300 Subject: [PATCH 300/357] Backport for IGNITE-2779: BinaryMarshaller caches must be cleaned during client reconnect. --- .../ignite/internal/binary/BinaryContext.java | 15 ++ .../CacheObjectBinaryProcessorImpl.java | 4 +- ...IgniteClientReconnectBinaryContexTest.java | 185 ++++++++++++++++++ .../IgniteClientReconnectTestSuite.java | 2 + 4 files changed, 205 insertions(+), 1 deletion(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectBinaryContexTest.java 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 b0fa68c30f0ff..bbf15bbcc4ecb 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 @@ -17,6 +17,7 @@ package org.apache.ignite.internal.binary; +import java.util.Iterator; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; @@ -1303,6 +1304,20 @@ public void unregisterBinarySchemas() { schemas = null; } + /** + * Unregisters the user types descriptors. + **/ + public void unregisterUserTypeDescriptors() { + Iterator, BinaryClassDescriptor>> it = descByCls.entrySet().iterator(); + + while (it.hasNext()) { + Map.Entry, BinaryClassDescriptor> e = it.next(); + + if (e.getValue().userType()) + it.remove(); + } + } + /** * Returns instance of {@link OptimizedMarshaller}. * 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 f6a396992adab..62fe328882a80 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 @@ -155,9 +155,11 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm @GridToStringExclude private IgniteBinary binaries; - /** Listener removes all registred binary schemas after the local client reconnected. */ + /** Listener removes all registred binary schemas and user type descriptors after the local client reconnected. */ private final GridLocalEventListener clientDisconLsnr = new GridLocalEventListener() { @Override public void onEvent(Event evt) { + binaryContext().unregisterUserTypeDescriptors(); + binaryContext().unregisterBinarySchemas(); } }; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectBinaryContexTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectBinaryContexTest.java new file mode 100644 index 0000000000000..7401799629fac --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectBinaryContexTest.java @@ -0,0 +1,185 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal; + +import java.util.Collection; +import java.util.Collections; +import java.util.concurrent.Callable; +import javax.cache.processor.EntryProcessor; +import javax.cache.processor.EntryProcessorException; +import javax.cache.processor.MutableEntry; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.binary.BinaryMarshaller; + +/** + * + */ +public class IgniteClientReconnectBinaryContexTest extends IgniteClientReconnectAbstractTest { + /** */ + public static final String DEFAULT_CACHE_NAME = "myCache"; + + /** {@inheritDoc} */ + @Override protected int serverCount() { + return 1; + } + + /** {@inheritDoc} */ + @Override protected int clientCount() { + return 1; + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + cfg.setPeerClassLoadingEnabled(false); + + cfg.setMarshaller(new BinaryMarshaller()); + + return cfg; + } + + /** + * @throws Exception If failed. + */ + public void testReconnectCleaningUsersMetadata() throws Exception { + Ignite client = grid(serverCount()); + + assertTrue(client.cluster().localNode().isClient()); + + Ignite srv = grid(0); + + srv.destroyCache(DEFAULT_CACHE_NAME); + + CacheConfiguration cacheCfg = new CacheConfiguration() + .setName(DEFAULT_CACHE_NAME) + .setCacheMode(CacheMode.REPLICATED); + + IgniteCache cache = client.createCache(cacheCfg); + + Integer key = 1; + UserClass val = new UserClass(1); + + cache.put(key, val); // For registering user types binary metadata + + reconnectServersRestart(log, client, Collections.singleton(srv), new Callable>() { + /** {@inheritDoc} */ + @Override public Collection call() throws Exception { + return Collections.singleton((Ignite)startGrid(0)); + } + }); + + cache = client.createCache(cacheCfg); + + cache.put(key, val); + + assertEquals(val, cache.get(key)); + } + + /** + * @throws Exception If failed. + */ + public void testReconnectCleaningUsersMetadata2() throws Exception { + Ignite client = grid(serverCount()); + + assertTrue(client.cluster().localNode().isClient()); + + Ignite srv = grid(0); + + srv.destroyCache(DEFAULT_CACHE_NAME); + + CacheConfiguration cacheCfg = new CacheConfiguration() + .setName(DEFAULT_CACHE_NAME) + .setCacheMode(CacheMode.REPLICATED); + + IgniteCache cache = client.createCache(cacheCfg); + + Integer key = 1; + final UserClass val = new UserClass(1); + + EntryProcessor ep = new DummyEntryProccessor(val); + + cache.invoke(key, ep); // For registering user types binary metadata + + reconnectServersRestart(log, client, Collections.singleton(srv), new Callable>() { + /** {@inheritDoc} */ + @Override public Collection call() throws Exception { + return Collections.singleton((Ignite)startGrid(0)); + } + }); + + cache = client.createCache(cacheCfg); + + cache.invoke(key, ep); + + assertEquals(val, cache.get(key)); + } + + /** */ + private static class UserClass { + /** */ + private final int field; + + /** + * @param field Value. + */ + private UserClass(int field) { + this.field = field; + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + UserClass val = (UserClass)o; + + return field == val.field; + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return field; + } + } + + /** */ + private static class DummyEntryProccessor implements EntryProcessor { + /** */ + private final UserClass val; + + /** Constructor. */ + public DummyEntryProccessor(UserClass val) { + this.val = val; + } + + /** {@inheritDoc} */ + @Override public UserClass process(MutableEntry entry, Object... arguments) + throws EntryProcessorException { + entry.setValue(val); + + return null; + } + } +} 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 67d88e1e29eac..03d3fe230c296 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 @@ -20,6 +20,7 @@ import junit.framework.TestSuite; import org.apache.ignite.internal.IgniteClientReconnectApiExceptionTest; import org.apache.ignite.internal.IgniteClientReconnectAtomicsTest; +import org.apache.ignite.internal.IgniteClientReconnectBinaryContexTest; import org.apache.ignite.internal.IgniteClientReconnectCacheTest; import org.apache.ignite.internal.IgniteClientReconnectCollectionsTest; import org.apache.ignite.internal.IgniteClientReconnectComputeTest; @@ -46,6 +47,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(IgniteClientReconnectApiExceptionTest.class); suite.addTestSuite(IgniteClientReconnectDiscoveryStateTest.class); suite.addTestSuite(IgniteClientReconnectCacheTest.class); + suite.addTestSuite(IgniteClientReconnectBinaryContexTest.class); suite.addTestSuite(IgniteClientReconnectContinuousProcessorTest.class); suite.addTestSuite(IgniteClientReconnectComputeTest.class); suite.addTestSuite(IgniteClientReconnectAtomicsTest.class); From 544e73a11feb29a499c6eebde97a1b90aa6e8ece Mon Sep 17 00:00:00 2001 From: Slava Koptilin Date: Thu, 28 Sep 2017 14:14:43 +0300 Subject: [PATCH 301/357] ignite-gg-12855: Fix IgniteCacheTopologySafeGetSelfTest#testGetTopologySafeNodeJoinPrimaryLeave --- .../cache/IgniteCacheTopologySafeGetSelfTest.java | 4 ++-- .../ignite/testframework/junits/GridAbstractTest.java | 9 +++++++++ 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheTopologySafeGetSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheTopologySafeGetSelfTest.java index 674f2b0adf503..6da99bbf7c5a3 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheTopologySafeGetSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheTopologySafeGetSelfTest.java @@ -147,7 +147,7 @@ public void checkGetTopologySafeNodeJoin(boolean failPrimary) throws Exception { IgniteInternalFuture nodeFut = startNodeAsync(); if (failPrimary) - stopGrid(1); + stopGrid(1, false, false); assertEquals(key, ((IgniteKernal)ignite(0)).internalCache("tx").getTopologySafe(key)); assertEquals(key, ((IgniteKernal)ignite(0)).internalCache("atomic").getTopologySafe(key)); @@ -219,4 +219,4 @@ private void releaseTx() { releaseLatch.countDown(); } -} \ No newline at end of file +} diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java index e853b5046c094..22c796f6cd3e1 100755 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java @@ -1182,6 +1182,15 @@ protected void stopGrid(int idx, boolean cancel) { } } + /** + * @param idx Index of the grid to stop. + * @param cancel Cancel flag. + * @param awaitTop Await topology change flag. + */ + protected void stopGrid(int idx, boolean cancel, boolean awaitTop) { + stopGrid(getTestGridName(idx), false, awaitTop); + } + /** * @param idx Index of the grid to stop. */ From ac8379ef55b7f091b97707e107d967544988493c Mon Sep 17 00:00:00 2001 From: Ilya Kasnacheev Date: Thu, 28 Sep 2017 16:21:34 +0300 Subject: [PATCH 302/357] GG-12826 Backport fixes for NPE in GridDhtPartitionDemander. --- .../preloader/GridDhtPartitionDemander.java | 43 +++++++++---------- 1 file changed, 21 insertions(+), 22 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 9d12d0edc97a2..ab2dae917d1f7 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 @@ -406,26 +406,35 @@ else if (delay > 0) { } /** - * @param fut Future. + * @param fut Rebalance future. * @param assigns Assignments. - * @throws IgniteCheckedException If failed. - * @return Partitions were requested. */ private void requestPartitions( final RebalanceFuture fut, GridDhtPreloaderAssignments assigns){ + assert fut != null; + if (topologyChanged(fut)) { fut.cancel(); return; } - for (Map.Entry e : assigns.entrySet()) { - final ClusterNode node = e.getKey(); + synchronized (fut) { + if (fut.isDone()) + return; - GridDhtPartitionDemandMessage d = e.getValue(); + // Must add all remaining node before send first request, for avoid race between add remaining node and + // processing response, see checkIsDone(boolean). + for (Map.Entry e : assigns.entrySet()) { + UUID nodeId = e.getKey().id(); - fut.appendPartitions(node.id(), d.partitions()); //Future preparation. + Collection parts = e.getValue().partitions(); + + assert parts != null : "Partitions are null [cache=" + cctx.name() + ", fromNode=" + nodeId + "]"; + + fut.remaining.put(nodeId, new T2<>(U.currentTimeMillis(), parts)); + } } for (Map.Entry e : assigns.entrySet()) { @@ -433,10 +442,10 @@ private void requestPartitions( final CacheConfiguration cfg = cctx.config(); - final Collection parts = fut.remaining.get(node.id()).get2(); - GridDhtPartitionDemandMessage d = e.getValue(); + final Collection parts = d.partitions(); + //Check remote node rebalancing API version. if (node.version().compareTo(GridDhtPreloader.REBALANCING_VER_2_SINCE) >= 0) { U.log(log, "Starting rebalancing [mode=" + cfg.getRebalanceMode() + @@ -445,7 +454,7 @@ private void requestPartitions( int lsnrCnt = cctx.gridConfig().getRebalanceThreadPoolSize(); - final List> sParts = new ArrayList<>(lsnrCnt); + final List> sParts = new ArrayList<>(lsnrCnt); for (int cnt = 0; cnt < lsnrCnt; cnt++) sParts.add(new HashSet()); @@ -472,6 +481,8 @@ private void requestPartitions( @Override public void run() { try { 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(finalCnt), initD, cctx.ioPolicy(), initD.timeout()); @@ -915,18 +926,6 @@ private boolean isInitial() { return topVer == null; } - /** - * @param nodeId Node id. - * @param parts Parts. - */ - private void appendPartitions(UUID nodeId, Collection parts) { - synchronized (this) { - assert parts != null : "Partitions are null [cache=" + cctx.name() + ", fromNode=" + nodeId + "]"; - - remaining.put(nodeId, new T2<>(U.currentTimeMillis(), parts)); - } - } - /** * Cancels this future. * From 1df9b02284516707133237f5fe1280d992d0fb89 Mon Sep 17 00:00:00 2001 From: Evgenii Zhuravlev Date: Fri, 29 Sep 2017 18:03:51 +0300 Subject: [PATCH 303/357] Merge branch 'ignite-1.8.11' into ignite-1.9.7 --- .../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 2ef6aa856ab86..a22ccaad367ff 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 @@ -565,7 +565,7 @@ public IgniteInternalFuture deployAll(Collection cfgs) if (!busyLock.enterBusy()) { IgniteCheckedException e = new IgniteCheckedException("Service deployment has been cancelled (node is stopping): " + - cfg.getName()); + cfgs); return new GridFinishedFuture<>(e); } From 890be7a55a0fd6af6b19dee0ca923706f8039e5a Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Tue, 3 Oct 2017 09:57:36 +0300 Subject: [PATCH 304/357] GG-12891 - Event EVT_CACHE_REBALANCE_PART_DATA_LOST should be fired only once --- .../dht/GridDhtPartitionTopologyImpl.java | 22 ++- .../GridDhtPartitionsExchangeFuture.java | 6 +- .../GridLostPartitionRebalanceTest.java | 140 +++++++++++++++--- 3 files changed, 130 insertions(+), 38 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 eb6658bd2a01b..f790ac64e6cd1 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 @@ -604,14 +604,6 @@ else if (!node2part.nodeId().equals(loc.id())) { changed = true; - if (cctx.events().isRecordable(EVT_CACHE_REBALANCE_PART_DATA_LOST)) { - DiscoveryEvent discoEvt = exchFut.discoveryEvent(); - - cctx.events().addPreloadEvent(p, - EVT_CACHE_REBALANCE_PART_DATA_LOST, discoEvt.eventNode(), - discoEvt.type(), discoEvt.timestamp()); - } - if (log.isDebugEnabled()) log.debug("Owned partition: " + locPart); } @@ -1237,6 +1229,10 @@ private List ownersAndMoving(int p, AffinityTopologyVersion topVer) /** {@inheritDoc} */ @Override public void detectLostPartitions(DiscoveryEvent discoEvt) { + // Method generates events only. + if (!cctx.events().isRecordable(EventType.EVT_CACHE_REBALANCE_PART_DATA_LOST)) + return; + lock.writeLock().lock(); try { @@ -1255,7 +1251,9 @@ private List ownersAndMoving(int p, AffinityTopologyVersion topVer) GridDhtPartitionState state = partMap.get(p); - if (state == OWNING) { + // We need to extra check owners() using discovery, + // because on coordinator a new owner is already determined. + if (state == OWNING && !F.isEmpty(owners(p))) { foundOwner = true; break; @@ -1272,11 +1270,9 @@ private List ownersAndMoving(int p, AffinityTopologyVersion topVer) } if (lost != null) { - // Update partition state on all nodes. for (Integer part : lost) { - if (cctx.events().isRecordable(EventType.EVT_CACHE_REBALANCE_PART_DATA_LOST)) - cctx.events().addPreloadEvent(part, EVT_CACHE_REBALANCE_PART_DATA_LOST, - discoEvt.eventNode(), discoEvt.type(), discoEvt.timestamp()); + cctx.events().addPreloadEvent(part, EVT_CACHE_REBALANCE_PART_DATA_LOST, + discoEvt.eventNode(), discoEvt.type(), discoEvt.timestamp()); } } } 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 0de932780a1ec..13fdb30137f58 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 @@ -1102,8 +1102,7 @@ private void sendPartitions(ClusterNode oldestNode) { } if (discoEvt.type() == EVT_NODE_LEFT || - discoEvt.type() == EVT_NODE_FAILED || - discoEvt.type() == EVT_NODE_JOINED) + discoEvt.type() == EVT_NODE_FAILED) detectLostPartitions(); Map m = null; @@ -1441,9 +1440,6 @@ private void onAllReceived() { } } - if (discoEvt.type() == EVT_NODE_LEFT || discoEvt.type() == EVT_NODE_FAILED) - detectLostPartitions(); - for (GridCacheContext cacheCtx : cctx.cacheContexts()) { if (!cacheCtx.isLocal()) cacheCtx.topology().checkEvictions(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridLostPartitionRebalanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridLostPartitionRebalanceTest.java index e063b1c138f48..56eed411ce56b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridLostPartitionRebalanceTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridLostPartitionRebalanceTest.java @@ -19,13 +19,18 @@ import java.util.ArrayList; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.concurrent.Callable; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.cache.CacheWriteSynchronizationMode; import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.CacheConfiguration; @@ -33,11 +38,15 @@ import org.apache.ignite.events.CacheRebalancingEvent; 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.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; +import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgnitePredicate; import org.apache.ignite.resources.IgniteInstanceResource; +import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.jetbrains.annotations.NotNull; import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_PART_DATA_LOST; @@ -46,14 +55,23 @@ */ public class GridLostPartitionRebalanceTest extends GridCommonAbstractTest { /** Cache name. */ - private static final String CACHE_NAME = "cache"; + private static final String CACHE_NAME = "test"; /** Latch. */ - private static CountDownLatch latch = new CountDownLatch(4); + private static CountDownLatch latch; + + /** Count. */ + private static AtomicInteger cnt; /** Failed flag. */ private static boolean failed; + /** Backups. */ + private int backups; + + /** Expected events. */ + private int expEvts; + /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { @@ -63,6 +81,8 @@ public class GridLostPartitionRebalanceTest extends GridCommonAbstractTest { ccfg.setAffinity(new RendezvousAffinityFunction(false, 32)); ccfg.setNodeFilter(NODE_FILTER); + ccfg.setBackups(backups); + ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); cfg.setCacheConfiguration(ccfg); @@ -100,10 +120,36 @@ public class GridLostPartitionRebalanceTest extends GridCommonAbstractTest { } }; + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + latch = new CountDownLatch(expEvts); + cnt = new AtomicInteger(0); + } + + /** + * @throws Exception If failed. + */ + public void testPartDataLostEvent1Backup() throws Exception { + expEvts = 3; + backups = 1; + + checkEvents(); + } + /** * @throws Exception If failed. */ - public void testPartDataLostEvent() throws Exception { + public void testPartDataLostEventNoBackups() throws Exception { + expEvts = 4; + backups = 0; + + checkEvents(); + } + + /** + * @throws Exception If failed. + */ + private void checkEvents() throws Exception { List srvrs = new ArrayList<>(); // Client router. It always up, so client is guaranteed to get @@ -124,23 +170,80 @@ public void testPartDataLostEvent() throws Exception { cache.put(i, i); // Stop node with 0 partition. - ClusterNode node = client.affinity(CACHE_NAME).mapPartitionToNode(0); + Set nodes = new HashSet<>(client.affinity(CACHE_NAME).mapPartitionToPrimaryAndBackups(0)); - for (Ignite srv : srvrs) { - if (node.equals(srv.cluster().localNode())) { - srv.close(); - - System.out.println(">> Stopped " + srv.name()); - - break; - } - } + final List stopped = stopAffinityNodes(srvrs, nodes); // Check that all nodes (and clients) got notified. assert latch.await(15, TimeUnit.SECONDS) : latch.getCount(); // Check that exchange was not finished when event received. assertFalse("Exchange was finished when event received.", failed); + + U.sleep(4_000); + + assertEquals("Fired more events than expected", expEvts, cnt.get()); + + startNodes(stopped); + + assertEquals("Fired unexpected events", expEvts, cnt.get()); + } + + /** + * @param nodeNames Node names. + */ + private void startNodes(List nodeNames) throws Exception { + List> futs = new ArrayList<>(); + + for (final String nodeName : nodeNames) { + IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { + @Override public Object call() throws Exception { + startGrid(nodeName); + + return null; + } + }); + + futs.add(fut); + } + + for (IgniteInternalFuture fut : futs) + fut.get(); + } + + /** + * @param srvrs Servers. + * @param nodes Nodes. + */ + @NotNull private List stopAffinityNodes(List srvrs, Set nodes) throws IgniteCheckedException { + List> futs = new ArrayList<>(); + + final List stopped = new ArrayList<>(); + + for (final Ignite srv : srvrs) { + final ClusterNode node = srv.cluster().localNode(); + + if (nodes.contains(node)) { + IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { + @Override public Object call() throws Exception { + srv.close(); + + System.out.println(">> Stopped " + srv.name() + " " + node.id()); + + stopped.add(srv.name()); + + return null; + } + }); + + futs.add(fut); + } + } + + for (IgniteInternalFuture fut : futs) + fut.get(); + + return stopped; } /** @@ -155,24 +258,21 @@ private static class Listener implements IgnitePredicate @IgniteInstanceResource private Ignite ignite; - /** Got. */ - private final AtomicBoolean got = new AtomicBoolean(false); - /** {@inheritDoc} */ @Override public boolean apply(CacheRebalancingEvent evt) { int part = evt.partition(); // AtomicBoolean because new owner will produce two events. - if (part == 0 && CACHE_NAME.equals(evt.cacheName()) && got.compareAndSet(false, true)) { + if (part == 0 && CACHE_NAME.equals(evt.cacheName())) { System.out.println(">> Received event for 0 partition. [node=" + ignite.name() + ", evt=" + evt + ", thread=" + Thread.currentThread().getName() + ']'); latch.countDown(); + cnt.incrementAndGet(); + if (exchangeCompleted(ignite)) failed = true; - - return false; } return true; From 955ae62887e652d819d3ba19e92921d94e6c8d7e Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Tue, 3 Oct 2017 12:32:30 +0300 Subject: [PATCH 305/357] GG-12775: Backport fixes for NPE in GridPArtitionDemander. Unmute test. --- .../discovery/GridDiscoveryManagerAliveCacheSelfTest.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java index f0c50ebeb3bf1..1c048b7e8466c 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java @@ -169,8 +169,6 @@ public void testAlives() throws Exception { * @throws Exception If failed. */ public void testAlivesClient() throws Exception { - fail("https://issues.apache.org/jira/browse/IGNITE-1583"); - clientMode = true; doTestAlive(); From 92153893ee5056f133337a1bb90451965a3422d4 Mon Sep 17 00:00:00 2001 From: Andrey Kuznetsov Date: Fri, 11 Aug 2017 15:51:31 +0300 Subject: [PATCH 306/357] IGNITE-6009: Fixed IgniteSemaphore test. (cherry picked from commit dadc42f) --- .../IgniteSemaphoreExample.java | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/examples/src/main/java/org/apache/ignite/examples/datastructures/IgniteSemaphoreExample.java b/examples/src/main/java/org/apache/ignite/examples/datastructures/IgniteSemaphoreExample.java index 1c078b02318bd..daa1023e1b56b 100644 --- a/examples/src/main/java/org/apache/ignite/examples/datastructures/IgniteSemaphoreExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/datastructures/IgniteSemaphoreExample.java @@ -37,12 +37,6 @@ public class IgniteSemaphoreExample { /** Number of items for each producer/consumer to produce/consume. */ private static final int OPS_COUNT = 100; - /** Number of producers. */ - private static final int NUM_PRODUCERS = 10; - - /** Number of consumers. */ - private static final int NUM_CONSUMERS = 10; - /** Synchronization semaphore name. */ private static final String SEM_NAME = IgniteSemaphoreExample.class.getSimpleName(); @@ -53,6 +47,12 @@ public class IgniteSemaphoreExample { */ public static void main(String[] args) { try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) { + // Number of producers; should be equal to number of consumers. + // This value should not exceed overall number of compute threads in a cluster, + // otherwise blocking consumer jobs can occupy all the threads leading to deadlock. + int producerConsumerCount = + ignite.configuration().getPublicThreadPoolSize() * ignite.cluster().forServers().nodes().size() / 2; + System.out.println(); System.out.println(">>> Cache atomic semaphore example started."); @@ -66,17 +66,17 @@ public static void main(String[] args) { IgniteSemaphore semaphore = ignite.semaphore(semaphoreName, 0, false, true); // Start consumers on all cluster nodes. - for (int i = 0; i < NUM_CONSUMERS; i++) + for (int i = 0; i < producerConsumerCount; i++) ignite.compute().withAsync().run(new Consumer(semaphoreName)); // Start producers on all cluster nodes. - for (int i = 0; i < NUM_PRODUCERS; i++) + for (int i = 0; i < producerConsumerCount; i++) ignite.compute().withAsync().run(new Producer(semaphoreName)); System.out.println("Master node is waiting for all other nodes to finish..."); // Wait for everyone to finish. - syncSemaphore.acquire(NUM_CONSUMERS + NUM_PRODUCERS); + syncSemaphore.acquire(2 * producerConsumerCount); } System.out.flush(); @@ -159,7 +159,7 @@ public Consumer(String semaphoreName) { System.out.println("Consumer finished [nodeId=" + Ignition.ignite().cluster().localNode().id() + ']'); // Gets the syncing semaphore - IgniteSemaphore sync = Ignition.ignite().semaphore(SEM_NAME, 3, true, true); + IgniteSemaphore sync = Ignition.ignite().semaphore(SEM_NAME, 0, true, true); // Signals the master thread. sync.release(); From 8c17b0efa664f8552b64d16dae2a3a03e2732347 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Thu, 5 Oct 2017 15:18:21 +0300 Subject: [PATCH 307/357] Backport of IGNITE-6545: Failure during Ignite Service.cancel() can break normal shutdown process. --- .../processors/service/GridServiceProcessor.java | 11 ++++++++++- 1 file changed, 10 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 a22ccaad367ff..4ebe1b00d042d 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 @@ -337,7 +337,16 @@ public void onContinuousProcessorStarted(GridKernalContext ctx) throws IgniteChe Service svc = ctx.service(); if (svc != null) - svc.cancel(ctx); + try { + svc.cancel(ctx); + } + catch (Throwable e) { + log.error("Failed to cancel service (ignoring) [name=" + ctx.name() + + ", execId=" + ctx.executionId() + ']', e); + + if (e instanceof Error) + throw e; + } ctx.executor().shutdownNow(); } From 74aa02b8faa2c871da40ff2ad0e93afc019702b9 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Fri, 6 Oct 2017 19:09:50 +0300 Subject: [PATCH 308/357] Backport of IGNITE-6562: Dynamic service deployment should use projection if NodeFilter is not set. --- .../ignite/internal/IgniteServicesImpl.java | 2 +- .../service/GridServiceProcessor.java | 60 +++++++++------- .../GridServiceProcessorAbstractSelfTest.java | 5 +- ...GridServiceProcessorMultiNodeSelfTest.java | 71 ++++++++++++++++++- 4 files changed, 106 insertions(+), 32 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteServicesImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteServicesImpl.java index 58b3a2a96ff1c..0141aea02c918 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteServicesImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteServicesImpl.java @@ -161,7 +161,7 @@ public IgniteServicesImpl(GridKernalContext ctx, ClusterGroupAdapter prj, boolea guard(); try { - saveOrGet(ctx.service().deployAll(cfgs)); + saveOrGet(ctx.service().deployAll(prj, cfgs)); } catch (IgniteCheckedException e) { throw U.convertException(e); 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 4ebe1b00d042d..5e6a0032cdbec 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 @@ -90,6 +90,7 @@ import org.apache.ignite.lang.IgniteBiPredicate; import org.apache.ignite.lang.IgniteCallable; import org.apache.ignite.lang.IgniteFuture; +import org.apache.ignite.lang.IgnitePredicate; import org.apache.ignite.lang.IgniteProductVersion; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.marshaller.Marshaller; @@ -298,15 +299,9 @@ public void onContinuousProcessorStarted(GridKernalContext ctx) throws IgniteChe ServiceConfiguration[] cfgs = ctx.config().getServiceConfiguration(); - if (cfgs != null) { - for (ServiceConfiguration c : cfgs) { - // Deploy only on server nodes by default. - if (c.getNodeFilter() == null) - c.setNodeFilter(ctx.cluster().get().forServers().predicate()); - } - - deployAll(Arrays.asList(cfgs)).get(); - } + if (cfgs != null) + // Deploy only on server nodes by default. + deployAll(Arrays.asList(cfgs), ctx.cluster().get().forServers().predicate()).get(); if (log.isDebugEnabled()) log.debug("Started service processor."); @@ -467,9 +462,8 @@ public IgniteInternalFuture deployMultiple(ClusterGroup prj, String name, Ser cfg.setService(svc); cfg.setTotalCount(totalCnt); cfg.setMaxPerNodeCount(maxPerNodeCnt); - cfg.setNodeFilter(F.alwaysTrue() == prj.predicate() ? null : prj.predicate()); - return deploy(cfg); + return deployAll(prj, Collections.singleton(cfg)); } /** @@ -491,14 +485,17 @@ public IgniteInternalFuture deployKeyAffinitySingleton(String name, Service s cfg.setTotalCount(1); cfg.setMaxPerNodeCount(1); - return deploy(cfg); + // Ignore projection here. + return deployAll(Collections.singleton(cfg), null); } /** * @param cfgs Service configurations. + * @param dfltNodeFilter Default NodeFilter. * @return Configurations to deploy. */ - private PreparedConfigurations prepareServiceConfigurations(Collection cfgs) { + private PreparedConfigurations prepareServiceConfigurations(Collection cfgs, + IgnitePredicate dfltNodeFilter) { List cfgsCp = new ArrayList<>(cfgs.size()); ServicesCompatibilityState state = markCompatibilityStateAsUsed(); @@ -510,6 +507,11 @@ private PreparedConfigurations prepareServiceConfigurations(Collection deployAll(ClusterGroup prj, Collection cfgs) { + if (prj == null) + // Deploy to servers by default if no projection specified. + return deployAll(cfgs, ctx.cluster().get().forServers().predicate()); + else if (prj.predicate() == F.alwaysTrue()) + return deployAll(cfgs, null); + else + // Deploy to predicate nodes by default. + return deployAll(cfgs, prj.predicate()); + } + /** * @param cfgs Service configurations. + * @param dfltNodeFilter Default NodeFilter. * @return Future for deployment. */ - public IgniteInternalFuture deployAll(Collection cfgs) { + private IgniteInternalFuture deployAll(Collection cfgs, IgnitePredicate dfltNodeFilter) { assert cfgs != null; if (!busyLock.enterBusy()) { @@ -580,8 +599,7 @@ public IgniteInternalFuture deployAll(Collection cfgs) } try { - - PreparedConfigurations srvCfg = prepareServiceConfigurations(cfgs); + PreparedConfigurations srvCfg = prepareServiceConfigurations(cfgs, dfltNodeFilter); List cfgsCp = srvCfg.cfgs; @@ -745,16 +763,6 @@ private void writeServiceToCache(GridServiceDeploymentCompoundFuture res, Servic } } - /** - * @param cfg Service configuration. - * @return Future for deployment. - */ - public IgniteInternalFuture deploy(ServiceConfiguration cfg) { - A.notNull(cfg, "cfg"); - - return deployAll(Collections.singleton(cfg)); - } - /** * @return Compatibility state. */ 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 6d91f36aab00b..debe4859d1c19 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 @@ -107,10 +107,7 @@ protected ServiceConfiguration[] services() { /** {@inheritDoc} */ @SuppressWarnings("ConstantConditions") @Override protected void beforeTestsStarted() throws Exception { - assert nodeCount() >= 1; - - for (int i = 0; i < nodeCount(); i++) - startGrid(i); + startGridsMultiThreaded(nodeCount()); } /** {@inheritDoc} */ 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 2958b7cb248c8..57b584f672f56 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 @@ -175,7 +175,7 @@ public void testDeployOnEachNodeButClientUpdateTopology() throws Exception { // 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)); + assertEquals(name, nodeCount() + servers, DummyService.started(name) - DummyService.cancelled(name)); checkCount(name, g.services().serviceDescriptors(), nodeCount() + servers); } @@ -188,6 +188,75 @@ public void testDeployOnEachNodeButClientUpdateTopology() throws Exception { } } + /** + * @throws Exception If failed. + */ + public void testDeployOnEachProjectionNodeUpdateTopology() throws Exception { + // Prestart client node. + Ignite client = startGrid("client", getConfiguration("client").setClientMode(true)); + + try { + final String name = "serviceOnEachProjectionNodeUpdateTopology"; + + Ignite g = randomGrid(); + + int prestartedSrvcs = g.cluster().forClients().nodes().size(); + + CountDownLatch latch = new CountDownLatch(prestartedSrvcs); + + DummyService.exeLatch(name, latch); + + IgniteServices svcs = g.services(g.cluster().forClients()).withAsync(); + + svcs.deployNodeSingleton(name, new DummyService()); + + IgniteFuture fut = svcs.future(); + + info("Deployed service: " + name); + + fut.get(); + + info("Finished waiting for service future: " + name); + + latch.await(); + + // Ensure service is deployed + assertNotNull(client.services().serviceProxy(name, Service.class, false, 2000)); + + assertEquals(name, prestartedSrvcs, DummyService.started(name)); + assertEquals(name, 0, DummyService.cancelled(name)); + + int servers = 2; + + int clients = 2; + + latch = new CountDownLatch(clients); + + DummyService.exeLatch(name, latch); + + startExtraNodes(servers, clients); + + try { + latch.await(); + + waitForDeployment(name, clients); + + // 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, clients + prestartedSrvcs, DummyService.started(name) - DummyService.cancelled(name)); + + checkCount(name, g.services().serviceDescriptors(), clients + prestartedSrvcs); + } + finally { + stopExtraNodes(servers + clients); + } + } + finally { + stopGrid("client"); + } + } + /** * @throws Exception If failed. */ From dd14cf4ac50e11ecaec1ae0e97a6ccd3bec9ef77 Mon Sep 17 00:00:00 2001 From: sboikov Date: Tue, 21 Mar 2017 14:15:52 +0300 Subject: [PATCH 309/357] More info in toString for lock futures. (cherry picked from commit 4322e00) --- .../distributed/dht/colocated/GridDhtColocatedLockFuture.java | 1 + .../processors/cache/distributed/near/GridNearLockFuture.java | 1 + 2 files changed, 2 insertions(+) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java index 79ca1087a3b88..4c37c538d4950 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java @@ -105,6 +105,7 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture private final long threadId; /** Keys to lock. */ + @GridToStringInclude private Collection keys; /** Future ID. */ 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 d3e3a15831bb2..d1f3fac1b5416 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 @@ -100,6 +100,7 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture keys; /** Future ID. */ From d3dee460c4c94d48e65d285ea1ec045f507a5d8c Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Wed, 11 Oct 2017 14:27:21 +0300 Subject: [PATCH 310/357] GG-12920 - Fix: After ExchangeActions backport: CacheGetInsideLockChangingTopologyTest is broken --- .../ignite/internal/processors/cache/ExchangeActions.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java index 0f5f914a2c003..c64efe9fb3976 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java @@ -83,7 +83,7 @@ public boolean cacheStopped(int cacheId) { public boolean cacheStarted(int cacheId) { if (cachesToStart != null) { for (CacheActionData cache : cachesToStart.values()) { - if (cache.desc.cacheId() == cacheId) + if (cache.desc.cacheId() == cacheId && !cache.request().clientStartOnly()) return true; } } From 67b19157b4bb95e66b0009e71a572c2577fb968a Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Fri, 6 Oct 2017 15:26:21 +0300 Subject: [PATCH 311/357] GG-12398: Backport test fix GridCacheAtomicInvalidPartitionHandlingSelfTest.testClockFullAsync. (cherry picked from commit 0519fb7) (cherry picked from commit fd88aef) --- .../GridCacheAtomicInvalidPartitionHandlingSelfTest.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java index 3fd4dd8e586eb..ff98bc1550a9a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java @@ -354,7 +354,8 @@ private void checkRestarts(CacheAtomicWriteOrderMode writeOrder, for (int r = 0; r < 20; r++) { int idx0 = rnd.nextInt(gridCnt - 1) + 1; - stopGrid(idx0); + // No wait for PartitionMapExchange + stopGrid(idx0, false, false); U.sleep(200); From d6232fb1fe079cdb92ef261da3b7aa591a144078 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Thu, 12 Oct 2017 20:32:11 +0300 Subject: [PATCH 312/357] GG-12703: Missed tests added. --- ...iteClientReconnectContinuousProcessorTest.java | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) 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 c86dcabb3a208..0ff5883d11b57 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,7 +23,6 @@ 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; @@ -111,6 +110,20 @@ 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. From 3cc9435fc2b4e25e35dcbb96774868472bc41dbf Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Fri, 13 Oct 2017 12:53:13 +0300 Subject: [PATCH 313/357] GG-12942: Fixed Fix Cache Failover 2 suite tests. Fixed tests: GridCachePartitionedTxSalvageSelfTest.testOptimisticTxSalvageBeforeTimeout GridCachePartitionedTxSalvageSelfTest.testPessimisticcTxSalvageBeforeTimeout --- .../distributed/near/GridCachePartitionedTxSalvageSelfTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java index c2de8c576c597..5a093440ceeb3 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java @@ -235,7 +235,7 @@ private void startTxAndPutKeys(final TransactionConcurrency mode, final boolean * @throws Exception If failed. */ private void stopNodeAndSleep(long timeout) throws Exception { - stopGrid(0); + stopGrid(0, false, false); info("Stopped grid."); From e46a9d1276bd98dc1d3c5f4d45567e1ffd3b169e Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Fri, 13 Oct 2017 13:36:06 +0300 Subject: [PATCH 314/357] GG-12941: Fixed Ignite Cache 5 suite hanging. Fixed GridCachePartitionEvictionDuringReadThroughSelfTest. (cherry picked from commit 3cc9435) --- ...tionEvictionDuringReadThroughSelfTest.java | 68 +++++++++++++------ 1 file changed, 47 insertions(+), 21 deletions(-) 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 index d5351f7e37776..5fef40024e0e4 100644 --- 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 @@ -18,8 +18,8 @@ package org.apache.ignite.internal.processors.cache.distributed; import java.util.LinkedHashSet; +import java.util.Set; 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; @@ -36,6 +36,7 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.configuration.NearCacheConfiguration; import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; @@ -43,6 +44,12 @@ * */ public class GridCachePartitionEvictionDuringReadThroughSelfTest extends GridCommonAbstractTest { + /** Failing key. */ + private static final int FAILING_KEY = 3; + + /** Data read grid index. */ + private static final int DATA_READ_GRID_IDX = 0; + /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(gridName); @@ -75,28 +82,37 @@ public class GridCachePartitionEvictionDuringReadThroughSelfTest extends GridCom * @throws Exception if failed. */ public void testPartitionRent() throws Exception { - startGrid(0); +// fail("https://issues.apache.org/jira/browse/IGNITE-5759"); + + startGrid(DATA_READ_GRID_IDX); final AtomicBoolean done = new AtomicBoolean(); - IgniteInternalFuture fut = GridTestUtils.runMultiThreadedAsync(new Callable() { + IgniteInternalFuture gridAndCacheAccessFut = GridTestUtils.runMultiThreadedAsync(new Callable() { @Override public Integer call() throws Exception { - LinkedHashSet set = new LinkedHashSet<>(); + final Set keysSet = new LinkedHashSet<>(); - set.add(1); - set.add(2); - set.add(3); - set.add(4); - set.add(5); + keysSet.add(1); + keysSet.add(2); + keysSet.add(FAILING_KEY); + keysSet.add(4); + keysSet.add(5); while (!done.get()) { try { - grid(0).cache("config").getAll(set); + grid(DATA_READ_GRID_IDX).cache("config").getAll(keysSet); } - catch (Throwable ignore) { - // No-op. + catch (Throwable ex) { + if (ex instanceof Error) + throw ex; + + if (ex instanceof InterruptedException) + Thread.currentThread().interrupt(); } + + if (Thread.currentThread().isInterrupted()) + throw new IgniteInterruptedCheckedException("Execution of [" + Thread.currentThread().getName() + "] Interrupted. Test is probably timed out"); } return null; @@ -115,11 +131,14 @@ public Integer call() throws Exception { } }); - startFut.get(); - - done.set(true); + try { + startFut.get(); + } + finally { + done.set(true); + } - fut.get(); + gridAndCacheAccessFut.get(); } /** @@ -136,13 +155,10 @@ private static class CacheStoreFactory implements Factory { - /** */ - private CountDownLatch releaseLatch = new CountDownLatch(1); - /** {@inheritDoc} */ @Override public Integer load(Integer key) throws CacheLoaderException { - if (key == 3) - throw new CacheLoaderException(); + if (key == FAILING_KEY) + throw new TestCacheLoaderExpectedException(); return key; } @@ -157,4 +173,14 @@ private static class HangingCacheStore extends CacheStoreAdapter Date: Fri, 13 Oct 2017 13:39:40 +0300 Subject: [PATCH 315/357] GG-12908 Sequential puts fail for keys implementing Externalizable --- .../processors/query/GridQueryProcessor.java | 114 ++--- .../cache/ExternalizableKeyValueTest.java | 461 ++++++++++++++++++ .../testsuites/IgniteCacheTestSuite5.java | 3 + 3 files changed, 517 insertions(+), 61 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ExternalizableKeyValueTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java index 76579fa958811..3af4263ab1430 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java @@ -246,8 +246,6 @@ private void initializeCache(GridCacheContext cctx) throws IgniteCheckedEx boolean keyMustDeserialize = mustDeserializeBinary(keyCls); boolean valMustDeserialize = mustDeserializeBinary(valCls); - boolean keyOrValMustDeserialize = keyMustDeserialize || valMustDeserialize; - if (keyCls == null) keyCls = Object.class; @@ -258,15 +256,15 @@ private void initializeCache(GridCacheContext cctx) throws IgniteCheckedEx desc.tableName(qryEntity.getTableName()); if (binaryEnabled) { - if (!valMustDeserialize && !SQL_TYPES.contains(valCls)) - desc.valueClass(Object.class); - else + if (valMustDeserialize || SQL_TYPES.contains(valCls)) desc.valueClass(valCls); - - if (!keyMustDeserialize && !SQL_TYPES.contains(keyCls)) - desc.keyClass(Object.class); else + desc.valueClass(Object.class); + + if (keyMustDeserialize || SQL_TYPES.contains(keyCls)) desc.keyClass(keyCls); + else + desc.keyClass(Object.class); } else { if (keyCls == null) @@ -284,7 +282,7 @@ private void initializeCache(GridCacheContext cctx) throws IgniteCheckedEx desc.keyTypeName(qryEntity.getKeyType()); desc.valueTypeName(qryEntity.getValueType()); - if (binaryEnabled && keyOrValMustDeserialize) { + if (binaryEnabled && (keyMustDeserialize || valMustDeserialize)) { if (mustDeserializeClss == null) mustDeserializeClss = new ArrayList<>(); @@ -295,17 +293,29 @@ private void initializeCache(GridCacheContext cctx) throws IgniteCheckedEx mustDeserializeClss.add(valCls); } + processMeta( + qryEntity, + desc, + coCtx, + binaryEnabled && !keyMustDeserialize, + binaryEnabled && !valMustDeserialize + ); + TypeId typeId; TypeId altTypeId = null; if (valCls == null || (binaryEnabled && (!valMustDeserialize))) { - processBinaryMeta(qryEntity, desc); - typeId = new TypeId(ccfg.getName(), ctx.cacheObjects().typeId(qryEntity.getValueType())); if (valCls != null) altTypeId = new TypeId(ccfg.getName(), valCls); + } + else { + typeId = new TypeId(ccfg.getName(), valCls); + altTypeId = new TypeId(ccfg.getName(), ctx.cacheObjects().typeId(qryEntity.getValueType())); + } + if (binaryEnabled && !keyMustDeserialize) { if (!cctx.customAffinityMapper() && qryEntity.getKeyType() != null) { // Need to setup affinity key for distributed joins. String affField = ctx.cacheObjects().affinityField(qryEntity.getKeyType()); @@ -315,8 +325,6 @@ private void initializeCache(GridCacheContext cctx) throws IgniteCheckedEx } } else { - processClassMeta(qryEntity, desc, coCtx); - AffinityKeyMapper keyMapper = cctx.config().getAffinityMapper(); if (keyMapper instanceof GridCacheDefaultAffinityKeyMapper) { @@ -326,9 +334,6 @@ private void initializeCache(GridCacheContext cctx) throws IgniteCheckedEx if (affField != null) desc.affinityKey(affField); } - - typeId = new TypeId(ccfg.getName(), valCls); - altTypeId = new TypeId(ccfg.getName(), ctx.cacheObjects().typeId(qryEntity.getValueType())); } addTypeByName(ccfg, desc); @@ -361,8 +366,6 @@ private void initializeCache(GridCacheContext cctx) throws IgniteCheckedEx boolean keyMustDeserialize = mustDeserializeBinary(keyCls); boolean valMustDeserialize = mustDeserializeBinary(valCls); - boolean keyOrValMustDeserialize = keyMustDeserialize || valMustDeserialize; - if (keyCls == null) keyCls = Object.class; @@ -374,15 +377,15 @@ private void initializeCache(GridCacheContext cctx) throws IgniteCheckedEx desc.name(simpleValType); if (binaryEnabled) { - if (!valMustDeserialize && !SQL_TYPES.contains(valCls)) - desc.valueClass(Object.class); - else + if (valMustDeserialize || SQL_TYPES.contains(valCls)) desc.valueClass(valCls); - - if (!keyMustDeserialize && !SQL_TYPES.contains(keyCls)) - desc.keyClass(Object.class); else + desc.valueClass(Object.class); + + if (keyMustDeserialize || SQL_TYPES.contains(keyCls)) desc.keyClass(keyCls); + else + desc.keyClass(Object.class); } else { desc.valueClass(valCls); @@ -392,7 +395,7 @@ private void initializeCache(GridCacheContext cctx) throws IgniteCheckedEx desc.keyTypeName(meta.getKeyType()); desc.valueTypeName(meta.getValueType()); - if (binaryEnabled && keyOrValMustDeserialize) { + if (binaryEnabled && (keyMustDeserialize || valMustDeserialize)) { if (mustDeserializeClss == null) mustDeserializeClss = new ArrayList<>(); @@ -1461,13 +1464,22 @@ private void processBinaryMeta(CacheTypeMetadata meta, TypeDescriptor d) } /** - * Processes declarative metadata for binary object. + * Processes declarative metadata. * * @param qryEntity Declared metadata. * @param d Type descriptor. + * @param coCtx Cache object context. + * @param useBinaryKey Whether a binary key should be used. + * @param useBinaryVal Whether a binary value should be used. * @throws IgniteCheckedException If failed. */ - private void processBinaryMeta(QueryEntity qryEntity, TypeDescriptor d) throws IgniteCheckedException { + private void processMeta( + QueryEntity qryEntity, + TypeDescriptor d, + CacheObjectContext coCtx, + boolean useBinaryKey, + boolean useBinaryVal + ) throws IgniteCheckedException { Map aliases = qryEntity.getAliases(); if (aliases == null) @@ -1500,40 +1512,20 @@ private void processBinaryMeta(QueryEntity qryEntity, TypeDescriptor d) throws I else isKeyField = (hasKeyFields ? keyFields.contains(entry.getKey()) : null); - BinaryProperty prop = buildBinaryProperty(entry.getKey(), U.classForName(entry.getValue(), Object.class, true), - aliases, isKeyField); - - d.addProperty(prop, false); - } - - processIndexes(qryEntity, d); - } - - /** - * Processes declarative metadata for binary object. - * - * @param qryEntity Declared metadata. - * @param d Type descriptor. - * @throws IgniteCheckedException If failed. - */ - private void processClassMeta( - QueryEntity qryEntity, - TypeDescriptor d, - CacheObjectContext coCtx - ) throws IgniteCheckedException { - Map aliases = qryEntity.getAliases(); - - if (aliases == null) - aliases = Collections.emptyMap(); - - for (Map.Entry entry : qryEntity.getFields().entrySet()) { - ClassProperty prop = buildClassProperty( - d.keyClass(), - d.valueClass(), - entry.getKey(), - U.classForName(entry.getValue(), Object.class), - aliases, - coCtx); + boolean isPropBinary = isKeyField != null && isKeyField ? useBinaryKey : useBinaryVal; + + GridQueryProperty prop = isPropBinary ? + buildBinaryProperty( + entry.getKey(), U.classForName(entry.getValue(), Object.class, true), aliases, isKeyField + ) : + buildClassProperty( + d.keyClass(), + d.valueClass(), + entry.getKey(), + U.classForName(entry.getValue(), Object.class), + aliases, + coCtx + ); d.addProperty(prop, false); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ExternalizableKeyValueTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ExternalizableKeyValueTest.java new file mode 100644 index 0000000000000..011553c2e1579 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/ExternalizableKeyValueTest.java @@ -0,0 +1,461 @@ +/* + * 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.Externalizable; +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import java.io.Serializable; +import java.util.Objects; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.query.annotations.QuerySqlField; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.marshaller.Marshaller; +import org.apache.ignite.marshaller.jdk.JdkMarshaller; +import org.apache.ignite.marshaller.optimized.OptimizedMarshaller; +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.common.GridCommonAbstractTest; + +/** + * + */ +public class ExternalizableKeyValueTest extends GridCommonAbstractTest { + /** + * Key class. + */ + public static class Key implements Serializable { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + @QuerySqlField(index = true) + protected int key; + + /** + * Default constructor. + */ + public Key() { + // No-op. + } + + /** + * Constructor. + * + * @param key the key + */ + public Key(final int key) { + this.key = key; + } + + /** {@inheritDoc} */ + @Override + public int hashCode() { + return Objects.hashCode(key); + } + + /** {@inheritDoc} */ + @Override + public boolean equals(final Object o) { + if (o == this) + return true; + + if (o instanceof Key) { + final Key that = (Key)o; + return key == that.key; + } + return false; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return "Key{" + + "key=" + key + + '}'; + } + } + + /** + * {@code Externalizable} Key class. + */ + public static class ExternalizableKey extends Key implements Externalizable { + /** */ + private static final long serialVersionUID = 0L; + + /** + * Default constructor. + */ + public ExternalizableKey() { + // No-op. + } + + /** + * Constructor + * + * @param key the key + */ + public ExternalizableKey(int key) { + super(key); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return "ExternalizableKey{" + + "key=" + key + + "}"; + } + + /** {@inheritDoc} */ + @Override + public void writeExternal(final ObjectOutput out) + throws IOException { + out.writeInt(key); + } + + /** {@inheritDoc} */ + @Override + public void readExternal(final ObjectInput in) throws IOException, ClassNotFoundException { + key = in.readInt(); + } + } + + /** + * Value class. + */ + public static class Value implements Serializable { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + @QuerySqlField(index = true) + protected Integer val; + + /** + * Default constructor. + */ + public Value() { + // No-op. + } + + /** + * Constructor. + * + * @param val Value. + */ + public Value(Integer val) { + this.val = val; + } + + /** + * Returns value. + * + * @return Value. + */ + public Integer getVal() { + return val; + } + + /** + * Sets value. + * + * @param val Value. + */ + public void setVal(Integer val) { + this.val = val; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return "Value{" + + "val=" + val + + '}'; + } + } + + /** + * {@code Externalizable} Value class. + */ + public static class ExternalizableValue extends Value implements Externalizable { + /** */ + private static final long serialVersionUID = 0L; + + /** + * Default constructor. + */ + public ExternalizableValue() { + // No-op. + } + + /** + * Constructor. + * + * @param val Value. + */ + public ExternalizableValue(Integer val) { + super(val); + } + + /** {@inheritDoc} */ + @Override + public void writeExternal(final ObjectOutput out) + throws IOException { + out.writeInt(val); + } + + /** {@inheritDoc} */ + @Override + public void readExternal(final ObjectInput in) throws IOException, ClassNotFoundException { + setVal(in.readInt()); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return "ExternalizableValue{" + + "val=" + val + + "} "; + } + } + + /** + * Marshaller type. + */ + public enum MarshallerType { + /** */ + JDK, + /** */ + OPTIMIZED, + /** */ + BINARY + } + + /** */ + private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + + /** */ + private static final String CACHE_NAME = "cacheWithExternalizedKeyAndOrValue"; + + /** + * Returns a marshaller for the specified type. + * + * @param marshallerType Marshaller type. + * @return Marshaller. {@code null} if {@link MarshallerType#BINARY} type + * is specified or type is unknown. + */ + private static Marshaller getMarshaller(MarshallerType marshallerType) { + switch (marshallerType) { + case JDK: + return new JdkMarshaller(); + case OPTIMIZED: + return new OptimizedMarshaller(); + } + + // Binary marshaller is used by default. + return null; + } + + /** */ + private MarshallerType marshallerType; + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(final String gridName) throws Exception { + IgniteConfiguration cfg = new IgniteConfiguration() + .setGridName(gridName) + .setPeerClassLoadingEnabled(false) + .setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(IP_FINDER)); + + Marshaller marshaller = getMarshaller(marshallerType); + + // By default a binary marshaller is used. + if (marshaller != null) + cfg.setMarshaller(marshaller); + + return cfg; + } + + /** + * @param keyCls Key class. + * @param valCls Val class. + */ + private static CacheConfiguration cacheConfig(Class keyCls, Class valCls) { + return new CacheConfiguration(CACHE_NAME).setIndexedTypes(keyCls, valCls); + } + + /** */ + private void doSequentialPut( + MarshallerType marshallerType, Class keyCls, Class valCls, K key1, V val1, K key2, V val2 + ) throws Exception { + this.marshallerType = marshallerType; + + startGrids(2); + + IgniteCache cache = grid(0).createCache(cacheConfig(keyCls, valCls)); + + cache.put(key1, val1); + cache.put(key2, val2); + + assertEquals(2, cache.size()); + } + + /** */ + public void doSequentialPutWithExternalizableKeyAndPrimitiveValue(MarshallerType marshallerType) throws Exception { + doSequentialPut( + marshallerType, + ExternalizableKey.class, + Integer.class, + new ExternalizableKey(1), + 1, + new ExternalizableKey(2), + 2 + ); + } + + /** */ + public void doSequentialPutWithExternalizableKeyAndExternalizableValue(MarshallerType marshallerType) throws Exception { + doSequentialPut( + marshallerType, + ExternalizableKey.class, + ExternalizableValue.class, + new ExternalizableKey(1), + new ExternalizableValue(111), + new ExternalizableKey(2), + new ExternalizableValue(222) + ); + } + + /** */ + public void doSequentialPutWithExternalizableKeyAndPojoValue(MarshallerType marshallerType) throws Exception { + doSequentialPut( + marshallerType, + ExternalizableKey.class, + Value.class, + new ExternalizableKey(1), + new Value(111), + new ExternalizableKey(2), + new Value(222) + ); + } + + /** */ + public void doSequentialPutWithPrimitiveKeyAndExternalizableValue(MarshallerType marshallerType) throws Exception { + doSequentialPut( + marshallerType, + Integer.class, + ExternalizableValue.class, + 1, + new ExternalizableValue(111), + 2, + new ExternalizableValue(222) + ); + } + + /** */ + public void doSequentialPutWithPojoKeyAndExternalizableValue(MarshallerType marshallerType) throws Exception { + doSequentialPut( + marshallerType, + Key.class, + ExternalizableValue.class, + new Key(1), + new ExternalizableValue(111), + new Key(2), + new ExternalizableValue(222) + ); + } + + /** */ + public void testSequentialPutWithExternalizableKeyAndPrimitiveValueAndBinaryMarshaller() throws Exception { + doSequentialPutWithExternalizableKeyAndPrimitiveValue(MarshallerType.BINARY); + } + + /** */ + public void testSequentialPutWithExternalizableKeyAndPrimitiveValueAndJdkMarshaller() throws Exception { + doSequentialPutWithExternalizableKeyAndPrimitiveValue(MarshallerType.JDK); + } + + /** */ + public void testSequentialPutWithExternalizableKeyAndPrimitiveValueAndOptimizedMarshaller() throws Exception { + doSequentialPutWithExternalizableKeyAndPrimitiveValue(MarshallerType.OPTIMIZED); + } + + /** */ + public void testSequentialPutWithExternalizableKeyAndExternalizableValueAndBinaryMarshaller() throws Exception { + doSequentialPutWithExternalizableKeyAndExternalizableValue(MarshallerType.BINARY); + } + + /** */ + public void testSequentialPutWithExternalizableKeyAndExternalizableValueAndJdkMarshaller() throws Exception { + doSequentialPutWithExternalizableKeyAndExternalizableValue(MarshallerType.JDK); + } + + /** */ + public void testSequentialPutWithExternalizableKeyAndExternalizableValueAndOptimizedMarshaller() throws Exception { + doSequentialPutWithExternalizableKeyAndExternalizableValue(MarshallerType.OPTIMIZED); + } + + /** */ + public void testSequentialPutWithExternalizableKeyAndPojoValueAndBinaryMarshaller() throws Exception { + doSequentialPutWithExternalizableKeyAndPojoValue(MarshallerType.BINARY); + } + + /** */ + public void testSequentialPutWithExternalizableKeyAndPojoValueAndJdkMarshaller() throws Exception { + doSequentialPutWithExternalizableKeyAndPojoValue(MarshallerType.JDK); + } + + /** */ + public void testSequentialPutWithExternalizableKeyAndPojoValueAndOptimizedMarshaller() throws Exception { + doSequentialPutWithExternalizableKeyAndPojoValue(MarshallerType.OPTIMIZED); + } + + /** */ + public void testSequentialPutWithPrimitiveKeyAndExternalizableValueAndBinaryMarshaller() throws Exception { + doSequentialPutWithPrimitiveKeyAndExternalizableValue(MarshallerType.BINARY); + } + + /** */ + public void testSequentialPutWithPrimitiveKeyAndExternalizableValueAndJdkMarshaller() throws Exception { + doSequentialPutWithPrimitiveKeyAndExternalizableValue(MarshallerType.JDK); + } + + /** */ + public void testSequentialPutWithPrimitiveKeyAndExternalizableValueAndOptimizedMarshaller() throws Exception { + doSequentialPutWithPrimitiveKeyAndExternalizableValue(MarshallerType.OPTIMIZED); + } + + /** */ + public void testSequentialPutWithPojoKeyAndExternalizableValueAndBinaryMarshaller() throws Exception { + doSequentialPutWithPojoKeyAndExternalizableValue(MarshallerType.BINARY); + } + + /** */ + public void testSequentialPutWithPojoKeyAndExternalizableValueAndJdkMarshaller() throws Exception { + doSequentialPutWithPojoKeyAndExternalizableValue(MarshallerType.JDK); + } + + /** */ + public void testSequentialPutWithPojoKeyAndExternalizableValueAndOptimizedMarshaller() throws Exception { + doSequentialPutWithPojoKeyAndExternalizableValue(MarshallerType.OPTIMIZED); + } +} 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 bfcc374ff3fec..160f85d0a1387 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 @@ -23,6 +23,7 @@ import org.apache.ignite.internal.processors.cache.CacheRebalancingSelfTest; import org.apache.ignite.internal.processors.cache.CacheSerializableTransactionsTest; import org.apache.ignite.internal.processors.cache.EntryVersionConsistencyReadThroughTest; +import org.apache.ignite.internal.processors.cache.ExternalizableKeyValueTest; import org.apache.ignite.internal.processors.cache.GridCacheOffHeapCleanupTest; import org.apache.ignite.internal.processors.cache.GridCacheSwapSpaceSpiConsistencySelfTest; import org.apache.ignite.internal.processors.cache.IgniteCachePutStackOverflowSelfTest; @@ -72,6 +73,8 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(GridCachePartitionEvictionDuringReadThroughSelfTest.class); + suite.addTestSuite(ExternalizableKeyValueTest.class); + return suite; } } From 11e3d2c39adcbe779c41316877903500c611921c Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Fri, 13 Oct 2017 18:12:55 +0300 Subject: [PATCH 316/357] GG-12941: Mute test CacheLateAffinityAssignmentTest.testNoForceKeysRequests --- .../cache/distributed/CacheLateAffinityAssignmentTest.java | 2 ++ 1 file changed, 2 insertions(+) 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 761a7f0979a1d..4dbcd327a9437 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 @@ -1768,6 +1768,8 @@ public void testServiceReassign() throws Exception { * @throws Exception If failed. */ public void testNoForceKeysRequests() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-5510"); + cacheC = new IgniteClosure() { @Override public CacheConfiguration[] apply(String s) { return null; From 5c35a58788807be8f1b7d79e6174b27f81d7a1f6 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Fri, 13 Oct 2017 18:25:50 +0300 Subject: [PATCH 317/357] GG-12941: Fixed hanged test CacheLateAffinityAssignmentTest.testJoinExchangeBecomeCoordinator. --- .../cache/distributed/CacheLateAffinityAssignmentTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 4dbcd327a9437..cf32c353d4d55 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 @@ -1103,7 +1103,7 @@ public void testJoinExchangeBecomeCoordinator() throws Exception { U.sleep(5000); for (int i = 0; i < NODES; i++) - stopGrid(i); + stopGrid(i, false, false); return null; } From 5b50c724ccb3732007416627a554252e781290a6 Mon Sep 17 00:00:00 2001 From: Konstantin Dudkov Date: Tue, 17 Oct 2017 11:43:30 +0300 Subject: [PATCH 318/357] IGNITE-6581 fix client deadlock --- .../ignite/spi/discovery/tcp/ClientImpl.java | 3 +- .../TcpDiscoveryClientConnectionFailTest.java | 161 ++++++++++++++++++ .../IgniteSpiDiscoverySelfTestSuite.java | 3 + 3 files changed, 166 insertions(+), 1 deletion(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryClientConnectionFailTest.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 6c12ceeb364bc..ce4f1b7fe4b9a 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 @@ -1704,7 +1704,8 @@ else if (msg == SPI_RECONNECT_FAILED) { onDisconnected(); - notifyDiscovery(EVT_CLIENT_NODE_DISCONNECTED, topVer, locNode, allVisibleNodes()); + if (joinLatch.getCount() == 0) + notifyDiscovery(EVT_CLIENT_NODE_DISCONNECTED, topVer, locNode, allVisibleNodes()); } UUID newId = UUID.randomUUID(); diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryClientConnectionFailTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryClientConnectionFailTest.java new file mode 100644 index 0000000000000..c77f1b7f62b45 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryClientConnectionFailTest.java @@ -0,0 +1,161 @@ +/* + * 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.io.IOException; +import java.io.OutputStream; +import java.net.Socket; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException; +import org.apache.ignite.internal.IgniteInternalFuture; +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.TcpDiscoveryClientHeartbeatMessage; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeAddFinishedMessage; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeAddedMessage; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * + */ +public class TcpDiscoveryClientConnectionFailTest extends GridCommonAbstractTest { + /** */ + private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); + /** */ + private static volatile boolean client; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + if (gridName.endsWith("0")) + cfg.setFailureDetectionTimeout(100); + + TestDiscoverySpi spi = new TestDiscoverySpi(gridName); + spi.setClientReconnectDisabled(false); + spi.setIpFinder(ipFinder); + + spi.setJoinTimeout(20000); + spi.setMaxMissedClientHeartbeats(2); + spi.failureDetectionTimeoutEnabled(true); + + cfg.setDiscoverySpi(spi); + + cfg.setClientMode(client); + + return cfg; + } + + /** */ + public void testClientFailsIfCantConnect() throws Exception { + client = false; + + startGrid(0); + + client = true; + + IgniteInternalFuture fut = multithreadedAsync(new Runnable() { + @Override public void run() { + try { + startGrid(1); + } + catch (Exception ignored) { + } + + } + }, 1, "client start thread"); + + try { + fut.get(10000); + } + catch (IgniteFutureTimeoutCheckedException e) { + fail("client node start hangs (possible deadlock detected)"); + } + + } + + /** + * + */ + private static class TestDiscoverySpi extends TcpDiscoverySpi { + /** */ + private final String name; + /** */ + private boolean block; + + /** */ + TestDiscoverySpi(String name) { + this.name = name; + } + + /** + * @param msg Message. + * @return {@code False} if should not further process message. + * @throws IOException If failed. + */ + private boolean onMessage(TcpDiscoveryAbstractMessage msg) throws IOException { + if (msg == null) + return true; + + boolean blocked = false; + + // we block first connection attempt only + if (msg instanceof TcpDiscoveryNodeAddedMessage) + block = !block; + + if (msg instanceof TcpDiscoveryClientHeartbeatMessage && block) + blocked = true; + + if (msg instanceof TcpDiscoveryNodeAddFinishedMessage && block) + blocked = true; + + return !blocked; + } + + /** {@inheritDoc} */ + @Override protected void writeToSocket(Socket sock, TcpDiscoveryAbstractMessage msg, byte[] data, + long timeout) throws IOException { + + if (onMessage(msg)) + super.writeToSocket(sock, msg, data, timeout); + } + + /** {@inheritDoc} */ + @Override protected void writeToSocket(TcpDiscoveryAbstractMessage msg, Socket sock, int res, + long timeout) throws IOException { + if (onMessage(msg)) + super.writeToSocket(msg, sock, res, timeout); + } + + /** {@inheritDoc} */ + @Override protected void writeToSocket(Socket sock, TcpDiscoveryAbstractMessage msg, + long timeout) throws IOException, IgniteCheckedException { + if (onMessage(msg)) + super.writeToSocket(sock, msg, timeout); + } + + /** {@inheritDoc} */ + @Override protected void writeToSocket(Socket sock, OutputStream out, TcpDiscoveryAbstractMessage msg, + long timeout) throws IOException, IgniteCheckedException { + if (onMessage(msg)) + super.writeToSocket(sock, out, msg, timeout); + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java index c506ca7e95bf2..aa9ac93078947 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java @@ -26,6 +26,7 @@ import org.apache.ignite.spi.discovery.tcp.TcpClientDiscoverySpiFailureTimeoutSelfTest; import org.apache.ignite.spi.discovery.tcp.TcpClientDiscoverySpiMulticastTest; import org.apache.ignite.spi.discovery.tcp.TcpClientDiscoverySpiSelfTest; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoveryClientConnectionFailTest; import org.apache.ignite.spi.discovery.tcp.TcpDiscoveryMarshallerCheckSelfTest; import org.apache.ignite.spi.discovery.tcp.TcpDiscoveryMultiThreadedTest; import org.apache.ignite.spi.discovery.tcp.TcpDiscoveryNodeAttributesUpdateOnReconnectTest; @@ -96,6 +97,8 @@ public static TestSuite suite() throws Exception { suite.addTest(new TestSuite(IgniteClientConnectTest.class)); suite.addTest(new TestSuite(IgniteClientReconnectMassiveShutdownTest.class)); + suite.addTest(new TestSuite(TcpDiscoveryClientConnectionFailTest.class)); + // SSL. suite.addTest(new TestSuite(TcpDiscoverySslSelfTest.class)); suite.addTest(new TestSuite(TcpDiscoverySslSecuredUnsecuredTest.class)); From 0ed98c02d3253d3dafe41a1b4f2d148b8cc0eb50 Mon Sep 17 00:00:00 2001 From: nikolay_tikhonov Date: Tue, 17 Oct 2017 18:19:11 +0300 Subject: [PATCH 319/357] GG-12881 Possible live-lock in discovery worker. Signed-off-by: nikolay_tikhonov --- .../GridDhtPartitionsExchangeFuture.java | 78 +++++++++++++++---- 1 file changed, 61 insertions(+), 17 deletions(-) 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 13fdb30137f58..3578af40c1f36 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 @@ -66,6 +66,7 @@ import org.apache.ignite.internal.processors.timeout.GridTimeoutObjectAdapter; import org.apache.ignite.internal.util.future.GridCompoundFuture; import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.lang.GridPlainRunnable; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.CI1; @@ -212,6 +213,13 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter discoThread = new ThreadLocal(){ + @Override protected Boolean initialValue() { + return false; + } + }; + /** * Dummy future created to trigger reassignments if partition topology changed while preloading. * @@ -1072,6 +1080,30 @@ private void sendPartitions(ClusterNode oldestNode) { } } + /** + * @param nodeIds Node ids. + * @param req Partition single request. + */ + private void sendPartitionRequest(Set nodeIds, GridDhtPartitionsSingleRequest req) { + for (UUID nodeId : nodeIds) { + if (log.isDebugEnabled()) + log.debug("Sending message: [node=" + nodeId + ", req=" + req + ']'); + + try { + // It is possible that some nodes finished exchange with previous coordinator. + cctx.io().send(nodeId, req, SYSTEM_POOL); + } + catch (ClusterTopologyCheckedException ignored) { + if (log.isDebugEnabled()) + log.debug("Node left during partition exchange [nodeId=" + nodeId + + ", exchId=" + exchId + ']'); + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to request partitions from node: " + nodeId, e); + } + } + } + /** {@inheritDoc} */ @Override public boolean onDone(@Nullable AffinityTopologyVersion res, @Nullable Throwable err) { boolean realExchange = !dummy && !forcePreload; @@ -1742,7 +1774,11 @@ private void onDiscoveryEvent(IgniteRunnable c) { assert discoEvts.isEmpty() : discoEvts; } + discoThread.set(true); + c.run(); + + discoThread.set(false); } /** @@ -1785,6 +1821,9 @@ public void onNodeLeft(final ClusterNode node) { try { onDiscoveryEvent(new IgniteRunnable() { @Override public void run() { + // true if method was invoke from disco thread, otherwise false. + boolean discoThread = GridDhtPartitionsExchangeFuture.this.discoThread.get(); + if (isDone() || !enterBusy()) return; @@ -1793,7 +1832,7 @@ public void onNodeLeft(final ClusterNode node) { boolean allReceived = false; Set reqFrom = null; - ClusterNode crd0; + final ClusterNode crd0; discoCache.updateAlives(node); @@ -1849,22 +1888,19 @@ public void onNodeLeft(final ClusterNode node) { } if (crdChanged && reqFrom != null) { - GridDhtPartitionsSingleRequest req = new GridDhtPartitionsSingleRequest(exchId); - - for (UUID nodeId : reqFrom) { - try { - // It is possible that some nodes finished exchange with previous coordinator. - cctx.io().send(nodeId, req, SYSTEM_POOL); - } - catch (ClusterTopologyCheckedException ignored) { - if (log.isDebugEnabled()) - log.debug("Node left during partition exchange [nodeId=" + nodeId + - ", exchId=" + exchId + ']'); - } - catch (IgniteCheckedException e) { - U.error(log, "Failed to request partitions from node: " + nodeId, e); - } + final GridDhtPartitionsSingleRequest req = new GridDhtPartitionsSingleRequest(exchId); + + if (discoThread) { + final Set reqFrom0 = reqFrom; + + cctx.kernalContext().closure().runLocalSafe(new GridPlainRunnable() { + @Override public void run() { + sendPartitionRequest(reqFrom0, req); + } + }, true); } + else + sendPartitionRequest(reqFrom, req); } for (Map.Entry m : singleMsgs.entrySet()) @@ -1872,7 +1908,15 @@ public void onNodeLeft(final ClusterNode node) { } else { if (crdChanged) { - sendPartitions(crd0); + if (discoThread) { + cctx.kernalContext().closure().runLocalSafe(new GridPlainRunnable() { + @Override public void run() { + sendPartitions(crd0); + } + }, true); + } + else + sendPartitions(crd0); for (Map.Entry m : fullMsgs.entrySet()) processMessage(m.getKey(), m.getValue()); From 656d458bf3e1e70accf7c56b650ffbdc2f0ab6ea Mon Sep 17 00:00:00 2001 From: vsisko Date: Wed, 18 Oct 2017 20:39:13 +0700 Subject: [PATCH 320/357] WC-322 Annotations for REST serialization. --- .../apache/ignite/internal/visor/file/VisorFileBlockTask.java | 3 ++- .../org/apache/ignite/internal/visor/log/VisorLogFile.java | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/file/VisorFileBlockTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/file/VisorFileBlockTask.java index 58a25eed8b673..8b73d20cec954 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/file/VisorFileBlockTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/file/VisorFileBlockTask.java @@ -23,6 +23,7 @@ import java.net.URISyntaxException; import java.net.URL; import java.nio.file.NoSuchFileException; +import org.apache.ignite.internal.LessNamingBean; import org.apache.ignite.internal.processors.task.GridInternal; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; @@ -50,7 +51,7 @@ public class VisorFileBlockTask extends VisorOneNodeTask Date: Mon, 23 Oct 2017 12:50:15 +0300 Subject: [PATCH 321/357] IGNITE-6071 White list of exceptions to suppress in createTcpClient. --- .../tcp/TcpCommunicationSpi.java | 3 +- .../ignite/spi/discovery/tcp/ServerImpl.java | 2 + ...tConnectAfterCommunicationFailureTest.java | 156 ++++++++++++++++++ .../IgniteClientReconnectTestSuite.java | 4 +- 4 files changed, 163 insertions(+), 2 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/IgniteClientConnectAfterCommunicationFailureTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java index de1f76d9f1b61..dffa181058c56 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 @@ -3144,7 +3144,8 @@ else if (X.hasCause(e, SocketTimeoutException.class)) } } - if (X.hasCause(errs, ConnectException.class, HandshakeException.class)) + if (!X.hasCause(errs, SocketTimeoutException.class, HandshakeTimeoutException.class, + IgniteSpiOperationTimeoutException.class)) throw errs; } 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 7816458407a11..640dcec85b0b0 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 @@ -724,6 +724,8 @@ else if (!spi.failureDetectionTimeoutEnabled() && reconCnt == spi.getReconnectCo finally { U.closeQuiet(sock); } + + U.sleep(200); } } catch (Throwable t) { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientConnectAfterCommunicationFailureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientConnectAfterCommunicationFailureTest.java new file mode 100644 index 0000000000000..301d5f24e4922 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientConnectAfterCommunicationFailureTest.java @@ -0,0 +1,156 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal; + +import java.util.Arrays; +import java.util.UUID; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.util.nio.GridCommunicationClient; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Tests client to be able restore connection to cluster on subsequent attempts after communication problems. + */ +public class IgniteClientConnectAfterCommunicationFailureTest extends GridCommonAbstractTest { + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setNetworkTimeout(500); + cfg.setCommunicationSpi(new TcpCommunicationSpi(gridName.contains("block"))); + + if (gridName.contains("client")) { + cfg.setClientMode(true); + } + + return cfg; + } + + /** + * @throws Exception If failed. + */ + public void testClientReconnects() throws Exception { + Ignite srv1 = startGrid("server1"); + Ignite srv2 = startGrid("server2"); + startGrid("client-block"); + + assertEquals(1, srv2.cluster().forClients().nodes().size()); + assertEquals(1, srv1.cluster().forClients().nodes().size()); + } + + /** + * @throws Exception If failed. + */ + public void testClientThreadsSuspended() throws Exception { + Ignite srv1 = startGrid("server1"); + Ignite srv2 = startGrid("server2"); + Ignite client = startGrid("client"); + + boolean blockedAnything = false; + + for (Thread thread : Thread.getAllStackTraces().keySet()) { + if (thread.getName().contains("%client%")) { + thread.suspend(); + blockedAnything = true; + } + } + + Thread.sleep(10000); + + for (Thread thread : Thread.getAllStackTraces().keySet()) { + if (thread.getName().contains("%client%")) + thread.resume(); + } + + for (int j = 0; j < 10; j++) { + boolean topOk = true; + + for (Ignite node : Arrays.asList(srv1, srv2, client)) { + if (node.cluster().nodes().size() != 3) { + U.warn(log, "Grid size is incorrect (will re-run check in 1000 ms) " + + "[name=" + node.name() + ", size=" + node.cluster().nodes().size() + ']'); + + topOk = false; + + break; + } + } + + if (topOk) + return; + else + Thread.sleep(1000); + } + + assertTrue(blockedAnything); + assertEquals(1, srv2.cluster().forClients().nodes().size()); + assertEquals(1, srv1.cluster().forClients().nodes().size()); + } + + /** + * Will never connect with the first node id, normal operation after. + */ + private class TcpCommunicationSpi extends org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi { + /** + * Whether this instance should actually block. + */ + private final boolean isBlocking; + + /** + * Local node ID that is prevented from creating connections. + */ + private volatile UUID blockedNodeId = null; + + /** + * + * @param isBlocking Whether this instance should actually block. + */ + public TcpCommunicationSpi(boolean isBlocking) { + this.isBlocking = isBlocking; + } + + /** {@inheritDoc} */ + @Override protected GridCommunicationClient createTcpClient(ClusterNode node, int connIdx) + throws IgniteCheckedException { + if (blockHandshakeOnce(getLocalNode().id())) { + throw new IgniteCheckedException("Node is blocked"); + } + + return super.createTcpClient(node, connIdx); + } + + /** Check if this connection is blocked. */ + private boolean blockHandshakeOnce(UUID nodeId) { + if (isBlocking && (blockedNodeId == null || blockedNodeId.equals(nodeId))) { + blockedNodeId = nodeId; + return true; + } + return false; + } + } +} 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 03d3fe230c296..d0e907cde01f8 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 @@ -18,6 +18,7 @@ package org.apache.ignite.testsuites; import junit.framework.TestSuite; +import org.apache.ignite.internal.IgniteClientConnectAfterCommunicationFailureTest; import org.apache.ignite.internal.IgniteClientReconnectApiExceptionTest; import org.apache.ignite.internal.IgniteClientReconnectAtomicsTest; import org.apache.ignite.internal.IgniteClientReconnectBinaryContexTest; @@ -43,6 +44,7 @@ public class IgniteClientReconnectTestSuite extends TestSuite { public static TestSuite suite() throws Exception { TestSuite suite = new TestSuite("Ignite Client Reconnect Test Suite"); + suite.addTestSuite(IgniteClientConnectAfterCommunicationFailureTest.class); suite.addTestSuite(IgniteClientReconnectStopTest.class); suite.addTestSuite(IgniteClientReconnectApiExceptionTest.class); suite.addTestSuite(IgniteClientReconnectDiscoveryStateTest.class); @@ -59,4 +61,4 @@ public static TestSuite suite() throws Exception { return suite; } -} \ No newline at end of file +} From d39e17fd7af5674a17ef9e81bc5936baabc782dc Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Fri, 27 Oct 2017 11:31:45 +0300 Subject: [PATCH 322/357] GG-12887 - Add custom cache validator. --- .../apache/ignite/IgniteSystemProperties.java | 6 + .../processors/cache/GridCacheAdapter.java | 35 ++ .../GridCachePartitionExchangeManager.java | 13 + .../processors/cache/GridCacheProcessor.java | 33 ++ .../processors/cache/IgniteCacheProxy.java | 7 + .../dht/GridPartitionedGetFuture.java | 10 + .../dht/GridPartitionedSingleGetFuture.java | 10 + .../dht/atomic/GridDhtAtomicCache.java | 13 + .../GridDhtPartitionsExchangeFuture.java | 57 +- .../distributed/near/GridNearGetFuture.java | 8 + .../GridCacheDistributedQueryManager.java | 11 + .../cache/transactions/IgniteTxHandler.java | 66 +++ .../cluster/GridUpdateNotifier.java | 6 - ...iteTopologyValidatorAbstractCacheTest.java | 9 +- ...eTopologyValidatorAbstractTxCacheTest.java | 10 +- .../GridLostPartitionRebalanceTest.java | 11 +- .../GridLostPartitionValidationTest.java | 536 ++++++++++++++++++ .../testsuites/IgniteCacheTestSuite3.java | 2 + .../processors/query/h2/IgniteH2Indexing.java | 37 +- .../query/h2/sql/GridSqlQueryParser.java | 45 ++ .../h2/twostep/GridMapQueryExecutor.java | 50 +- .../GridLostPartitionValidationQueryTest.java | 300 ++++++++++ .../IgniteCacheWithIndexingTestSuite.java | 3 + 23 files changed, 1250 insertions(+), 28 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridLostPartitionValidationTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridLostPartitionValidationQueryTest.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 f570e1246aa1d..3c5a5a6c5d44b 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -23,6 +23,7 @@ import java.util.Map; import java.util.Properties; import javax.net.ssl.HostnameVerifier; +import org.apache.ignite.lang.IgniteClosure; import org.apache.ignite.lang.IgnitePredicate; import org.apache.ignite.marshaller.optimized.OptimizedMarshaller; import org.jetbrains.annotations.Nullable; @@ -600,6 +601,11 @@ public final class IgniteSystemProperties { /** Ignite marshaller cache reread pause. */ public static final String IGNITE_MARSHALLER_CACHE_REREAD_PAUSE = "IGNITE_MARSHALLER_CACHE_REREAD_PAUSE"; + /** Class name of the closure {@link IgniteClosure} that + * will be invoked once per topology and validates cache. If it returns or throws exception, + * that means cache is invalid.*/ + public static final String IGNITE_CACHE_VALIDATOR = "IGNITE_CACHE_VALIDATOR"; + /** * Enforces singleton. */ 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 2fc6359c33b5b..ca1b8c90163d4 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 @@ -86,6 +86,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheAdapter; 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.distributed.dht.GridDhtTopologyFuture; 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; @@ -728,6 +729,8 @@ else if (modes.heap) { if (modes.primary || modes.backup) { AffinityTopologyVersion topVer = ctx.affinity().affinityTopologyVersion(); + validateCache(topVer); + GridCacheSwapManager swapMgr = ctx.isNear() ? ctx.near().dht().context().swap() : ctx.swap(); if (modes.swap) @@ -773,6 +776,8 @@ public String toString() { if (!ctx.isLocal()) { AffinityTopologyVersion topVer = ctx.affinity().affinityTopologyVersion(); + validateCache(topVer); + int part = ctx.affinity().partition(cacheKey); boolean nearKey; @@ -853,6 +858,24 @@ public String toString() { } } + /** + * @throws IgniteCheckedException If validation failed. + */ + private void validateCache(AffinityTopologyVersion topVer) throws IgniteCheckedException { + if (isLocal()) + return; + + GridDhtTopologyFuture fut = ctx.shared().exchange().exchangeFuture(topVer); + + if (fut == null) + fut = ctx.topologyVersionFuture(); + + Throwable exc = fut.validateCache(ctx); + + if (exc != null) + throw new IgniteCheckedException(exc); + } + /** * @param key Key. * @param heap Read heap flag. @@ -1873,6 +1896,18 @@ protected final IgniteInternalFuture> getAllAsync0( int keysSize = keys.size(); + if (!isLocal()) { + GridDhtTopologyFuture fut = ctx.shared().exchange().exchangeFuture(topVer); + + if (fut == null) + fut = ctx.topologyVersionFuture(); + + Throwable exc = fut.validateCache(ctx); + + if (exc != null) + return new GridFinishedFuture<>(exc); + } + final Map map = keysSize == 1 ? (Map)new IgniteBiTuple<>() : U.newHashMap(keysSize); 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 a41004bc771c7..183debd4148d1 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 @@ -698,6 +698,19 @@ public List exchangeFutures() { return exchFuts.values(); } + /** + * @param topVer Topology version. + * @return Exchange future. + */ + @Nullable public GridDhtPartitionsExchangeFuture exchangeFuture(AffinityTopologyVersion topVer) { + for (GridDhtPartitionsExchangeFuture fut : exchFuts.values()) { + if (fut.topologyVersion().equals(topVer)) + return fut; + } + + return null; + } + /** * @return {@code True} if pending future queue is empty. */ 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 488cc2dd37e23..8a041b2a1a4ab 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 @@ -42,6 +42,7 @@ import javax.management.MBeanServer; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.cache.CacheAtomicWriteOrderMode; import org.apache.ignite.cache.CacheExistsException; import org.apache.ignite.cache.CacheMemoryMode; @@ -123,6 +124,7 @@ import org.jetbrains.annotations.Nullable; import static org.apache.ignite.IgniteSystemProperties.IGNITE_CACHE_REMOVED_ENTRIES_TTL; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_CACHE_VALIDATOR; import static org.apache.ignite.IgniteSystemProperties.IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK; import static org.apache.ignite.IgniteSystemProperties.getBoolean; import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC; @@ -199,6 +201,9 @@ public class GridCacheProcessor extends GridProcessorAdapter { /** */ private Map clientReconnectReqs; + /** */ + private IgniteClosure validator; + /** * @param ctx Kernal context. */ @@ -319,6 +324,27 @@ else if (cfg.getWriteSynchronizationMode() != FULL_SYNC && throw new IgniteCheckedException("Cannot set both cache writer factory and cache store factory " + "for cache: " + U.maskName(cfg.getName())); } + + try { + String validatorClsName = IgniteSystemProperties.getString(IGNITE_CACHE_VALIDATOR, null); + + if (validatorClsName != null) { + ClassLoader ldr = U.resolveClassLoader(cacheObjCtx.kernalContext().config()); + + Class validatorCls = ldr.loadClass(validatorClsName); + + if (IgniteClosure.class.isAssignableFrom(validatorCls)) + validator = (IgniteClosure)validatorCls.newInstance(); + + if (validator != null) + cacheObjCtx.kernalContext().resource().injectGeneric(validator); + } + } + catch (IllegalAccessException | InstantiationException | ClassNotFoundException e) { + throw new IgniteCheckedException("Unable to instantiate cache validator defined by " + + IGNITE_CACHE_VALIDATOR + " system property", e); + } + } /** @@ -3868,6 +3894,13 @@ private static String unmaskNull(String name) { return name == NULL_NAME ? null : name; } + /** + * Cache validator. + */ + public IgniteClosure cacheValidator() { + return validator; + } + /** * */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java index f806d05ab919c..2234c33c430c0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java @@ -855,6 +855,13 @@ private void validate(Query qry) { (qry instanceof SqlQuery || qry instanceof SqlFieldsQuery || qry instanceof TextQuery)) throw new CacheException("Failed to execute query. Add module 'ignite-indexing' to the classpath " + "of all Ignite nodes."); + + if (!ctx.isLocal()) { + Throwable exc = ctx.topologyVersionFuture().validateCache(ctx); + + if (exc != null) + throw new CacheException(exc); + } } /** {@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 60688cc8c0fb2..4dcb300e92fb1 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 @@ -245,6 +245,16 @@ private void map( return; } + GridDhtTopologyFuture exchFut = cctx.topologyVersionFuture(); + + Throwable exc = exchFut.validateCache(cctx); + + if (exc != null) { + onDone(exc); + + return; + } + Map> mappings = U.newHashMap(cacheNodes.size()); final int keysSize = keys.size(); 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 2ed5c60128e30..b667a10f889d2 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 @@ -207,6 +207,16 @@ public void init() { */ @SuppressWarnings("unchecked") private void map(AffinityTopologyVersion topVer) { + GridDhtTopologyFuture exchFut = cctx.topologyVersionFuture(); + + Throwable exc = exchFut.validateCache(cctx); + + if (exc != null) { + onDone(exc); + + return; + } + ClusterNode node = mapKeyToNode(topVer); if (node == null) { 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 ad16f1f52fc74..574a5677671cc 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 @@ -67,6 +67,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtFuture; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtInvalidPartitionException; 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.distributed.dht.GridPartitionedGetFuture; import org.apache.ignite.internal.processors.cache.distributed.dht.GridPartitionedSingleGetFuture; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloader; @@ -1851,6 +1852,18 @@ private void updateAllAsyncInternal0( IgniteCacheExpiryPolicy expiry = null; try { + if (!isLocal()) { + GridDhtTopologyFuture fut = ctx.shared().exchange().exchangeFuture(req.topologyVersion()); + + if (fut == null) + fut = ctx.topologyVersionFuture(); + + Throwable exc = fut.validateCache(ctx); + + if (exc != null) + throw new IgniteException(exc); + } + // If batch store update is enabled, we need to lock all entries. // First, need to acquire locks on cache entries, then check filter. List locked = lockEntries(req, req.topologyVersion()); 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 3578af40c1f36..59084f542566c 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 @@ -64,6 +64,7 @@ import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.timeout.GridTimeoutObjectAdapter; +import org.apache.ignite.internal.util.IgniteUtils; import org.apache.ignite.internal.util.future.GridCompoundFuture; import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.lang.GridPlainRunnable; @@ -76,6 +77,7 @@ import org.apache.ignite.internal.util.typedef.internal.LT; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteClosure; import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.lang.IgniteRunnable; import org.jetbrains.annotations.Nullable; @@ -186,6 +188,9 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter cacheValidRes; + /** Custom cache validation results. */ + private volatile Map customCacheValidRes; + /** Skip preload flag. */ private boolean skipPreload; @@ -471,6 +476,15 @@ public void init() throws IgniteInterruptedCheckedException { skipPreload = cctx.kernalContext().clientNode(); + if (log.isInfoEnabled()) { + log.info("Started exchange init [topVer=" + topologyVersion() + + ", crd=" + crdNode + + ", evt=" + IgniteUtils.gridEventName(discoEvt.type()) + + ", evtNode=" + discoEvt.eventNode().id() + + ", customEvt=" + (discoEvt.type() == EVT_DISCOVERY_CUSTOM_EVT + ? ((DiscoveryCustomEvent)discoEvt).customMessage() : null) + ']'); + } + ExchangeType exchange; Collection receivedCaches; @@ -1138,17 +1152,41 @@ private void sendPartitionRequest(Set nodeIds, GridDhtPartitionsSingleRequ detectLostPartitions(); Map m = null; + Map m2 = null; + + IgniteClosure validator = cctx.cache().cacheValidator(); for (GridCacheContext cacheCtx : cctx.cacheContexts()) { - if (cacheCtx.config().getTopologyValidator() != null && !CU.isSystemCache(cacheCtx.name())) { + if (CU.isSystemCache(cacheCtx.name())) + continue; + + if (cacheCtx.config().getTopologyValidator() != null) { if (m == null) m = new HashMap<>(); m.put(cacheCtx.cacheId(), cacheCtx.config().getTopologyValidator().validate(discoEvt.topologyNodes())); } + + if (validator != null) { + if (m2 == null) + m2 = new HashMap<>(); + + Throwable t; + + try { + t = validator.apply(cacheCtx.name()); + } + catch (Throwable e) { + t = e; + } + + if (t != null) + m2.put(cacheCtx.cacheId(), t); + } } cacheValidRes = m != null ? m : Collections.emptyMap(); + customCacheValidRes = m2 != null ? m2 : Collections.emptyMap(); } cctx.exchange().onExchangeDone(this, err); @@ -1197,6 +1235,23 @@ private void sendPartitionRequest(Set nodeIds, GridDhtPartitionsSingleRequ } } + if (cctx.kernalContext().cache().cacheValidator() != null) { + Throwable t; + + Map resMap = customCacheValidRes; + + if (resMap != null) + t = resMap.get(cctx.cacheId()); + else + // Do not cache results, because it's better to do on exchange done. + t = cctx.kernalContext().cache().cacheValidator().apply(cctx.name()); + + if (t != null) { + return new IgniteCheckedException("Failed to perform cache operation " + + "(cache is not valid): " + cctx.name(), t); + } + } + return null; } 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 a318724c70ae1..b061593b109fa 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 @@ -256,6 +256,14 @@ private void map( Map> mapped, final AffinityTopologyVersion topVer ) { + Throwable exc = cctx.topologyVersionFuture().validateCache(cctx); + + if (exc != null) { + onDone(exc); + + return; + } + Collection affNodes = CU.affinityNodes(cctx, topVer); if (affNodes.isEmpty()) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java index eb5e214b89474..8457180e7b3a7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java @@ -34,6 +34,7 @@ import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFuture; import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata; import org.apache.ignite.internal.util.GridBoundedConcurrentOrderedSet; import org.apache.ignite.internal.util.GridCloseableIteratorAdapter; @@ -219,6 +220,16 @@ protected void removeQueryFuture(long reqId) { threads.put(req.id(), Thread.currentThread()); try { + GridDhtTopologyFuture fut = cctx.shared().exchange().exchangeFuture(req.topologyVersion()); + + if (fut == null) + fut = cctx.topologyVersionFuture(); + + Throwable exc = fut.validateCache(cctx); + + if (exc != null) + throw new IgniteCheckedException(exc); + GridCacheQueryInfo info = distributedQueryInfo(sndId, req); if (info == null) 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 bfbd86cba5acf..9c7dfd579309c 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 @@ -18,8 +18,11 @@ package org.apache.ignite.internal.processors.cache.transactions; import java.util.Collection; +import java.util.HashSet; import java.util.List; +import java.util.Set; import java.util.UUID; +import javax.cache.CacheException; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.cluster.ClusterNode; @@ -38,9 +41,11 @@ import org.apache.ignite.internal.processors.cache.distributed.GridCacheTxRecoveryFuture; import org.apache.ignite.internal.processors.cache.distributed.GridCacheTxRecoveryRequest; import org.apache.ignite.internal.processors.cache.distributed.GridCacheTxRecoveryResponse; +import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxPrepareRequest; import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxRemoteAdapter; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtInvalidPartitionException; 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.distributed.dht.GridDhtTxFinishFuture; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishResponse; @@ -120,6 +125,40 @@ public IgniteInternalFuture processNearTxPrepareRequest(final UUID nearNodeId ", node=" + nearNodeId + ']'); } + if (ctx.cache().cacheValidator() != null) { + for (GridCacheContext cctx : caches(req)) { + GridDhtTopologyFuture fut = ctx.exchange().exchangeFuture(req.topologyVersion()); + + if (fut == null) + fut = cctx.topologyVersionFuture(); + + Throwable exc = fut.validateCache(cctx); + + if (exc != null) { + GridNearTxPrepareResponse resp = new GridNearTxPrepareResponse( + req.version(), + req.futureId(), + req.miniId(), + req.version(), + req.version(), + null, + new CacheException(exc), + null, + req.deployInfo() != null + ); + + try { + ctx.io().send(nearNodeId, resp, req.policy()); + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to prepare DHT transaction: [req=" + req + ']', e); + } + + return new GridFinishedFuture<>(resp); + } + } + } + IgniteInternalFuture fut = prepareTx(nearNodeId, null, req); assert req.txState() != null || fut.error() != null || @@ -1037,6 +1076,33 @@ else if (e instanceof IgniteTxHeuristicCheckedException) { (ctx.tm().tx(req.version()) == null && ctx.tm().nearTx(req.version()) == null); } + /** + * @param req Request. + */ + private Collection caches(GridDistributedTxPrepareRequest req) { + Set caches = new HashSet<>(); + + if (!F.isEmpty(req.reads())) { + for (IgniteTxEntry entry : req.reads()) { + GridCacheContext cctx = ctx.cacheContext(entry.cacheId()); + + if (cctx != null) + caches.add(cctx); + } + } + + if (!F.isEmpty(req.writes())) { + for (IgniteTxEntry entry : req.writes()) { + GridCacheContext cctx = ctx.cacheContext(entry.cacheId()); + + if (cctx != null) + caches.add(cctx); + } + } + + return caches; + } + /** * @param nodeId Node ID. * @param req Request. 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 df956b6e62972..ffd16fbe0f915 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 @@ -17,16 +17,10 @@ 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.io.PrintWriter; import java.io.StringWriter; import java.io.UnsupportedEncodingException; -import java.net.URL; -import java.net.URLConnection; import java.util.Collection; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.atomic.AtomicReference; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTopologyValidatorAbstractCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTopologyValidatorAbstractCacheTest.java index 1f923f18ef4de..bab36a68d6a65 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTopologyValidatorAbstractCacheTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTopologyValidatorAbstractCacheTest.java @@ -119,11 +119,14 @@ protected void putValid(String cacheName) { */ protected void getInvalid(String cacheName) { try { - assert grid(0).cache(cacheName).get(KEY_VAL).equals(KEY_VAL); - } - catch (CacheException ignored) { + grid(0).cache(cacheName).get(KEY_VAL); + assert false : "topology validation broken"; } + catch (CacheException ex) { + assert ex.getCause() instanceof IgniteCheckedException && + ex.getCause().getMessage().contains("cache topology is not valid"); + } } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTopologyValidatorAbstractTxCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTopologyValidatorAbstractTxCacheTest.java index fd386bbfbc023..95bac85704253 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTopologyValidatorAbstractTxCacheTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTopologyValidatorAbstractTxCacheTest.java @@ -59,8 +59,9 @@ public abstract class IgniteTopologyValidatorAbstractTxCacheTest extends IgniteT } assertEmpty(null); // rolled back - assertEmpty(CACHE_NAME_1); // rolled back - assertEmpty(CACHE_NAME_2); // rolled back + + getInvalid(CACHE_NAME_1); + getInvalid(CACHE_NAME_2); try (Transaction tx = grid(0).transactions().txStart(TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ)) { putValid(null); @@ -68,7 +69,7 @@ public abstract class IgniteTopologyValidatorAbstractTxCacheTest extends IgniteT } assertEmpty(null); // rolled back - assertEmpty(CACHE_NAME_1); // rolled back + getInvalid(CACHE_NAME_1); // rolled back startGrid(1); @@ -94,7 +95,8 @@ public abstract class IgniteTopologyValidatorAbstractTxCacheTest extends IgniteT } assertEmpty(null); // rolled back - assertEmpty(CACHE_NAME_1); // rolled back + + getInvalid(CACHE_NAME_1); try (Transaction tx = grid(0).transactions().txStart(TransactionConcurrency.OPTIMISTIC, TransactionIsolation.REPEATABLE_READ)) { putValid(CACHE_NAME_1); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridLostPartitionRebalanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridLostPartitionRebalanceTest.java index 56eed411ce56b..99255e5d458d6 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridLostPartitionRebalanceTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridLostPartitionRebalanceTest.java @@ -51,7 +51,7 @@ import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_PART_DATA_LOST; /** - * + * Tests EVT_CACHE_REBALANCE_PART_DATA_LOST events. */ public class GridLostPartitionRebalanceTest extends GridCommonAbstractTest { /** Cache name. */ @@ -120,12 +120,6 @@ public class GridLostPartitionRebalanceTest extends GridCommonAbstractTest { } }; - /** {@inheritDoc} */ - @Override protected void beforeTest() throws Exception { - latch = new CountDownLatch(expEvts); - cnt = new AtomicInteger(0); - } - /** * @throws Exception If failed. */ @@ -150,6 +144,9 @@ public void testPartDataLostEventNoBackups() throws Exception { * @throws Exception If failed. */ private void checkEvents() throws Exception { + latch = new CountDownLatch(expEvts); + cnt = new AtomicInteger(0); + List srvrs = new ArrayList<>(); // Client router. It always up, so client is guaranteed to get diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridLostPartitionValidationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridLostPartitionValidationTest.java new file mode 100644 index 0000000000000..db5ede8e2f29d --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridLostPartitionValidationTest.java @@ -0,0 +1,536 @@ +/* + * 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.rebalancing; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.locks.Lock; +import javax.cache.Cache; +import javax.cache.CacheException; +import javax.cache.processor.EntryProcessorException; +import javax.cache.processor.MutableEntry; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.cache.CacheEntryProcessor; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; +import org.apache.ignite.cache.query.ScanQuery; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.events.CacheRebalancingEvent; +import org.apache.ignite.events.Event; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.lang.IgniteClosure; +import org.apache.ignite.lang.IgnitePredicate; +import org.apache.ignite.resources.IgniteInstanceResource; +import org.apache.ignite.resources.LoggerResource; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.apache.ignite.transactions.Transaction; +import org.jetbrains.annotations.NotNull; + +import static org.apache.ignite.IgniteSystemProperties.IGNITE_CACHE_VALIDATOR; +import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; +import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_PART_DATA_LOST; +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; + +/** + * Tests that cache operations validated. + */ +public class GridLostPartitionValidationTest extends GridCommonAbstractTest { + /** Cache name. */ + private static final String CACHE_NAME = "test"; + + /** Tx cache name. */ + private static final String TX_CACHE_NAME = "tx_test"; + + /** Backups. */ + private int backups; + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + CacheConfiguration ccfg = new CacheConfiguration(CACHE_NAME); + + ccfg.setAffinity(new RendezvousAffinityFunction(false, 32)); + ccfg.setBackups(backups); + + CacheConfiguration ccfg2 = new CacheConfiguration(TX_CACHE_NAME); + + ccfg2.setAffinity(new RendezvousAffinityFunction(false, 32)); + ccfg2.setBackups(backups); + ccfg2.setAtomicityMode(TRANSACTIONAL); + + cfg.setCacheConfiguration(ccfg, ccfg2); + + cfg.setIncludeEventTypes(EVT_CACHE_REBALANCE_PART_DATA_LOST); + + Map, int[]> listeners = new HashMap<>(); + + listeners.put(new Listener(), new int[]{EVT_CACHE_REBALANCE_PART_DATA_LOST}); + + cfg.setLocalEventListeners(listeners); + + cfg.setClientMode(gridName.contains("client")); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + System.clearProperty(IGNITE_CACHE_VALIDATOR); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + System.setProperty(IGNITE_CACHE_VALIDATOR, CacheValidator.class.getName()); + } + + /** + * @throws Exception If failed. + */ + public void testPartDataLostEvent1Backup() throws Exception { + backups = 1; + + checkValidator(); + } + + /** + * @throws Exception If failed. + */ + @SuppressWarnings("ThrowableNotThrown") + private void checkValidator() throws Exception { + List srvrs = new ArrayList<>(); + + srvrs.add(startGrid("server-0")); + + final Ignite client = startGrid("client"); + + srvrs.add(startGrid("server-1")); + srvrs.add(startGrid("server-2")); + + awaitPartitionMapExchange(); + + final IgniteCache cache = client.cache(CACHE_NAME); + final IgniteCache txCache = client.cache(TX_CACHE_NAME); + + for (int i = 0; i < 10_000; i++) { + cache.put(i, i); + txCache.put(i, i); + } + + // Stop node with 0 partition. + Set nodes = new HashSet<>(client.affinity(CACHE_NAME).mapPartitionToPrimaryAndBackups(0)); + + List stopped = stopAffinityNodes(srvrs, nodes); + + awaitPartitionMapExchange(); + + for (Iterator iter = srvrs.iterator(); iter.hasNext(); ) { + Ignite srvr = iter.next(); + + if (stopped.contains(srvr.name())) + iter.remove(); + } + + Ignite srvr = F.first(srvrs); + + checkThrows(client); + checkThrows(srvr); + } + + /** + * @param ignite Client. + */ + @SuppressWarnings("ThrowableNotThrown") + private void checkThrows(final Ignite ignite) { + final IgniteCache cache = ignite.cache(CACHE_NAME); + final IgniteCache txCache = ignite.cache(TX_CACHE_NAME); + + boolean client = ignite.cluster().localNode().isClient(); + + assertThrows(new Callable() { + @Override public Object call() throws Exception { + cache.query(new ScanQuery<>()).getAll(); + + return null; + } + }); + + assertThrows(new Callable() { + @Override public Object call() throws Exception { + cache.get(0); + + return null; + } + }); + + assertThrows(new Callable() { + @Override public Object call() throws Exception { + cache.put(0, 0); + + return null; + } + }); + + assertThrows(new Callable() { + @Override public Object call() throws Exception { + cache.remove(0); + + return null; + } + }); + + assertThrows(new Callable() { + @Override public Object call() throws Exception { + cache.invoke(0, new CacheEntryProcessor() { + @Override public Object process(MutableEntry entry, + Object... arguments) throws EntryProcessorException { + + return null; + } + }); + + return null; + } + }); + + if (!client) { + assertThrows(new Callable() { + @Override public Object call() throws Exception { + cache.localPeek(0); + + return null; + } + }); + } + + assertThrows(new Callable() { + @Override public Object call() throws Exception { + Lock lock = cache.lock(0); + + lock.lock(); + + return null; + } + }); + + if (!client) { + assertThrows(new Callable() { + @Override public Object call() throws Exception { + for (Cache.Entry entry : cache.localEntries()) + System.out.println(entry); + + return null; + } + }); + } + + assertThrows(new Callable() { + @Override public Object call() throws Exception { + txCache.query(new ScanQuery<>()).getAll(); + + return null; + } + }); + + assertThrows(new Callable() { + @Override public Object call() throws Exception { + txCache.get(0); + + return null; + } + }); + + assertThrows(new Callable() { + @Override public Object call() throws Exception { + txCache.put(0, 0); + + return null; + } + }); + + assertThrows(new Callable() { + @Override public Object call() throws Exception { + txCache.remove(0); + + return null; + } + }); + + assertThrows(new Callable() { + @Override public Object call() throws Exception { + txCache.invoke(0, new CacheEntryProcessor() { + @Override public Object process(MutableEntry entry, + Object... arguments) throws EntryProcessorException { + return null; + } + }); + + return null; + } + }); + + assertThrows(new Callable() { + @Override public Object call() throws Exception { + try (Transaction tx = ignite.transactions().txStart(OPTIMISTIC, READ_COMMITTED)) { + txCache.get(0); + + tx.commit(); + } + + return null; + } + }); + + assertThrows(new Callable() { + @Override public Object call() throws Exception { + try (Transaction tx = ignite.transactions().txStart(OPTIMISTIC, READ_COMMITTED)) { + txCache.put(0, 0); + + tx.commit(); + } + + return null; + } + }); + + assertThrows(new Callable() { + @Override public Object call() throws Exception { + try (Transaction tx = ignite.transactions().txStart(OPTIMISTIC, READ_COMMITTED)) { + txCache.remove(0); + + tx.commit(); + } + + return null; + } + }); + + assertThrows(new Callable() { + @Override public Object call() throws Exception { + try (Transaction tx = ignite.transactions().txStart(OPTIMISTIC, READ_COMMITTED)) { + txCache.invoke(0, new CacheEntryProcessor() { + @Override public Object process(MutableEntry entry, + Object... arguments) throws EntryProcessorException { + return null; + } + }); + + tx.commit(); + } + + return null; + } + }); + + assertThrows(new Callable() { + @Override public Object call() throws Exception { + try (Transaction tx = ignite.transactions().txStart(PESSIMISTIC, READ_COMMITTED)) { + txCache.get(0); + + tx.commit(); + } + + return null; + } + }); + + assertThrows(new Callable() { + @Override public Object call() throws Exception { + try (Transaction tx = ignite.transactions().txStart(PESSIMISTIC, READ_COMMITTED)) { + txCache.put(0, 0); + + tx.commit(); + } + + return null; + } + }); + + assertThrows(new Callable() { + @Override public Object call() throws Exception { + try (Transaction tx = ignite.transactions().txStart(PESSIMISTIC, READ_COMMITTED)) { + txCache.remove(0); + + tx.commit(); + } + + return null; + } + }); + + assertThrows(new Callable() { + @Override public Object call() throws Exception { + try (Transaction tx = ignite.transactions().txStart(PESSIMISTIC, READ_COMMITTED)) { + txCache.invoke(0, new CacheEntryProcessor() { + @Override public Object process(MutableEntry entry, + Object... arguments) throws EntryProcessorException { + + return null; + } + }); + + tx.commit(); + } + + return null; + } + }); + + if (!client) { + assertThrows(new Callable() { + @Override public Object call() throws Exception { + txCache.localPeek(0); + + return null; + } + }); + } + + if (!client) { + assertThrows(new Callable() { + @Override public Object call() throws Exception { + for (Cache.Entry entry : txCache.localEntries()) + System.out.println(entry); + + return null; + } + }); + } + } + + /** + * @param c Closure. + */ + private void assertThrows(Callable c) { + try { + c.call(); + + assert false : "Exception was not thrown"; + } + catch (CacheException | IgniteException e) { + log.info("Caught expected exception: " + e.getClass()); + } + catch (Exception e) { + e.printStackTrace(); + + assert false : "Wrong exception was thrown: " + e.getClass(); + } + } + + /** + * @param srvrs Servers. + * @param nodes Nodes. + */ + @NotNull private List stopAffinityNodes(List srvrs, Set nodes) throws IgniteCheckedException { + List> futs = new ArrayList<>(); + + final List stopped = new ArrayList<>(); + + for (final Ignite srv : srvrs) { + final ClusterNode node = srv.cluster().localNode(); + + if (nodes.contains(node)) { + IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { + @Override public Object call() throws Exception { + srv.close(); + + System.out.println(">> Stopped " + srv.name() + " " + node.id()); + + stopped.add(srv.name()); + + return null; + } + }); + + futs.add(fut); + } + } + + for (IgniteInternalFuture fut : futs) + fut.get(); + + return stopped; + } + + /** + * + */ + private static class Listener implements IgnitePredicate { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + + /** Ignite. */ + @SuppressWarnings("unused") + @IgniteInstanceResource + private Ignite ignite; + + /** {@inheritDoc} */ + @Override public boolean apply(CacheRebalancingEvent evt) { + ignite.cluster().nodeLocalMap().putIfAbsent(evt.cacheName(), false); + + return true; + } + } + + /** + * + */ + public static class CacheValidator implements IgniteClosure { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + /** */ + @IgniteInstanceResource + private Ignite ignite; + + /** Logger. */ + @SuppressWarnings("unused") + @LoggerResource + private IgniteLogger log; + + /** {@inheritDoc} */ + @Override public Throwable apply(String cacheName) { + log.info(">>> Validator"); + + Object val = ignite.cluster().nodeLocalMap().get(cacheName); + + return Boolean.FALSE.equals(val) ? new IllegalStateException("Illegal cache state " + + Thread.currentThread().getName()) : null; + } + } +} 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 7a66883f93f24..6ae7d22d1b138 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 @@ -60,6 +60,7 @@ import org.apache.ignite.internal.processors.cache.distributed.rebalancing.GridCacheRebalancingSyncSelfTest; import org.apache.ignite.internal.processors.cache.distributed.rebalancing.GridCacheRebalancingUnmarshallingFailedSelfTest; import org.apache.ignite.internal.processors.cache.distributed.rebalancing.GridLostPartitionRebalanceTest; +import org.apache.ignite.internal.processors.cache.distributed.rebalancing.GridLostPartitionValidationTest; import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCacheDaemonNodeReplicatedSelfTest; import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCacheReplicatedAtomicGetAndTransformStoreSelfTest; import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCacheReplicatedBasicApiTest; @@ -155,6 +156,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(GridCacheRebalancingAsyncSelfTest.class); suite.addTestSuite(GridCacheRabalancingDelayedPartitionMapExchangeSelfTest.class); suite.addTestSuite(GridLostPartitionRebalanceTest.class); + suite.addTestSuite(GridLostPartitionValidationTest.class); // Test for byte array value special case. suite.addTestSuite(GridCacheLocalByteArrayValuesSelfTest.class); 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 85fb4989658d3..4080c1ff06bfc 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 @@ -48,6 +48,7 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -85,7 +86,6 @@ import org.apache.ignite.internal.processors.cache.query.GridCacheTwoStepQuery; import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode; import org.apache.ignite.internal.processors.query.GridQueryCacheObjectsIterator; -import org.apache.ignite.internal.processors.query.GridRunningQueryInfo; import org.apache.ignite.internal.processors.query.GridQueryCancel; import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata; import org.apache.ignite.internal.processors.query.GridQueryFieldsResult; @@ -871,6 +871,9 @@ public GridQueryFieldsResult queryLocalSqlFields(@Nullable final String spaceNam Prepared p = GridSqlQueryParser.prepared((JdbcPreparedStatement)stmt); + if (ctx != null && ctx.cache().cacheValidator() != null) + validateCaches(p); + if (!p.isQuery()) { SqlFieldsQuery fldsQry = new SqlFieldsQuery(qry); @@ -1287,7 +1290,15 @@ public GridCloseableIterator> queryLocalSql(@Nullable runs.put(run.id(), run); try { - ResultSet rs = executeSqlQueryWithTimer(spaceName, conn, sql, params, true, 0, cancel); + PreparedStatement stmt = preparedStatementWithParams(conn, sql, params, true); + + if (ctx != null && ctx.cache().cacheValidator() != null) { + Prepared p = GridSqlQueryParser.prepared((JdbcPreparedStatement)stmt); + + validateCaches(p); + } + + ResultSet rs = executeSqlQueryWithTimer(spaceName, stmt, conn, sql, params, 0, cancel); return new KeyValIterator(rs); } @@ -1633,7 +1644,7 @@ else if (star > 0) { (upper.startsWith("WHERE") || upper.startsWith("ORDER") || upper.startsWith("LIMIT") ? " " : " WHERE "); - if(tableAlias != null) + if (tableAlias != null) t = tableAlias; qry = "SELECT " + t + "." + KEY_FIELD_NAME + ", " + t + "." + VAL_FIELD_NAME + from + qry; @@ -2928,6 +2939,26 @@ public FieldsIterator(ResultSet data) throws IgniteCheckedException { } } + /** + * @param p Prepared. + */ + private void validateCaches(Prepared p) throws IgniteCheckedException { + Set tbls = GridSqlQueryParser.getTables(p); + + if (!F.isEmpty(tbls)) { + for (GridH2Table tbl : tbls) { + if (tbl != null) { + GridCacheContext cctx = tbl.rowDescriptor().context(); + + Throwable exc = cctx.topologyVersionFuture().validateCache(cctx); + + if (exc != null) + throw new IgniteCheckedException(exc); + } + } + } + } + /** * Special key/value iterator based on database result set. */ diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java index 0f940e9c3846c..e13b61833bdb4 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java @@ -22,12 +22,15 @@ import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.IdentityHashMap; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.Set; import javax.cache.CacheException; import org.apache.ignite.IgniteException; +import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table; import org.h2.command.Command; import org.h2.command.CommandContainer; import org.h2.command.Prepared; @@ -1062,6 +1065,48 @@ private GridSqlElement parseExpression0(Expression expression, boolean calcTypes expression.getClass().getSimpleName() + ']'); } + /** + * @param stmt Prepared statement. + * @return Tables set. + */ + public static Set getTables(Prepared stmt) { + Set res = new HashSet<>(); + + Set tbls = null; + + if (stmt instanceof Query) + tbls = ((Query)stmt).getTables(); + + else if (stmt instanceof Merge) + tbls = Collections.singleton(MERGE_TABLE.get((Merge)stmt)); + + else if (stmt instanceof Insert) + tbls = Collections.singleton(INSERT_TABLE.get((Insert)stmt)); + + else if (stmt instanceof Delete) { + TableFilter filter = DELETE_FROM.get((Delete)stmt); + + if (filter != null) + tbls = Collections.singleton(filter.getTable()); + } + + else if (stmt instanceof Update) { + TableFilter filter = UPDATE_TARGET.get((Update)stmt); + + if (filter != null) + tbls = Collections.singleton(filter.getTable()); + } + + if (tbls != null) { + for (Table table : tbls) { + if (table instanceof GridH2Table) + res.add((GridH2Table)table); + } + } + + return res; + } + /** * @param cond Condition. * @param o Object. 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 fc002cb3ce200..bac7c8c16e6b7 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 @@ -554,7 +554,7 @@ private void onQueryRequest0( } } - qr = new QueryResults(reqId, qrys.size(), mainCctx); + qr = new QueryResults(reqId, qrys.size(), mainCctx, cacheIds); if (nodeRess.put(reqId, segmentId, qr) != null) throw new IllegalStateException(); @@ -649,6 +649,12 @@ private void onQueryRequest0( throw new QueryCancelledException(); } + // Validate cache + Throwable exc = qr.validateCaches(); + + if (exc != null) + throw exc; + // Send the first page. sendNextPage(nodeRess, node, qr, qryIdx, segmentId, pageSize); @@ -741,8 +747,17 @@ private void onNextPageRequest(ClusterNode node, GridQueryNextPageRequest req) { sendError(node, req.queryRequestId(), new CacheException("No query result found for request: " + req)); else if (qr.canceled) sendError(node, req.queryRequestId(), new QueryCancelledException()); - else - sendNextPage(nodeRess, node, qr, req.query(), req.segmentId(), req.pageSize()); + else { + Throwable exc = qr.validateCaches(); + + if (exc == null) + sendNextPage(nodeRess, node, qr, req.query(), req.segmentId(), req.pageSize()); + else { + nodeRess.remove(req.queryRequestId(), req.segmentId(), qr); + + sendError(node, req.queryRequestId(), new CacheException(exc.getMessage(), exc)); + } + } } /** @@ -970,6 +985,9 @@ private class QueryResults { /** */ private final GridCacheContext cctx; + /** */ + private final List cacheIds; + /** */ private volatile boolean canceled; @@ -977,11 +995,14 @@ private class QueryResults { * @param qryReqId Query request ID. * @param qrys Number of queries. * @param cctx Cache context. + * @param cacheIds Cache IDs. */ @SuppressWarnings("unchecked") - private QueryResults(long qryReqId, int qrys, GridCacheContext cctx) { + private QueryResults(long qryReqId, int qrys, GridCacheContext cctx, + List cacheIds) { this.qryReqId = qryReqId; this.cctx = cctx; + this.cacheIds = cacheIds; results = new AtomicReferenceArray<>(qrys); cancels = new GridQueryCancel[qrys]; @@ -1049,6 +1070,27 @@ void cancel(boolean forceQryCancel) { } } } + + /** + * Validate caches. + * + * @return {@link Throwable} if cache is not valid or {@code null} otherwise. + */ + Throwable validateCaches() { + // Validate cache + for (Integer cacheId : cacheIds) { + GridCacheContext cctx = ctx.cache().context().cacheContext(cacheId); + + if (!cctx.isLocal()) { + Throwable exc = cctx.topologyVersionFuture().validateCache(cctx); + + if (exc != null) + return exc; + } + } + + return null; + } } /** diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridLostPartitionValidationQueryTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridLostPartitionValidationQueryTest.java new file mode 100644 index 0000000000000..5ca7938a2491c --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridLostPartitionValidationQueryTest.java @@ -0,0 +1,300 @@ +/* + * 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.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.Callable; +import javax.cache.CacheException; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteDataStreamer; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.cache.QueryEntity; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.cache.query.SqlQuery; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.events.CacheRebalancingEvent; +import org.apache.ignite.events.Event; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.lang.IgniteClosure; +import org.apache.ignite.lang.IgnitePredicate; +import org.apache.ignite.resources.IgniteInstanceResource; +import org.apache.ignite.resources.LoggerResource; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.jetbrains.annotations.NotNull; + +import static org.apache.ignite.IgniteSystemProperties.IGNITE_CACHE_VALIDATOR; +import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; +import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_PART_DATA_LOST; + +/** + * Tests that SQL query validated. + */ +public class GridLostPartitionValidationQueryTest extends GridCommonAbstractTest { + /** Cache name. */ + private static final String CACHE_NAME = "test"; + + /** Tx cache name. */ + private static final String TX_CACHE_NAME = "tx_test"; + + /** Backups. */ + private int backups; + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + CacheConfiguration ccfg1 = new CacheConfiguration(CACHE_NAME); + + ccfg1.setAffinity(new RendezvousAffinityFunction(false, 32)); + ccfg1.setBackups(backups); + + QueryEntity qe1 = new QueryEntity(Integer.class.getName(), Integer.class.getName()); + qe1.setTableName("T1"); + + ccfg1.setQueryEntities(Collections.singleton(qe1)); + + CacheConfiguration ccfg2 = new CacheConfiguration(TX_CACHE_NAME); + + ccfg2.setAffinity(new RendezvousAffinityFunction(false, 32)); + ccfg2.setBackups(backups); + ccfg2.setAtomicityMode(TRANSACTIONAL); + + QueryEntity qe2 = new QueryEntity(Integer.class.getName(), Integer.class.getName()); + qe2.setTableName("T2"); + + ccfg2.setQueryEntities(Collections.singleton(qe2)); + + cfg.setCacheConfiguration(ccfg1, ccfg2); + + cfg.setIncludeEventTypes(EVT_CACHE_REBALANCE_PART_DATA_LOST); + + Map, int[]> listeners = new HashMap<>(); + + listeners.put(new Listener(), new int[]{EVT_CACHE_REBALANCE_PART_DATA_LOST}); + + cfg.setLocalEventListeners(listeners); + + cfg.setClientMode(gridName.contains("client")); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + System.clearProperty(IGNITE_CACHE_VALIDATOR); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + System.setProperty(IGNITE_CACHE_VALIDATOR, CacheValidator.class.getName()); + } + + /** + * @throws Exception If failed. + */ + public void testPartDataLostEvent1Backup() throws Exception { + backups = 1; + + checkValidator(); + } + + /** + * @throws Exception If failed. + */ + @SuppressWarnings("ThrowableNotThrown") + private void checkValidator() throws Exception { + List srvrs = new ArrayList<>(); + + srvrs.add(startGrid("server-0")); + + final Ignite client = startGrid("client"); + + srvrs.add(startGrid("server-1")); + srvrs.add(startGrid("server-2")); + + awaitPartitionMapExchange(); + + final IgniteCache cache = client.cache(CACHE_NAME); + + IgniteDataStreamer streamer1 = client.dataStreamer(CACHE_NAME); + IgniteDataStreamer streamer2 = client.dataStreamer(TX_CACHE_NAME); + + for (int i = 0; i < 100_000; i++) { + streamer1.addData(i, i); + streamer2.addData(i, i); + } + + streamer1.close(); + streamer2.close(); + + @SuppressWarnings("unchecked") + final SqlQuery qry1 = new SqlQuery(Integer.class, "SELECT a.* FROM T1 a, \"tx_test\".T2 b LIMIT 10000"); + final SqlFieldsQuery qry2 = new SqlFieldsQuery("SELECT a.* FROM T1 a, \"tx_test\".T2 b LIMIT 10000"); + + // Stop node with 0 partition. + Set nodes = new HashSet<>(client.affinity(CACHE_NAME).mapPartitionToPrimaryAndBackups(0)); + + List stopped = stopAffinityNodes(srvrs, nodes); + + awaitPartitionMapExchange(); + + for (Iterator iter = srvrs.iterator(); iter.hasNext(); ) { + Ignite srvr = iter.next(); + + if (stopped.contains(srvr.name())) + iter.remove(); + } + + GridTestUtils.assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + cache.query(qry1).getAll(); + + return null; + } + }, CacheException.class, null); + + GridTestUtils.assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + cache.query(qry2).getAll(); + + return null; + } + }, CacheException.class, null); + + final IgniteCache srvCache = srvrs.get(0).cache(CACHE_NAME); + + qry1.setLocal(true); + + GridTestUtils.assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + srvCache.query(qry1).getAll(); + + return null; + } + }, CacheException.class, null); + + GridTestUtils.assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + srvCache.query(qry2).getAll(); + + return null; + } + }, CacheException.class, null); + } + + /** + * @param srvrs Servers. + * @param nodes Nodes. + */ + @NotNull private List stopAffinityNodes(List srvrs, Set nodes) throws IgniteCheckedException { + List> futs = new ArrayList<>(); + + final List stopped = new ArrayList<>(); + + for (final Ignite srv : srvrs) { + final ClusterNode node = srv.cluster().localNode(); + + if (nodes.contains(node)) { + IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { + @Override public Object call() throws Exception { + srv.close(); + + System.out.println(">> Stopped " + srv.name() + " " + node.id()); + + stopped.add(srv.name()); + + return null; + } + }); + + futs.add(fut); + } + } + + for (IgniteInternalFuture fut : futs) + fut.get(); + + return stopped; + } + + /** + * + */ + private static class Listener implements IgnitePredicate { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + + /** Ignite. */ + @SuppressWarnings("unused") + @IgniteInstanceResource + private Ignite ignite; + + /** {@inheritDoc} */ + @Override public boolean apply(CacheRebalancingEvent evt) { + if (TX_CACHE_NAME.equals(evt.cacheName())) + ignite.cluster().nodeLocalMap().putIfAbsent(evt.cacheName(), false); + + return true; + } + } + + /** + * + */ + public static class CacheValidator implements IgniteClosure { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + /** */ + @IgniteInstanceResource + private Ignite ignite; + + /** Logger. */ + @SuppressWarnings("unused") + @LoggerResource + private IgniteLogger log; + + /** {@inheritDoc} */ + @Override public Throwable apply(String cacheName) { + log.info(">>> Validator " + cacheName); + + Object val = ignite.cluster().nodeLocalMap().get(cacheName); + + return Boolean.FALSE.equals(val) ? new IllegalStateException("Illegal cache state " + + Thread.currentThread().getName()) : null; + } + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java index 79492149d753e..ceeb022e83977 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java +++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java @@ -31,6 +31,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheOffheapIndexGetSelfTest; import org.apache.ignite.internal.processors.cache.GridCacheSwapSelfTest; import org.apache.ignite.internal.processors.cache.GridIndexingWithNoopSwapSelfTest; +import org.apache.ignite.internal.processors.cache.GridLostPartitionValidationQueryTest; import org.apache.ignite.internal.processors.cache.IgniteCacheConfigurationPrimitiveTypesSelfTest; import org.apache.ignite.internal.processors.cache.IgniteCacheStarvationOnRebalanceTest; import org.apache.ignite.internal.processors.cache.IgniteClientReconnectQueriesTest; @@ -84,6 +85,8 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(ClientReconnectAfterClusterRestartTest.class); + suite.addTestSuite(GridLostPartitionValidationQueryTest.class); + return suite; } } From 31944326c9aba6dac08482c5cadcd562ba7577d1 Mon Sep 17 00:00:00 2001 From: Denis Mekhanikov Date: Fri, 27 Oct 2017 17:37:58 +0300 Subject: [PATCH 323/357] revert previous changes on IGNITE-5860 --- .../ignite/spi/discovery/tcp/ClientImpl.java | 52 ++-- .../ignite/spi/discovery/tcp/ServerImpl.java | 250 +++++++++--------- .../spi/discovery/tcp/TcpDiscoveryImpl.java | 2 +- .../tcp/TcpClientDiscoverySpiSelfTest.java | 229 +--------------- 4 files changed, 149 insertions(+), 384 deletions(-) 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 ce4f1b7fe4b9a..c32c8e057d9fd 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 @@ -26,7 +26,6 @@ import java.net.SocketTimeoutException; import java.util.ArrayDeque; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -468,8 +467,7 @@ else if (state == DISCONNECTED) { } /** - * @param prevAddr If reconnect is in progress, then previous address of the router the client was connected to - * and {@code null} otherwise. + * @param recon {@code True} if reconnects. * @param timeout Timeout. * @return Opened socket or {@code null} if timeout. * @throws InterruptedException If interrupted. @@ -477,9 +475,9 @@ else if (state == DISCONNECTED) { * @see TcpDiscoverySpi#joinTimeout */ @SuppressWarnings("BusyWait") - @Nullable private T2 joinTopology(InetSocketAddress prevAddr, long timeout) + @Nullable private T2 joinTopology(boolean recon, long timeout) throws IgniteSpiException, InterruptedException { - List addrs = null; + Collection addrs = null; long startTime = U.currentTimeMillis(); @@ -488,7 +486,7 @@ else if (state == DISCONNECTED) { throw new InterruptedException(); while (addrs == null || addrs.isEmpty()) { - addrs = new ArrayList<>(spi.resolvedAddresses()); + addrs = spi.resolvedAddresses(); if (!F.isEmpty(addrs)) { if (log.isDebugEnabled()) @@ -508,25 +506,17 @@ else if (state == DISCONNECTED) { } } - // process failed node last - if (prevAddr != null) { - int idx = addrs.indexOf(prevAddr); - - if (idx != -1) - Collections.swap(addrs, idx, 0); - } - Collection addrs0 = new ArrayList<>(addrs); + Iterator it = addrs.iterator(); + boolean wait = false; - for (int i = addrs.size() - 1; i >= 0; i--) { + while (it.hasNext()) { if (Thread.currentThread().isInterrupted()) throw new InterruptedException(); - InetSocketAddress addr = addrs.get(i); - - boolean recon = prevAddr != null; + InetSocketAddress addr = it.next(); T3 sockAndRes; @@ -540,7 +530,7 @@ else if (state == DISCONNECTED) { } if (sockAndRes == null) { - addrs.remove(i); + it.remove(); continue; } @@ -862,8 +852,8 @@ private NavigableSet allVisibleNodes() { } /** {@inheritDoc} */ - @Override protected Collection threads() { - return Arrays.asList(sockWriter, msgWorker); + @Override protected IgniteSpiThread workerThread() { + return msgWorker; } /** @@ -1349,20 +1339,15 @@ private class Reconnector extends IgniteSpiThread { private boolean clientAck; /** */ - private final boolean join; - - /** */ - private final InetSocketAddress prevAddr; + private boolean join; /** * @param join {@code True} if reconnects during join. - * @param prevAddr Address of the node, that this client was previously connected to. */ - protected Reconnector(boolean join, InetSocketAddress prevAddr) { + protected Reconnector(boolean join) { super(spi.ignite().name(), "tcp-client-disco-reconnector", log); this.join = join; - this.prevAddr = prevAddr; } /** @@ -1392,7 +1377,7 @@ public void cancel() { try { while (true) { - T2 joinRes = joinTopology(prevAddr, timeout); + T2 joinRes = joinTopology(true, timeout); if (joinRes == null) { if (join) { @@ -1627,10 +1612,6 @@ else if (msg instanceof TcpDiscoveryNodeFailedMessage && } else if (msg instanceof SocketClosedMessage) { if (((SocketClosedMessage)msg).sock == currSock) { - Socket sock = currSock.sock; - - InetSocketAddress prevAddr = new InetSocketAddress(sock.getInetAddress(), sock.getPort()); - currSock = null; boolean join = joinLatch.getCount() > 0; @@ -1659,7 +1640,8 @@ else if (msg instanceof SocketClosedMessage) { assert reconnector == null; - reconnector = new Reconnector(join, prevAddr); + final Reconnector reconnector = new Reconnector(join); + this.reconnector = reconnector; reconnector.start(); } } @@ -1830,7 +1812,7 @@ private void tryJoin() throws InterruptedException { T2 joinRes; try { - joinRes = joinTopology(null, spi.joinTimeout); + joinRes = joinTopology(false, spi.joinTimeout); } catch (IgniteSpiException e) { joinError(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 640dcec85b0b0..189b37a6a0899 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 @@ -222,9 +222,6 @@ class ServerImpl extends TcpDiscoveryImpl { /** Pending custom messages that should not be sent between NodeAdded and NodeAddFinished messages. */ private Queue pendingCustomMsgs = new ArrayDeque<>(); - /** Messages history used for client reconnect. */ - private final EnsuredMessageHistory msgHist = new EnsuredMessageHistory(); - /** If non-shared IP finder is used this flag shows whether IP finder contains local address. */ private boolean ipFinderHasLocAddr; @@ -1645,23 +1642,8 @@ private void clearNodeAddedMessage(TcpDiscoveryAbstractMessage msg) { } /** {@inheritDoc} */ - @Override protected Collection threads() { - Collection threads; - - synchronized (mux) { - threads = new ArrayList<>(readers.size() + clientMsgWorkers.size() + 4); - threads.addAll(readers); - } - - threads.addAll(clientMsgWorkers.values()); - threads.add(tcpSrvr); - threads.add(ipFinderCleaner); - threads.add(msgWorker); - threads.add(statsPrinter); - - threads.removeAll(Collections.singleton(null)); - - return threads; + @Override protected IgniteSpiThread workerThread() { + return msgWorker; } /** @@ -2109,9 +2091,7 @@ else if (msg instanceof TcpDiscoveryNodeLeftMessage) else if (msg instanceof TcpDiscoveryNodeFailedMessage) clearClientAddFinished(((TcpDiscoveryNodeFailedMessage)msg).failedNodeId()); - synchronized (msgs) { - msgs.add(msg); - } + msgs.add(msg); } /** @@ -2173,16 +2153,14 @@ private boolean mapsEqual(Map m1, Map m2) { // Client connection failed before it received TcpDiscoveryNodeAddedMessage. List res = null; - synchronized (msgs) { - for (TcpDiscoveryAbstractMessage msg : msgs) { - if (msg instanceof TcpDiscoveryNodeAddedMessage) { - if (node.id().equals(((TcpDiscoveryNodeAddedMessage)msg).node().id())) - res = new ArrayList<>(msgs.size()); - } - - if (res != null) - res.add(prepare(msg, node.id())); + for (TcpDiscoveryAbstractMessage msg : msgs) { + if (msg instanceof TcpDiscoveryNodeAddedMessage) { + if (node.id().equals(((TcpDiscoveryNodeAddedMessage)msg).node().id())) + res = new ArrayList<>(msgs.size()); } + + if (res != null) + res.add(prepare(msg, node.id())); } if (log.isDebugEnabled()) { @@ -2195,26 +2173,20 @@ private boolean mapsEqual(Map m1, Map m2) { return res; } else { - Collection cp; + if (msgs.isEmpty()) + return Collections.emptyList(); - boolean skip; + Collection cp = new ArrayList<>(msgs.size()); - synchronized (msgs) { - if (msgs.isEmpty()) - return Collections.emptyList(); + boolean skip = true; - cp = new ArrayList<>(msgs.size()); - - skip = true; - - for (TcpDiscoveryAbstractMessage msg : msgs) { - if (skip) { - if (msg.id().equals(lastMsgId)) - skip = false; - } - else - cp.add(prepare(msg, node.id())); + for (TcpDiscoveryAbstractMessage msg : msgs) { + if (skip) { + if (msg.id().equals(lastMsgId)) + skip = false; } + else + cp.add(prepare(msg, node.id())); } cp = !skip ? cp : null; @@ -2503,6 +2475,9 @@ private class RingMessageWorker extends MessageWorkerAdapter pending = msgHist.messages(msg.lastMessageId(), node); + + if (pending != null) { + msg.pendingMessages(pending); + msg.success(true); + + if (log.isDebugEnabled()) + log.debug("Accept client reconnect, restored pending messages " + + "[locNodeId=" + locNodeId + ", clientNodeId=" + nodeId + ']'); + } + else { + if (log.isDebugEnabled()) + log.debug("Failing reconnecting client node because failed to restore pending " + + "messages [locNodeId=" + locNodeId + ", clientNodeId=" + nodeId + ']'); + + TcpDiscoveryNodeFailedMessage nodeFailedMsg = new TcpDiscoveryNodeFailedMessage(locNodeId, + node.id(), node.internalOrder()); + + processNodeFailedMessage(nodeFailedMsg); + + if (nodeFailedMsg.verified()) + msgHist.add(nodeFailedMsg); + } + } + else if (log.isDebugEnabled()) + log.debug("Reconnecting client node is already failed [nodeId=" + nodeId + ']'); + + if (isLocNodeRouter) { + ClientMessageWorker wrk = clientMsgWorkers.get(nodeId); + + if (wrk != null) + wrk.addMessage(msg); + else if (log.isDebugEnabled()) + log.debug("Failed to reconnect client node (disconnected during the process) [locNodeId=" + + locNodeId + ", clientNodeId=" + nodeId + ']'); + } + else { + if (sendMessageToRemotes(msg)) + sendMessageAcrossRing(msg); + } + } + else { + if (sendMessageToRemotes(msg)) + sendMessageAcrossRing(msg); + } + } + else { + if (isLocalNodeCoordinator()) + addMessage(new TcpDiscoveryDiscardMessage(locNodeId, msg.id(), false)); - if (msg.verified() && msg.routerNodeId().equals(getLocalNodeId())) { - ClientMessageWorker wrk = clientMsgWorkers.get(nodeId); + if (isLocNodeRouter) { + ClientMessageWorker wrk = clientMsgWorkers.get(nodeId); - if (wrk != null) - wrk.addMessage(msg); - else if (log.isDebugEnabled()) - log.debug("Failed to reconnect client node (disconnected during the process) [locNodeId=" + - locNodeId + ", clientNodeId=" + nodeId + ']'); + if (wrk != null) + wrk.addMessage(msg); + else if (log.isDebugEnabled()) + log.debug("Failed to reconnect client node (disconnected during the process) [locNodeId=" + + locNodeId + ", clientNodeId=" + nodeId + ']'); + } + else { + if (ring.hasRemoteNodes() && !isLocalNodeCoordinator()) + sendMessageAcrossRing(msg); + } } } @@ -4055,6 +4093,9 @@ private void processNodeAddedMessage(TcpDiscoveryNodeAddedMessage msg) { processNodeAddFinishedMessage(addFinishMsg); + if (addFinishMsg.verified()) + msgHist.add(addFinishMsg); + addMessage(new TcpDiscoveryDiscardMessage(locNodeId, msg.id(), false)); return; @@ -5115,6 +5156,9 @@ private void processHeartbeatMessage(TcpDiscoveryHeartbeatMessage msg) { locNodeId, clientNode.id(), clientNode.internalOrder()); processNodeFailedMessage(nodeFailedMsg); + + if (nodeFailedMsg.verified()) + msgHist.add(nodeFailedMsg); } } } @@ -5312,6 +5356,9 @@ private void processCustomMessage(TcpDiscoveryCustomEventMessage msg) { ackMsg.topologyVersion(msg.topologyVersion()); processCustomMessage(ackMsg); + + if (ackMsg.verified()) + msgHist.add(ackMsg); } catch (IgniteCheckedException e) { U.error(log, "Failed to marshal discovery custom message.", e); @@ -5413,8 +5460,12 @@ private void checkPendingCustomMessages() { if (joiningEmpty && isLocalNodeCoordinator()) { TcpDiscoveryCustomEventMessage msg; - while ((msg = pollPendingCustomeMessage()) != null) + while ((msg = pollPendingCustomeMessage()) != null) { processCustomMessage(msg); + + if (msg.verified()) + msgHist.add(msg); + } } } @@ -5968,7 +6019,7 @@ else if (msg instanceof TcpDiscoveryClientReconnectMessage) { if (clientMsgWrk.getState() == State.NEW) clientMsgWrk.start(); - processClientReconnectMessage((TcpDiscoveryClientReconnectMessage)msg); + msgWorker.addMessage(msg); continue; } @@ -6205,67 +6256,6 @@ else if (msg instanceof TcpDiscoveryRingLatencyCheckMessage) { } } - /** - * Processes client reconnect message. - * - * @param msg Client reconnect message. - */ - private void processClientReconnectMessage(TcpDiscoveryClientReconnectMessage msg) { - UUID nodeId = msg.creatorNodeId(); - UUID locNodeId = getLocalNodeId(); - - boolean isLocNodeRouter = locNodeId.equals(msg.routerNodeId()); - - if (isLocNodeRouter) { - TcpDiscoveryNode node = ring.node(nodeId); - ClientMessageWorker wrk = clientMsgWorkers.get(nodeId); - - if (wrk != null && node != null) { - if (!msg.verified()) { - msg.verify(getLocalNodeId()); - - Collection pending = msgHist.messages(msg.lastMessageId(), node); - - if (pending != null) { - msg.success(true); - msg.pendingMessages(pending); - - TcpDiscoveryClientReconnectMessage msgCp = new TcpDiscoveryClientReconnectMessage( - msg.creatorNodeId(), msg.routerNodeId(), msg.lastMessageId()); - msgCp.client(msg.client()); - - msgWorker.addMessage(msgCp); - - if (log.isDebugEnabled()) { - log.debug("Accept client reconnect, restored pending messages " + - "[locNodeId=" + locNodeId + ", clientNodeId=" + nodeId + ']'); - } - } - else { - if (log.isDebugEnabled()) - log.debug("Failing reconnecting client node because failed to restore pending " + - "messages [locNodeId=" + locNodeId + ", clientNodeId=" + nodeId + ']'); - - TcpDiscoveryNodeFailedMessage nodeFailedMsg = new TcpDiscoveryNodeFailedMessage(locNodeId, - node.id(), node.internalOrder()); - - msgWorker.addMessage(nodeFailedMsg); - } - } - else - wrk.addMessage(msg); - } - else if (log.isDebugEnabled()) - log.debug("Failed to reconnect client node (disconnected during the process) [locNodeId=" + - locNodeId + ", clientNodeId=" + nodeId + ']'); - - if (wrk != null) - wrk.addMessage(msg); - } - else - msgWorker.addMessage(msg); - } - /** * Processes client heartbeat message. * 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 1fa7139e7e956..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 @@ -292,7 +292,7 @@ protected static String threadStatus(Thread t) { * * @return Worker thread. */ - protected abstract Collection threads(); + protected abstract IgniteSpiThread workerThread(); /** * @throws IgniteSpiException If failed. 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 819108148cf16..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 @@ -62,8 +62,8 @@ import org.apache.ignite.spi.IgniteSpiException; import org.apache.ignite.spi.IgniteSpiOperationTimeoutException; import org.apache.ignite.spi.IgniteSpiOperationTimeoutHelper; -import org.apache.ignite.spi.IgniteSpiThread; 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.TcpDiscoveryClientReconnectMessage; @@ -82,14 +82,13 @@ 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.events.EventType.EVT_NODE_SEGMENTED; -import static org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi.DFLT_HEARTBEAT_FREQ; /** * Client-based discovery tests. */ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest { /** */ - private static final TcpDiscoveryVmIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); /** */ protected static final AtomicInteger srvIdx = new AtomicInteger(); @@ -124,9 +123,6 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest { /** */ private static CountDownLatch clientFailedLatch; - /** */ - private static CountDownLatch clientReconnectedLatch; - /** */ private static CountDownLatch msgLatch; @@ -142,9 +138,6 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest { /** */ protected long netTimeout = TcpDiscoverySpi.DFLT_NETWORK_TIMEOUT; - /** */ - protected Integer reconnectCnt; - /** */ private boolean longSockTimeouts; @@ -215,9 +208,6 @@ else if (gridName.startsWith("client")) { disco.setJoinTimeout(joinTimeout); disco.setNetworkTimeout(netTimeout); - if (reconnectCnt != null) - disco.setReconnectCount(reconnectCnt); - disco.setClientReconnectDisabled(reconnectDisabled); if (disco instanceof TestTcpDiscoverySpi) @@ -264,8 +254,6 @@ protected TcpDiscoverySpi getDiscoverySpi() { clientIpFinder = null; joinTimeout = TcpDiscoverySpi.DFLT_JOIN_TIMEOUT; netTimeout = TcpDiscoverySpi.DFLT_NETWORK_TIMEOUT; - maxMissedClientHbs = TcpDiscoverySpi.DFLT_MAX_MISSED_CLIENT_HEARTBEATS; - reconnectCnt = null; longSockTimeouts = false; assert G.allGrids().isEmpty(); @@ -546,176 +534,6 @@ public void testClientReconnectOnRouterFail() throws Exception { checkNodes(2, 3); } - /** - * Client should reconnect to available server without EVT_CLIENT_NODE_RECONNECTED event. - * - * @throws Exception If failed. - */ - public void testClientReconnectOnRouterSuspend() throws Exception { - reconnectAfterSuspend(false); - } - - /** - * Client should receive all topology updates after reconnect. - * - * @throws Exception If failed. - */ - public void testClientReconnectOnRouterSuspendTopologyChange() throws Exception { - reconnectAfterSuspend(true); - } - - /** - * @param changeTop If {@code true} topology is changed after client disconnects - * @throws Exception if failed. - */ - private void reconnectAfterSuspend(boolean changeTop) throws Exception { - reconnectCnt = 2; - - startServerNodes(2); - - Ignite srv0 = G.ignite("server-0"); - TcpDiscoveryNode srv0Node = (TcpDiscoveryNode)srv0.cluster().localNode(); - - TcpDiscoveryNode srv1Node = (TcpDiscoveryNode)G.ignite("server-1").cluster().localNode(); - - clientIpFinder = new TcpDiscoveryVmIpFinder(); - - clientIpFinder.setAddresses( - Collections.singleton("localhost:" + srv0Node.discoveryPort())); - - startClientNodes(1); - - Ignite client = G.ignite("client-0"); - TcpDiscoveryNode clientNode = (TcpDiscoveryNode)client.cluster().localNode(); - TestTcpDiscoverySpi clientSpi = (TestTcpDiscoverySpi)client.configuration().getDiscoverySpi(); - - UUID clientNodeId = clientNode.id(); - - checkNodes(2, 1); - - clientIpFinder.setAddresses(Collections.singleton("localhost:" + srv1Node.discoveryPort())); - - srvFailedLatch = new CountDownLatch(1); - - attachListeners(2, 1); - - log.info("Pausing router"); - - TestTcpDiscoverySpi srvSpi = (TestTcpDiscoverySpi)srv0.configuration().getDiscoverySpi(); - - int joinedNodesNum = 3; - final CountDownLatch srvJoinedLatch = new CountDownLatch(joinedNodesNum); - - if (changeTop) { - client.events().localListen(new IgnitePredicate() { - @Override public boolean apply(Event e) { - srvJoinedLatch.countDown(); - - return true; - } - }, EVT_NODE_JOINED); - } - - srvSpi.pauseAll(true); - - if (changeTop) - startServerNodes(joinedNodesNum); - - try { - await(srvFailedLatch, 60_000); - - if (changeTop) - await(srvJoinedLatch, 5000); - - assertEquals("connected", clientSpi.getSpiState()); - assertEquals(clientNodeId, clientNode.id()); - assertEquals(srv1Node.id(), clientNode.clientRouterNodeId()); - } - finally { - srvSpi.resumeAll(); - } - } - - /** - * - */ - public void testReconnectAfterPause() throws Exception { - maxMissedClientHbs = 2; - - startServerNodes(2); - startClientNodes(1); - - Ignite client = G.ignite("client-0"); - TestTcpDiscoverySpi clientSpi = (TestTcpDiscoverySpi)client.configuration().getDiscoverySpi(); - - clientReconnectedLatch = new CountDownLatch(1); - - attachListeners(0, 1); - - clientSpi.pauseAll(false); - - try { - clientSpi.brakeConnection(); - - Thread.sleep(maxMissedClientHbs * DFLT_HEARTBEAT_FREQ * 2); - } - finally { - clientSpi.resumeAll(); - } - - await(clientReconnectedLatch); - } - - /** - * @throws Exception if failed. - */ - public void testReconnectAfterMassiveTopologyChange() throws Exception { - clientIpFinder = IP_FINDER; - - maxMissedClientHbs = 100; - netTimeout = 100000; - - int initSrvsNum = 5; - int killNum = 2; - int iterations = 3; - - startServerNodes(initSrvsNum); - startClientNodes(1); - - Ignite client = G.ignite("client-0"); - TcpDiscoveryNode clientNode = (TcpDiscoveryNode)client.cluster().localNode(); - TestTcpDiscoverySpi clientSpi = (TestTcpDiscoverySpi)client.configuration().getDiscoverySpi(); - final UUID clientNodeId = clientNode.id(); - - final CountDownLatch srvJoinedLatch = new CountDownLatch(iterations * killNum); - - client.events().localListen(new IgnitePredicate() { - @Override public boolean apply(Event e) { - srvJoinedLatch.countDown(); - - return true; - } - }, EVT_NODE_JOINED); - - int minAliveSrvId = 0; - - for (int i = 0; i < iterations; i++) { - startServerNodes(killNum); - - for (int j = 0; j < killNum; j++) { - failServer(minAliveSrvId); - - minAliveSrvId++; - } - - Thread.sleep(500); - } - - await(srvJoinedLatch); - assertEquals("connected", clientSpi.getSpiState()); - assertEquals(clientNodeId, clientNode.id()); - } - /** * @throws Exception If failed. */ @@ -1569,16 +1387,17 @@ else if (evt.type() == EVT_CLIENT_NODE_RECONNECTED) { srvSpi.failNode(client.cluster().localNode().id(), null); - assertTrue(disconnectLatch.await(5000, MILLISECONDS)); - assertTrue(failLatch.await(5000, MILLISECONDS)); - if (changeTop) { - startServerNodes(1); + Ignite g = startGrid("server-" + srvIdx.getAndIncrement()); + + srvNodeIds.add(g.cluster().localNode().id()); clientSpi.resumeAll(); } + assertTrue(disconnectLatch.await(5000, MILLISECONDS)); assertTrue(reconnectLatch.await(5000, MILLISECONDS)); + assertTrue(failLatch.await(5000, MILLISECONDS)); assertTrue(joinLatch.await(5000, MILLISECONDS)); long topVer = changeTop ? 5L : 4L; @@ -2184,20 +2003,6 @@ private void attachListeners(int srvCnt, int clientCnt) throws Exception { }, EVT_NODE_FAILED); } } - - if (clientReconnectedLatch != null) { - for (int i = 0; i < clientCnt; i++) { - G.ignite("client-" + i).events().localListen(new IgnitePredicate() { - @Override public boolean apply(Event evt) { - info("Reconnected event fired on client: " + evt); - - clientReconnectedLatch.countDown(); - - return true; - } - }, EVT_CLIENT_NODE_RECONNECTED); - } - } } /** @@ -2267,16 +2072,7 @@ else if (srvNodeIds.contains(id)) * @throws InterruptedException If interrupted. */ protected void await(CountDownLatch latch) throws InterruptedException { - await(latch, awaitTime()); - } - - /** - * @param latch Latch. - * @param timeout Timeout. - * @throws InterruptedException If interrupted. - */ - protected void await(CountDownLatch latch, long timeout) throws InterruptedException { - assertTrue("Latch count: " + latch.getCount(), latch.await(timeout, MILLISECONDS)); + assertTrue("Latch count: " + latch.getCount(), latch.await(awaitTime(), MILLISECONDS)); } /** @@ -2487,10 +2283,8 @@ public void pauseSocketWrite() { public void pauseAll(boolean suspend) { pauseResumeOperation(true, openSockLock, writeLock); - if (suspend) { - for (Thread t : impl.threads()) - t.suspend(); - } + if (suspend) + impl.workerThread().suspend(); } /** @@ -2499,8 +2293,7 @@ public void pauseAll(boolean suspend) { public void resumeAll() { pauseResumeOperation(false, openSockLock, writeLock); - for (IgniteSpiThread t : impl.threads()) - t.resume(); + impl.workerThread().resume(); } /** {@inheritDoc} */ From b541fdeedc1c6cff5c233c71ebc622f2c01c7824 Mon Sep 17 00:00:00 2001 From: Denis Mekhanikov Date: Fri, 27 Oct 2017 14:12:36 +0300 Subject: [PATCH 324/357] ignite-5860 Try process TcpDiscoveryClientReconnectMessage from socket reader instead of always processing it on coordinator. (cherry picked from commit 56a63f8) --- .../ignite/spi/discovery/tcp/ClientImpl.java | 52 ++- .../ignite/spi/discovery/tcp/ServerImpl.java | 311 +++++++++--------- .../spi/discovery/tcp/TcpDiscoveryImpl.java | 4 +- .../tcp/TcpClientDiscoverySpiSelfTest.java | 272 ++++++++++++++- 4 files changed, 459 insertions(+), 180 deletions(-) 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 c32c8e057d9fd..3657b514d2780 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 @@ -26,6 +26,7 @@ import java.net.SocketTimeoutException; import java.util.ArrayDeque; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -467,7 +468,8 @@ else if (state == DISCONNECTED) { } /** - * @param recon {@code True} if reconnects. + * @param prevAddr If reconnect is in progress, then previous address of the router the client was connected to + * and {@code null} otherwise. * @param timeout Timeout. * @return Opened socket or {@code null} if timeout. * @throws InterruptedException If interrupted. @@ -475,9 +477,9 @@ else if (state == DISCONNECTED) { * @see TcpDiscoverySpi#joinTimeout */ @SuppressWarnings("BusyWait") - @Nullable private T2 joinTopology(boolean recon, long timeout) + @Nullable private T2 joinTopology(InetSocketAddress prevAddr, long timeout) throws IgniteSpiException, InterruptedException { - Collection addrs = null; + List addrs = null; long startTime = U.currentTimeMillis(); @@ -486,7 +488,7 @@ else if (state == DISCONNECTED) { throw new InterruptedException(); while (addrs == null || addrs.isEmpty()) { - addrs = spi.resolvedAddresses(); + addrs = new ArrayList<>(spi.resolvedAddresses()); if (!F.isEmpty(addrs)) { if (log.isDebugEnabled()) @@ -506,17 +508,25 @@ else if (state == DISCONNECTED) { } } - Collection addrs0 = new ArrayList<>(addrs); + // Process failed node last. + if (prevAddr != null) { + int idx = addrs.indexOf(prevAddr); - Iterator it = addrs.iterator(); + if (idx != -1) + Collections.swap(addrs, idx, 0); + } + + Collection addrs0 = new ArrayList<>(addrs); boolean wait = false; - while (it.hasNext()) { + for (int i = addrs.size() - 1; i >= 0; i--) { if (Thread.currentThread().isInterrupted()) throw new InterruptedException(); - InetSocketAddress addr = it.next(); + InetSocketAddress addr = addrs.get(i); + + boolean recon = prevAddr != null; T3 sockAndRes; @@ -530,7 +540,7 @@ else if (state == DISCONNECTED) { } if (sockAndRes == null) { - it.remove(); + addrs.remove(i); continue; } @@ -852,8 +862,8 @@ private NavigableSet allVisibleNodes() { } /** {@inheritDoc} */ - @Override protected IgniteSpiThread workerThread() { - return msgWorker; + @Override protected Collection threads() { + return Arrays.asList(sockWriter, msgWorker); } /** @@ -1339,15 +1349,20 @@ private class Reconnector extends IgniteSpiThread { private boolean clientAck; /** */ - private boolean join; + private final boolean join; + + /** */ + private final InetSocketAddress prevAddr; /** * @param join {@code True} if reconnects during join. + * @param prevAddr Address of the node, that this client was previously connected to. */ - protected Reconnector(boolean join) { + protected Reconnector(boolean join, InetSocketAddress prevAddr) { super(spi.ignite().name(), "tcp-client-disco-reconnector", log); this.join = join; + this.prevAddr = prevAddr; } /** @@ -1377,7 +1392,7 @@ public void cancel() { try { while (true) { - T2 joinRes = joinTopology(true, timeout); + T2 joinRes = joinTopology(prevAddr, timeout); if (joinRes == null) { if (join) { @@ -1612,6 +1627,10 @@ else if (msg instanceof TcpDiscoveryNodeFailedMessage && } else if (msg instanceof SocketClosedMessage) { if (((SocketClosedMessage)msg).sock == currSock) { + Socket sock = currSock.sock; + + InetSocketAddress prevAddr = new InetSocketAddress(sock.getInetAddress(), sock.getPort()); + currSock = null; boolean join = joinLatch.getCount() > 0; @@ -1640,8 +1659,7 @@ else if (msg instanceof SocketClosedMessage) { assert reconnector == null; - final Reconnector reconnector = new Reconnector(join); - this.reconnector = reconnector; + reconnector = new Reconnector(join, prevAddr); reconnector.start(); } } @@ -1812,7 +1830,7 @@ private void tryJoin() throws InterruptedException { T2 joinRes; try { - joinRes = joinTopology(false, spi.joinTimeout); + joinRes = joinTopology(null, spi.joinTimeout); } catch (IgniteSpiException e) { joinError(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 189b37a6a0899..0f85e0612b942 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 @@ -222,6 +222,9 @@ class ServerImpl extends TcpDiscoveryImpl { /** Pending custom messages that should not be sent between NodeAdded and NodeAddFinished messages. */ private Queue pendingCustomMsgs = new ArrayDeque<>(); + /** Messages history used for client reconnect. */ + private final EnsuredMessageHistory msgHist = new EnsuredMessageHistory(); + /** If non-shared IP finder is used this flag shows whether IP finder contains local address. */ private boolean ipFinderHasLocAddr; @@ -1642,8 +1645,23 @@ private void clearNodeAddedMessage(TcpDiscoveryAbstractMessage msg) { } /** {@inheritDoc} */ - @Override protected IgniteSpiThread workerThread() { - return msgWorker; + @Override protected Collection threads() { + Collection threads; + + synchronized (mux) { + threads = new ArrayList<>(readers.size() + clientMsgWorkers.size() + 4); + threads.addAll(readers); + } + + threads.addAll(clientMsgWorkers.values()); + threads.add(tcpSrvr); + threads.add(ipFinderCleaner); + threads.add(msgWorker); + threads.add(statsPrinter); + + threads.removeAll(Collections.singleton(null)); + + return threads; } /** @@ -2091,7 +2109,9 @@ else if (msg instanceof TcpDiscoveryNodeLeftMessage) else if (msg instanceof TcpDiscoveryNodeFailedMessage) clearClientAddFinished(((TcpDiscoveryNodeFailedMessage)msg).failedNodeId()); - msgs.add(msg); + synchronized (msgs) { + msgs.add(msg); + } } /** @@ -2153,14 +2173,16 @@ private boolean mapsEqual(Map m1, Map m2) { // Client connection failed before it received TcpDiscoveryNodeAddedMessage. List res = null; - for (TcpDiscoveryAbstractMessage msg : msgs) { - if (msg instanceof TcpDiscoveryNodeAddedMessage) { - if (node.id().equals(((TcpDiscoveryNodeAddedMessage)msg).node().id())) - res = new ArrayList<>(msgs.size()); - } + synchronized (msgs) { + for (TcpDiscoveryAbstractMessage msg : msgs) { + if (msg instanceof TcpDiscoveryNodeAddedMessage) { + if (node.id().equals(((TcpDiscoveryNodeAddedMessage)msg).node().id())) + res = new ArrayList<>(msgs.size()); + } - if (res != null) - res.add(prepare(msg, node.id())); + if (res != null) + res.add(prepare(msg, node.id())); + } } if (log.isDebugEnabled()) { @@ -2173,20 +2195,26 @@ private boolean mapsEqual(Map m1, Map m2) { return res; } else { - if (msgs.isEmpty()) - return Collections.emptyList(); + Collection cp; - Collection cp = new ArrayList<>(msgs.size()); + boolean skip; - boolean skip = true; + synchronized (msgs) { + if (msgs.isEmpty()) + return Collections.emptyList(); - for (TcpDiscoveryAbstractMessage msg : msgs) { - if (skip) { - if (msg.id().equals(lastMsgId)) - skip = false; + cp = new ArrayList<>(msgs.size()); + + skip = true; + + for (TcpDiscoveryAbstractMessage msg : msgs) { + if (skip) { + if (msg.id().equals(lastMsgId)) + skip = false; + } + else + cp.add(prepare(msg, node.id())); } - else - cp.add(prepare(msg, node.id())); } cp = !skip ? cp : null; @@ -2475,9 +2503,6 @@ private class RingMessageWorker extends MessageWorkerAdapter pending = msgHist.messages(msg.lastMessageId(), node); - - if (pending != null) { - msg.pendingMessages(pending); - msg.success(true); - - if (log.isDebugEnabled()) - log.debug("Accept client reconnect, restored pending messages " + - "[locNodeId=" + locNodeId + ", clientNodeId=" + nodeId + ']'); - } - else { - if (log.isDebugEnabled()) - log.debug("Failing reconnecting client node because failed to restore pending " + - "messages [locNodeId=" + locNodeId + ", clientNodeId=" + nodeId + ']'); - - TcpDiscoveryNodeFailedMessage nodeFailedMsg = new TcpDiscoveryNodeFailedMessage(locNodeId, - node.id(), node.internalOrder()); - - processNodeFailedMessage(nodeFailedMsg); - - if (nodeFailedMsg.verified()) - msgHist.add(nodeFailedMsg); - } - } - else if (log.isDebugEnabled()) - log.debug("Reconnecting client node is already failed [nodeId=" + nodeId + ']'); - - if (isLocNodeRouter) { - ClientMessageWorker wrk = clientMsgWorkers.get(nodeId); - - if (wrk != null) - wrk.addMessage(msg); - else if (log.isDebugEnabled()) - log.debug("Failed to reconnect client node (disconnected during the process) [locNodeId=" + - locNodeId + ", clientNodeId=" + nodeId + ']'); - } - else { - if (sendMessageToRemotes(msg)) - sendMessageAcrossRing(msg); - } - } - else { - if (sendMessageToRemotes(msg)) - sendMessageAcrossRing(msg); - } - } - else { - if (isLocalNodeCoordinator()) - addMessage(new TcpDiscoveryDiscardMessage(locNodeId, msg.id(), false)); - - if (isLocNodeRouter) { - ClientMessageWorker wrk = clientMsgWorkers.get(nodeId); - - if (wrk != null) - wrk.addMessage(msg); - else if (log.isDebugEnabled()) - log.debug("Failed to reconnect client node (disconnected during the process) [locNodeId=" + - locNodeId + ", clientNodeId=" + nodeId + ']'); - } - else { - if (ring.hasRemoteNodes() && !isLocalNodeCoordinator()) - sendMessageAcrossRing(msg); - } - } - } - /** * Processes node added message. * @@ -4093,9 +4025,6 @@ private void processNodeAddedMessage(TcpDiscoveryNodeAddedMessage msg) { processNodeAddFinishedMessage(addFinishMsg); - if (addFinishMsg.verified()) - msgHist.add(addFinishMsg); - addMessage(new TcpDiscoveryDiscardMessage(locNodeId, msg.id(), false)); return; @@ -5156,9 +5085,6 @@ private void processHeartbeatMessage(TcpDiscoveryHeartbeatMessage msg) { locNodeId, clientNode.id(), clientNode.internalOrder()); processNodeFailedMessage(nodeFailedMsg); - - if (nodeFailedMsg.verified()) - msgHist.add(nodeFailedMsg); } } } @@ -5356,9 +5282,6 @@ private void processCustomMessage(TcpDiscoveryCustomEventMessage msg) { ackMsg.topologyVersion(msg.topologyVersion()); processCustomMessage(ackMsg); - - if (ackMsg.verified()) - msgHist.add(ackMsg); } catch (IgniteCheckedException e) { U.error(log, "Failed to marshal discovery custom message.", e); @@ -5460,12 +5383,8 @@ private void checkPendingCustomMessages() { if (joiningEmpty && isLocalNodeCoordinator()) { TcpDiscoveryCustomEventMessage msg; - while ((msg = pollPendingCustomeMessage()) != null) { + while ((msg = pollPendingCustomeMessage()) != null) processCustomMessage(msg); - - if (msg.verified()) - msgHist.add(msg); - } } } @@ -6010,24 +5929,22 @@ else if (msg instanceof TcpDiscoveryJoinRequestMessage) { } } else if (msg instanceof TcpDiscoveryClientReconnectMessage) { - if (clientMsgWrk != null) { - TcpDiscoverySpiState state = spiStateCopy(); + TcpDiscoverySpiState state = spiStateCopy(); - if (state == CONNECTED) { - spi.writeToSocket(msg, sock, RES_OK, sockTimeout); + if (state == CONNECTED) { + spi.writeToSocket(msg, sock, RES_OK, sockTimeout); - if (clientMsgWrk.getState() == State.NEW) - clientMsgWrk.start(); + if (clientMsgWrk != null && clientMsgWrk.getState() == State.NEW) + clientMsgWrk.start(); - msgWorker.addMessage(msg); + processClientReconnectMessage((TcpDiscoveryClientReconnectMessage)msg); - continue; - } - else { - spi.writeToSocket(msg, sock, RES_CONTINUE_JOIN, sockTimeout); + continue; + } + else { + spi.writeToSocket(msg, sock, RES_CONTINUE_JOIN, sockTimeout); - break; - } + break; } } else if (msg instanceof TcpDiscoveryDuplicateIdMessage) { @@ -6256,6 +6173,100 @@ else if (msg instanceof TcpDiscoveryRingLatencyCheckMessage) { } } + /** + * Processes client reconnect message. + * + * @param msg Client reconnect message. + */ + private void processClientReconnectMessage(TcpDiscoveryClientReconnectMessage msg) { + UUID nodeId = msg.creatorNodeId(); + + UUID locNodeId = getLocalNodeId(); + + boolean isLocNodeRouter = msg.routerNodeId().equals(locNodeId); + + TcpDiscoveryNode node = ring.node(nodeId); + + assert node == null || node.isClient(); + + if (node != null) { + node.clientRouterNodeId(msg.routerNodeId()); + node.aliveCheck(spi.maxMissedClientHbs); + } + + if (!msg.verified()) { + if (isLocNodeRouter || isLocalNodeCoordinator()) { + if (node != null) { + Collection pending = msgHist.messages(msg.lastMessageId(), node); + + if (pending != null) { + msg.verify(locNodeId); + msg.pendingMessages(pending); + msg.success(true); + + if (log.isDebugEnabled()) + log.debug("Accept client reconnect, restored pending messages " + + "[locNodeId=" + locNodeId + ", clientNodeId=" + nodeId + ']'); + } + else if (!isLocalNodeCoordinator()) { + if (log.isDebugEnabled()) + log.debug("Failed to restore pending messages for reconnecting client. " + + "Forwarding reconnection message to coordinator " + + "[locNodeId=" + locNodeId + ", clientNodeId=" + nodeId + ']'); + } + else { + msg.verify(locNodeId); + + if (log.isDebugEnabled()) + log.debug("Failing reconnecting client node because failed to restore pending " + + "messages [locNodeId=" + locNodeId + ", clientNodeId=" + nodeId + ']'); + + TcpDiscoveryNodeFailedMessage nodeFailedMsg = new TcpDiscoveryNodeFailedMessage(locNodeId, + node.id(), node.internalOrder()); + + msgWorker.addMessage(nodeFailedMsg); + } + } + else { + msg.verify(locNodeId); + + if (log.isDebugEnabled()) + log.debug("Reconnecting client node is already failed [nodeId=" + nodeId + ']'); + } + + if (msg.verified() && isLocNodeRouter) { + ClientMessageWorker wrk = clientMsgWorkers.get(nodeId); + + if (wrk != null) + wrk.addMessage(msg); + else if (log.isDebugEnabled()) + log.debug("Failed to reconnect client node (disconnected during the process) [locNodeId=" + + locNodeId + ", clientNodeId=" + nodeId + ']'); + } + else + msgWorker.addMessage(msg); + } + else + msgWorker.addMessage(msg); + } + else { + if (isLocalNodeCoordinator()) + msgWorker.addMessage(new TcpDiscoveryDiscardMessage(locNodeId, msg.id(), false)); + + if (isLocNodeRouter) { + ClientMessageWorker wrk = clientMsgWorkers.get(nodeId); + + if (wrk != null) + wrk.addMessage(msg); + else if (log.isDebugEnabled()) + log.debug("Failed to reconnect client node (disconnected during the process) [locNodeId=" + + locNodeId + ", clientNodeId=" + nodeId + ']'); + } + else if (ring.hasRemoteNodes() && !isLocalNodeCoordinator()) + msgWorker.addMessage(msg); + } + } + /** * Processes client heartbeat message. * 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 cb85dc1c44b86..79936f3c3c1f5 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 @@ -290,9 +290,9 @@ protected static String threadStatus(Thread t) { /** * FOR TEST ONLY!!! * - * @return Worker thread. + * @return Worker threads. */ - protected abstract IgniteSpiThread workerThread(); + protected abstract Collection threads(); /** * @throws IgniteSpiException If failed. 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 419497753bbc2..190556a2db2d0 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 @@ -62,8 +62,8 @@ import org.apache.ignite.spi.IgniteSpiException; import org.apache.ignite.spi.IgniteSpiOperationTimeoutException; import org.apache.ignite.spi.IgniteSpiOperationTimeoutHelper; +import org.apache.ignite.spi.IgniteSpiThread; 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.TcpDiscoveryClientReconnectMessage; @@ -88,7 +88,7 @@ */ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest { /** */ - private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + private static final TcpDiscoveryVmIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); /** */ protected static final AtomicInteger srvIdx = new AtomicInteger(); @@ -123,6 +123,9 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest { /** */ private static CountDownLatch clientFailedLatch; + /** */ + private static CountDownLatch clientReconnectedLatch; + /** */ private static CountDownLatch msgLatch; @@ -138,6 +141,9 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest { /** */ protected long netTimeout = TcpDiscoverySpi.DFLT_NETWORK_TIMEOUT; + /** */ + protected Integer reconnectCnt; + /** */ private boolean longSockTimeouts; @@ -208,6 +214,9 @@ else if (gridName.startsWith("client")) { disco.setJoinTimeout(joinTimeout); disco.setNetworkTimeout(netTimeout); + if (reconnectCnt != null) + disco.setReconnectCount(reconnectCnt); + disco.setClientReconnectDisabled(reconnectDisabled); if (disco instanceof TestTcpDiscoverySpi) @@ -254,6 +263,7 @@ protected TcpDiscoverySpi getDiscoverySpi() { clientIpFinder = null; joinTimeout = TcpDiscoverySpi.DFLT_JOIN_TIMEOUT; netTimeout = TcpDiscoverySpi.DFLT_NETWORK_TIMEOUT; + maxMissedClientHbs = TcpDiscoverySpi.DFLT_MAX_MISSED_CLIENT_HEARTBEATS; longSockTimeouts = false; assert G.allGrids().isEmpty(); @@ -534,6 +544,221 @@ public void testClientReconnectOnRouterFail() throws Exception { checkNodes(2, 3); } + /** + * Client should reconnect to available server without EVT_CLIENT_NODE_RECONNECTED event. + * + * @throws Exception If failed. + */ + public void testClientReconnectOnRouterSuspend() throws Exception { + reconnectAfterSuspend(false); + } + + /** + * Client should receive all topology updates after reconnect. + * + * @throws Exception If failed. + */ + public void testClientReconnectOnRouterSuspendTopologyChange() throws Exception { + reconnectAfterSuspend(true); + } + + /** + * @param changeTop If {@code true} topology is changed after client disconnects + * @throws Exception if failed. + */ + private void reconnectAfterSuspend(boolean changeTop) throws Exception { + reconnectCnt = 2; + + startServerNodes(2); + + Ignite srv0 = grid("server-0"); + TcpDiscoveryNode srv0Node = (TcpDiscoveryNode)srv0.cluster().localNode(); + + TcpDiscoveryNode srv1Node = (TcpDiscoveryNode)grid("server-1").cluster().localNode(); + + clientIpFinder = new TcpDiscoveryVmIpFinder(); + + clientIpFinder.setAddresses( + Collections.singleton("localhost:" + srv0Node.discoveryPort())); + + startClientNodes(1); + + Ignite client = grid("client-0"); + TcpDiscoveryNode clientNode = (TcpDiscoveryNode)client.cluster().localNode(); + TestTcpDiscoverySpi clientSpi = (TestTcpDiscoverySpi)client.configuration().getDiscoverySpi(); + + UUID clientNodeId = clientNode.id(); + + checkNodes(2, 1); + + clientIpFinder.setAddresses(Collections.singleton("localhost:" + srv1Node.discoveryPort())); + + srvFailedLatch = new CountDownLatch(1); + + attachListeners(2, 1); + + log.info("Pausing router"); + + TestTcpDiscoverySpi srvSpi = (TestTcpDiscoverySpi)srv0.configuration().getDiscoverySpi(); + + int joinedNodesNum = 3; + final CountDownLatch srvJoinedLatch = new CountDownLatch(joinedNodesNum); + + if (changeTop) { + client.events().localListen(new IgnitePredicate() { + @Override public boolean apply(Event e) { + srvJoinedLatch.countDown(); + + return true; + } + }, EVT_NODE_JOINED); + } + + srvSpi.pauseAll(true); + + if (changeTop) + startServerNodes(joinedNodesNum); + + try { + await(srvFailedLatch, 60_000); + + if (changeTop) + await(srvJoinedLatch, 5000); + + assertEquals("connected", clientSpi.getSpiState()); + assertEquals(clientNodeId, clientNode.id()); + assertEquals(srv1Node.id(), clientNode.clientRouterNodeId()); + } + finally { + srvSpi.resumeAll(); + } + } + + /** + * @throws Exception if failed. + */ + public void testClientReconnectHistoryMissingOnRouter() throws Exception { + maxMissedClientHbs = 30; + netTimeout = 60000; + + startServerNodes(2); + + Ignite srv0 = grid("server-0"); + TcpDiscoveryNode srv0Node = (TcpDiscoveryNode)srv0.cluster().localNode(); + + clientIpFinder = new TcpDiscoveryVmIpFinder(); + clientIpFinder.setAddresses( + Collections.singleton("localhost:" + srv0Node.discoveryPort())); + + startClientNodes(1); + + attachListeners(0, 1); + + Ignite client = grid("client-0"); + TcpDiscoveryNode clientNode = (TcpDiscoveryNode)client.cluster().localNode(); + TestTcpDiscoverySpi clientSpi = (TestTcpDiscoverySpi)client.configuration().getDiscoverySpi(); + UUID clientNodeId = clientNode.id(); + + checkNodes(2, 1); + + clientSpi.pauseAll(true); + + stopGrid(srv0.name()); + + startServerNodes(1); + + Ignite srv2 = grid("server-2"); + TcpDiscoveryNode srv2Node = (TcpDiscoveryNode)srv2.cluster().localNode(); + clientIpFinder.setAddresses( + Collections.singleton("localhost:" + srv2Node.discoveryPort())); + + clientSpi.resumeAll(); + + awaitPartitionMapExchange(); + + assertEquals("connected", clientSpi.getSpiState()); + assertEquals(clientNodeId, clientNode.id()); + assertEquals(srv2Node.id(), clientNode.clientRouterNodeId()); + } + + /** + * @throws Exception If failed. + */ + public void testReconnectAfterPause() throws Exception { + startServerNodes(2); + startClientNodes(1); + + Ignite client = grid("client-0"); + TestTcpDiscoverySpi clientSpi = (TestTcpDiscoverySpi)client.configuration().getDiscoverySpi(); + + clientReconnectedLatch = new CountDownLatch(1); + + attachListeners(0, 1); + + clientSpi.pauseAll(false); + + try { + clientSpi.brakeConnection(); + + Thread.sleep(maxMissedClientHbs * clientSpi.getHeartbeatFrequency() * 2); + } + finally { + clientSpi.resumeAll(); + } + + await(clientReconnectedLatch); + } + + /** + * @throws Exception if failed. + */ + public void testReconnectAfterMassiveTopologyChange() throws Exception { + clientIpFinder = IP_FINDER; + + maxMissedClientHbs = 30; + netTimeout = 60000; + + int initSrvsNum = 5; + int killNum = 3; + int iterations = 10; + + startServerNodes(initSrvsNum); + startClientNodes(1); + + Ignite client = grid("client-0"); + TcpDiscoveryNode clientNode = (TcpDiscoveryNode)client.cluster().localNode(); + TestTcpDiscoverySpi clientSpi = (TestTcpDiscoverySpi)client.configuration().getDiscoverySpi(); + final UUID clientNodeId = clientNode.id(); + + final CountDownLatch srvJoinedLatch = new CountDownLatch(iterations * killNum); + + client.events().localListen(new IgnitePredicate() { + @Override public boolean apply(Event e) { + srvJoinedLatch.countDown(); + + return true; + } + }, EVT_NODE_JOINED); + + int minAliveSrvId = 0; + + for (int i = 0; i < iterations; i++) { + for (int j = 0; j < killNum; j++) { + stopGrid(minAliveSrvId); + + minAliveSrvId++; + } + + startServerNodes(killNum); + + awaitPartitionMapExchange(); + } + + await(srvJoinedLatch); + assertEquals("connected", clientSpi.getSpiState()); + assertEquals(clientNodeId, clientNode.id()); + } + /** * @throws Exception If failed. */ @@ -1387,17 +1612,16 @@ else if (evt.type() == EVT_CLIENT_NODE_RECONNECTED) { srvSpi.failNode(client.cluster().localNode().id(), null); - if (changeTop) { - Ignite g = startGrid("server-" + srvIdx.getAndIncrement()); + assertTrue(disconnectLatch.await(5000, MILLISECONDS)); + assertTrue(failLatch.await(5000, MILLISECONDS)); - srvNodeIds.add(g.cluster().localNode().id()); + if (changeTop) { + startServerNodes(1); clientSpi.resumeAll(); } - assertTrue(disconnectLatch.await(5000, MILLISECONDS)); assertTrue(reconnectLatch.await(5000, MILLISECONDS)); - assertTrue(failLatch.await(5000, MILLISECONDS)); assertTrue(joinLatch.await(5000, MILLISECONDS)); long topVer = changeTop ? 5L : 4L; @@ -2003,6 +2227,20 @@ private void attachListeners(int srvCnt, int clientCnt) throws Exception { }, EVT_NODE_FAILED); } } + + if (clientReconnectedLatch != null) { + for (int i = 0; i < clientCnt; i++) { + G.ignite("client-" + i).events().localListen(new IgnitePredicate() { + @Override public boolean apply(Event evt) { + info("Reconnected event fired on client: " + evt); + + clientReconnectedLatch.countDown(); + + return true; + } + }, EVT_CLIENT_NODE_RECONNECTED); + } + } } /** @@ -2072,7 +2310,16 @@ else if (srvNodeIds.contains(id)) * @throws InterruptedException If interrupted. */ protected void await(CountDownLatch latch) throws InterruptedException { - assertTrue("Latch count: " + latch.getCount(), latch.await(awaitTime(), MILLISECONDS)); + await(latch, awaitTime()); + } + + /** + * @param latch Latch. + * @param timeout Timeout. + * @throws InterruptedException If interrupted. + */ + protected void await(CountDownLatch latch, long timeout) throws InterruptedException { + assertTrue("Latch count: " + latch.getCount(), latch.await(timeout, MILLISECONDS)); } /** @@ -2283,8 +2530,10 @@ public void pauseSocketWrite() { public void pauseAll(boolean suspend) { pauseResumeOperation(true, openSockLock, writeLock); - if (suspend) - impl.workerThread().suspend(); + if (suspend) { + for (Thread t : impl.threads()) + t.suspend(); + } } /** @@ -2293,7 +2542,8 @@ public void pauseAll(boolean suspend) { public void resumeAll() { pauseResumeOperation(false, openSockLock, writeLock); - impl.workerThread().resume(); + for (IgniteSpiThread t : impl.threads()) + t.resume(); } /** {@inheritDoc} */ From 0ecc05526c03f7e1099dfc660971b20916dc66d5 Mon Sep 17 00:00:00 2001 From: Oleg Ostanin Date: Tue, 24 Oct 2017 14:53:16 +0300 Subject: [PATCH 325/357] IGNITE-6660 Python Redis example fails for python 3 run. --- examples/config/redis/example-redis.xml | 74 +++++++++++++++++++++++++ examples/redis/redis-example.py | 16 +++--- 2 files changed, 82 insertions(+), 8 deletions(-) create mode 100644 examples/config/redis/example-redis.xml diff --git a/examples/config/redis/example-redis.xml b/examples/config/redis/example-redis.xml new file mode 100644 index 0000000000000..5db27a1848d6f --- /dev/null +++ b/examples/config/redis/example-redis.xml @@ -0,0 +1,74 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 127.0.0.1:47500..47509 + + + + + + + + diff --git a/examples/redis/redis-example.py b/examples/redis/redis-example.py index f6c4f367048de..8c72a69a7f309 100644 --- a/examples/redis/redis-example.py +++ b/examples/redis/redis-example.py @@ -29,34 +29,34 @@ r.set('k1', 1) # check. -print 'Value for "k1": %s' % r.get('k1') +print('Value for "k1": %s' % r.get('k1')) # change entry's value. r.set('k1', 'new_val') # check. -print 'Value for "k1": %s' % r.get('k1') +print('Value for "k1": %s' % r.get('k1')) # set another entry. r.set('k2', 2) # check. -print 'Value for "k2": %s' % r.get('k2') +print('Value for "k2": %s' % r.get('k2')) # get both values. -print 'Values for "k1" and "k2": %s' % r.mget('k1', 'k2') +print('Values for "k1" and "k2": %s' % r.mget('k1', 'k2')) # delete one entry. r.delete('k1') # check one entry left. -print 'Values for "k1" and "k2": %s' % r.mget('k1', 'k2') +print('Values for "k1" and "k2": %s' % r.mget('k1', 'k2')) # check db size -print 'Db size: %d' % r.dbsize() +print('Db size: %d' % r.dbsize()) # increment. -print 'Value for incremented "inc_k" : %s' % r.incr('inc_k') +print('Value for incremented "inc_k" : %s' % r.incr('inc_k')) # increment again. -print 'Value for incremented "inc_k" : %s' % r.incr('inc_k') +print('Value for incremented "inc_k" : %s' % r.incr('inc_k')) From 18ed82c6b5cf2674bc98b83f94f7899776dd2334 Mon Sep 17 00:00:00 2001 From: voipp Date: Tue, 14 Feb 2017 15:08:59 +0300 Subject: [PATCH 326/357] IGNITE-4492 Add MBean for StripedExecutor. This closes #1491. (cherry picked from commit 8e12513) --- .../apache/ignite/internal/IgniteKernal.java | 48 ++++- .../StripedExecutorMXBeanAdapter.java | 90 ++++++++++ .../ignite/internal/util/StripedExecutor.java | 55 +++++- .../ignite/mxbean/StripedExecutorMXBean.java | 90 ++++++++++ .../internal/util/StripedExecutorTest.java | 168 ++++++++++++++++++ .../IgniteComputeGridTestSuite.java | 2 + 6 files changed, 447 insertions(+), 6 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/StripedExecutorMXBeanAdapter.java create mode 100644 modules/core/src/main/java/org/apache/ignite/mxbean/StripedExecutorMXBean.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/util/StripedExecutorTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java index 623072e3e6b53..81c9e548ced4b 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 @@ -168,6 +168,7 @@ import org.apache.ignite.marshaller.optimized.OptimizedMarshaller; import org.apache.ignite.mxbean.ClusterLocalNodeMetricsMXBean; import org.apache.ignite.mxbean.IgniteMXBean; +import org.apache.ignite.mxbean.StripedExecutorMXBean; import org.apache.ignite.mxbean.ThreadPoolMXBean; import org.apache.ignite.plugin.IgnitePlugin; import org.apache.ignite.plugin.PluginNotFoundException; @@ -308,6 +309,10 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable { @GridToStringExclude private ObjectName restExecSvcMBean; + /** */ + @GridToStringExclude + private ObjectName stripedExecSvcMBean; + /** Kernal start timestamp. */ private long startTime = U.currentTimeMillis(); @@ -1011,6 +1016,7 @@ public void start( restExecSvc, utilityCachePool, marshCachePool); + registerStripedExecutorMBean(stripedExecSvc); // Lifecycle bean notifications. notifyLifecycleBeans(AFTER_NODE_START); @@ -1598,7 +1604,14 @@ private void registerLocalNodeMBean() throws IgniteCheckedException { } } - /** @throws IgniteCheckedException If registration failed. */ + /** + * @param execSvc + * @param sysExecSvc + * @param p2pExecSvc + * @param mgmtExecSvc + * @param restExecSvc + * @throws IgniteCheckedException If failed. + */ private void registerExecutorMBeans( ExecutorService execSvc, ExecutorService sysExecSvc, @@ -1648,8 +1661,34 @@ private ObjectName registerExecutorMBean(ExecutorService exec, String name) thro return res; } catch (JMException e) { - throw new IgniteCheckedException("Failed to register executor service MBean [name=" + name + ", exec=" + exec + ']', - e); + throw new IgniteCheckedException("Failed to register executor service MBean [name=" + name + + ", exec=" + exec + ']', e); + } + } + + /** + * @param stripedExecSvc Executor service. + * @throws IgniteCheckedException If registration failed. + */ + private void registerStripedExecutorMBean(StripedExecutor stripedExecSvc) throws IgniteCheckedException { + if (stripedExecSvc != null) { + String name = "StripedExecutor"; + + try { + stripedExecSvcMBean = U.registerMBean( + cfg.getMBeanServer(), + cfg.getGridName(), + "Thread Pools", + name, + new StripedExecutorMXBeanAdapter(stripedExecSvc), + StripedExecutorMXBean.class); + + if (log.isDebugEnabled()) + log.debug("Registered executor service MBean: " + stripedExecSvcMBean); + } catch (JMException e) { + throw new IgniteCheckedException("Failed to register executor service MBean [name=" + + name + ", exec=" + stripedExecSvc + ']', e); + } } } @@ -2116,7 +2155,8 @@ else if (state == STARTING) unregisterMBean(locNodeMBean) & unregisterMBean(utilityExecSvcMBean) & unregisterMBean(marshallerExecSvcMBean) & - unregisterMBean(restExecSvcMBean) + unregisterMBean(restExecSvcMBean) & + unregisterMBean(stripedExecSvcMBean) )) errOnStop = false; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/StripedExecutorMXBeanAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/StripedExecutorMXBeanAdapter.java new file mode 100644 index 0000000000000..e6811b7aee762 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/StripedExecutorMXBeanAdapter.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal; + +import java.util.concurrent.ExecutorService; +import org.apache.ignite.internal.util.StripedExecutor; +import org.apache.ignite.mxbean.StripedExecutorMXBean; + +/** + * Adapter for {@link StripedExecutorMXBean} which delegates all method calls to the underlying + * {@link ExecutorService} instance. + */ +public class StripedExecutorMXBeanAdapter implements StripedExecutorMXBean { + /** */ + private final StripedExecutor exec; + + /** + * @param exec Executor service + */ + StripedExecutorMXBeanAdapter(StripedExecutor exec) { + assert exec != null; + + this.exec = exec; + } + + /** {@inheritDoc} */ + @Override public void checkStarvation() { + exec.checkStarvation(); + } + + /** {@inheritDoc} */ + @Override public int getStripesCount() { + return exec.stripes(); + } + + /** {@inheritDoc} */ + @Override public boolean isShutdown() { + return exec.isShutdown(); + } + + /** {@inheritDoc} */ + @Override public boolean isTerminated() { + return exec.isTerminated(); + } + + /** {@inheritDoc} */ + @Override public int getTotalQueueSize() { + return exec.queueSize(); + } + + /** {@inheritDoc} */ + @Override public long getTotalCompletedTasksCount() { + return exec.completedTasks(); + } + + /** {@inheritDoc} */ + @Override public long[] getStripesCompletedTasksCounts() { + return exec.stripesCompletedTasks(); + } + + /** {@inheritDoc} */ + @Override public int getActiveCount() { + return exec.activeStripesCount(); + } + + /** {@inheritDoc} */ + @Override public boolean[] getStripesActiveStatuses() { + return exec.stripesActiveStatuses(); + } + + /** {@inheritDoc} */ + @Override public int[] getStripesQueueSizes() { + return exec.stripesQueueSizes(); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java b/modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java index 201cb34c1e4e8..e70f0ce2b761a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java @@ -55,9 +55,10 @@ public class StripedExecutor implements ExecutorService { private final IgniteLogger log; /** - * Constructor. - * * @param cnt Count. + * @param gridName Node name. + * @param poolName Pool name. + * @param log Logger. */ public StripedExecutor(int cnt, String gridName, String poolName, final IgniteLogger log) { A.ensure(cnt > 0, "cnt > 0"); @@ -267,6 +268,56 @@ public long completedTasks() { return cnt; } + /** + * @return Completed tasks per stripe count. + */ + public long[] stripesCompletedTasks() { + long[] res = new long[stripes()]; + + for (int i = 0; i < res.length; i++) + res[i] = stripes[i].completedCnt; + + return res; + } + + /** + * @return Number of active tasks per stripe. + */ + public boolean[] stripesActiveStatuses() { + boolean[] res = new boolean[stripes()]; + + for (int i = 0; i < res.length; i++) + res[i] = stripes[i].active; + + return res; + } + + /** + * @return Number of active tasks. + */ + public int activeStripesCount() { + int res = 0; + + for (boolean status : stripesActiveStatuses()) { + if (status) + res++; + } + + return res; + } + + /** + * @return Size of queue per stripe. + */ + public int[] stripesQueueSizes() { + int[] res = new int[stripes()]; + + for (int i = 0; i < res.length; i++) + res[i] = stripes[i].queueSize(); + + return res; + } + /** * Operation not supported. */ diff --git a/modules/core/src/main/java/org/apache/ignite/mxbean/StripedExecutorMXBean.java b/modules/core/src/main/java/org/apache/ignite/mxbean/StripedExecutorMXBean.java new file mode 100644 index 0000000000000..7428b19600d20 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/mxbean/StripedExecutorMXBean.java @@ -0,0 +1,90 @@ +/* + * 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.mxbean; + +/** + * MBean that provides access to information about striped executor service. + */ +@MXBeanDescription("MBean that provides access to information about striped executor service.") +public interface StripedExecutorMXBean { + /** + * Checks for starvation in striped pool, dumps in log information if potential starvation + * was found. + */ + @MXBeanDescription("Checks for starvation in striped pool.") + public void checkStarvation(); + + /** + * @return Stripes count. + */ + @MXBeanDescription("Stripes count.") + public int getStripesCount(); + + /** + * + * @return {@code True} if this executor has been shut down. + */ + @MXBeanDescription("True if this executor has been shut down.") + public boolean isShutdown(); + + /** + * Note that + * {@code isTerminated()} is never {@code true} unless either {@code shutdown()} or + * {@code shutdownNow()} was called first. + * + * @return {@code True} if all tasks have completed following shut down. + */ + @MXBeanDescription("True if all tasks have completed following shut down.") + public boolean isTerminated(); + + /** + * @return Return total queue size of all stripes. + */ + @MXBeanDescription("Total queue size of all stripes.") + public int getTotalQueueSize(); + + /** + * @return Completed tasks count. + */ + @MXBeanDescription("Completed tasks count of all stripes.") + public long getTotalCompletedTasksCount(); + + /** + * @return Number of completed tasks per stripe. + */ + @MXBeanDescription("Number of completed tasks per stripe.") + public long[] getStripesCompletedTasksCounts(); + + /** + * @return Number of active tasks. + */ + @MXBeanDescription("Number of active tasks of all stripes.") + public int getActiveCount(); + + /** + * @return Number of active tasks per stripe. + */ + @MXBeanDescription("Number of active tasks per stripe.") + public boolean[] getStripesActiveStatuses(); + + /** + * @return Size of queue per stripe. + */ + @MXBeanDescription("Size of queue per stripe.") + public int[] getStripesQueueSizes(); +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/StripedExecutorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/StripedExecutorTest.java new file mode 100644 index 0000000000000..543907fcde60a --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/util/StripedExecutorTest.java @@ -0,0 +1,168 @@ +/* + * 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.util; + +import org.apache.ignite.logger.java.JavaLogger; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * + */ +public class StripedExecutorTest extends GridCommonAbstractTest { + /** */ + private StripedExecutor stripedExecSvc; + + /** {@inheritDoc} */ + @Override public void beforeTest() { + stripedExecSvc = new StripedExecutor(3, "foo name", "pool name", new JavaLogger()); + } + + /** {@inheritDoc} */ + @Override public void afterTest() { + stripedExecSvc.shutdown(); + } + + /** + * @throws Exception If failed. + */ + public void testCompletedTasks() throws Exception { + stripedExecSvc.execute(0, new TestRunnable()); + stripedExecSvc.execute(1, new TestRunnable()); + + sleepASec(); + + assertEquals(2, stripedExecSvc.completedTasks()); + } + + /** + * @throws Exception If failed. + */ + public void testStripesCompletedTasks() throws Exception { + stripedExecSvc.execute(0, new TestRunnable()); + stripedExecSvc.execute(1, new TestRunnable()); + + sleepASec(); + + long[] completedTaks = stripedExecSvc.stripesCompletedTasks(); + + assertEquals(1, completedTaks[0]); + assertEquals(1, completedTaks[1]); + assertEquals(0, completedTaks[2]); + } + + /** + * @throws Exception If failed. + */ + public void testStripesActiveStatuses() throws Exception { + stripedExecSvc.execute(0, new TestRunnable()); + stripedExecSvc.execute(1, new TestRunnable(true)); + + sleepASec(); + + boolean[] statuses = stripedExecSvc.stripesActiveStatuses(); + + assertFalse(statuses[0]); + assertTrue(statuses[1]); + assertFalse(statuses[0]); + } + + /** + * @throws Exception If failed. + */ + public void testActiveStripesCount() throws Exception { + stripedExecSvc.execute(0, new TestRunnable()); + stripedExecSvc.execute(1, new TestRunnable(true)); + + sleepASec(); + + assertEquals(1, stripedExecSvc.activeStripesCount()); + } + + /** + * @throws Exception If failed. + */ + public void testStripesQueueSizes() throws Exception { + stripedExecSvc.execute(0, new TestRunnable()); + stripedExecSvc.execute(0, new TestRunnable(true)); + stripedExecSvc.execute(0, new TestRunnable(true)); + stripedExecSvc.execute(1, new TestRunnable(true)); + stripedExecSvc.execute(1, new TestRunnable(true)); + stripedExecSvc.execute(1, new TestRunnable(true)); + + sleepASec(); + + int[] queueSizes = stripedExecSvc.stripesQueueSizes(); + + assertEquals(1, queueSizes[0]); + assertEquals(2, queueSizes[1]); + assertEquals(0, queueSizes[2]); + } + + /** + * @throws Exception If failed. + */ + public void testQueueSize() throws Exception { + stripedExecSvc.execute(1, new TestRunnable()); + stripedExecSvc.execute(1, new TestRunnable(true)); + stripedExecSvc.execute(1, new TestRunnable(true)); + + sleepASec(); + + assertEquals(1, stripedExecSvc.queueSize()); + } + + /** + * + */ + private final class TestRunnable implements Runnable { + /** */ + private final boolean infinitely; + + /** + * + */ + public TestRunnable() { + this(false); + } + + /** + * @param infinitely {@code True} if should sleep infinitely. + */ + public TestRunnable(boolean infinitely) { + this.infinitely = infinitely; + } + + /** {@inheritDoc} */ + @Override public void run() { + try { + while (infinitely) + sleepASec(); + } + catch (InterruptedException e) { + info("Got interrupted exception while sleeping: " + e); + } + } + } + + /** + * @throws InterruptedException If interrupted. + */ + private void sleepASec() throws InterruptedException { + Thread.sleep(1000); + } +} \ No newline at end of file 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 9e6c72e0e6f9c..7e299ca9a8cff 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 @@ -74,6 +74,7 @@ import org.apache.ignite.internal.managers.checkpoint.GridCheckpointTaskSelfTest; import org.apache.ignite.internal.managers.communication.GridCommunicationManagerListenersSelfTest; import org.apache.ignite.internal.processors.compute.PublicThreadpoolStarvationTest; +import org.apache.ignite.internal.util.StripedExecutorTest; import org.apache.ignite.p2p.GridMultinodeRedeployContinuousModeSelfTest; import org.apache.ignite.p2p.GridMultinodeRedeployIsolatedModeSelfTest; import org.apache.ignite.p2p.GridMultinodeRedeployPrivateModeSelfTest; @@ -156,6 +157,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(IgniteRoundRobinErrorAfterClientReconnectTest.class); suite.addTestSuite(PublicThreadpoolStarvationTest.class); suite.addTestSuite(IgniteComputeJobOneThreadTest.class); + suite.addTestSuite(StripedExecutorTest.class); return suite; } From 0a63367b9926fd216e49ef8614cd3dee6e5d7633 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Fri, 27 Oct 2017 15:03:07 +0300 Subject: [PATCH 327/357] Backported IGNITE-5100 Do not remove active exchange futures from ExchangeFutureSet. (cherry picked from commit ecb9e4d) --- .../apache/ignite/IgniteSystemProperties.java | 3 + .../GridCachePartitionExchangeManager.java | 15 +++- ...ePartitionExchangeManagerHistSizeTest.java | 76 +++++++++++++++++++ .../testsuites/IgniteCacheTestSuite5.java | 3 + .../query/h2/DmlStatementsProcessor.java | 11 +++ 5 files changed, 104 insertions(+), 4 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/GridCachePartitionExchangeManagerHistSizeTest.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 3c5a5a6c5d44b..4bae00e4b739c 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -157,6 +157,9 @@ public final class IgniteSystemProperties { */ public static final String IGNITE_CONSOLE_APPENDER = "IGNITE_CONSOLE_APPENDER"; + /** Maximum size for exchange history. Default value is {@code 1000}.*/ + public static final String IGNITE_EXCHANGE_HISTORY_SIZE = "IGNITE_EXCHANGE_HISTORY_SIZE"; + /** * Name of the system property defining name of command line program. */ 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 183debd4148d1..93fe6a835053b 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 @@ -117,7 +117,8 @@ */ public class GridCachePartitionExchangeManager extends GridCacheSharedManagerAdapter { /** Exchange history size. */ - private static final int EXCHANGE_HISTORY_SIZE = 1000; + private static final int EXCHANGE_HISTORY_SIZE = + IgniteSystemProperties.getInteger(IgniteSystemProperties.IGNITE_EXCHANGE_HISTORY_SIZE, 1_000); /** Atomic reference for pending timeout object. */ private AtomicReference pendingResend = new AtomicReference<>(); @@ -1995,8 +1996,14 @@ private ExchangeFutureSet() { GridDhtPartitionsExchangeFuture fut) { GridDhtPartitionsExchangeFuture cur = super.addx(fut); - while (size() > EXCHANGE_HISTORY_SIZE) + while (size() > EXCHANGE_HISTORY_SIZE) { + GridDhtPartitionsExchangeFuture last = last(); + + if (!last.isDone()) + break; + removeLast(); + } // Return the value in the set. return cur == null ? fut : cur; @@ -2004,8 +2011,8 @@ private ExchangeFutureSet() { /** {@inheritDoc} */ @Nullable @Override public synchronized GridDhtPartitionsExchangeFuture removex( - GridDhtPartitionsExchangeFuture val - ) { + GridDhtPartitionsExchangeFuture val) { + return super.removex(val); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridCachePartitionExchangeManagerHistSizeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridCachePartitionExchangeManagerHistSizeTest.java new file mode 100644 index 0000000000000..934f40d94376f --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/GridCachePartitionExchangeManagerHistSizeTest.java @@ -0,0 +1,76 @@ +/* + * 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.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +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.common.GridCommonAbstractTest; + +import static org.apache.ignite.IgniteSystemProperties.IGNITE_EXCHANGE_HISTORY_SIZE; + +/** + * Test exchange history size parameter effect. + */ +public class GridCachePartitionExchangeManagerHistSizeTest extends GridCommonAbstractTest { + /** */ + private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + + /** */ + private String oldHistVal; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER); + + cfg.setCacheConfiguration(new CacheConfiguration()); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + oldHistVal = System.getProperty(IGNITE_EXCHANGE_HISTORY_SIZE); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + + if (oldHistVal != null) + System.setProperty(IGNITE_EXCHANGE_HISTORY_SIZE, oldHistVal); + else + System.clearProperty(IGNITE_EXCHANGE_HISTORY_SIZE); + } + + + /** + * @throws Exception If failed. + */ + public void testSingleExchangeHistSize() throws Exception { + System.setProperty(IGNITE_EXCHANGE_HISTORY_SIZE, "1"); + + startGridsMultiThreaded(10); + } +} 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 160f85d0a1387..d4a5a8be08e22 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 @@ -18,6 +18,7 @@ package org.apache.ignite.testsuites; import junit.framework.TestSuite; +import org.apache.ignite.internal.GridCachePartitionExchangeManagerHistSizeTest; import org.apache.ignite.internal.processors.cache.CacheKeepBinaryTransactionTest; import org.apache.ignite.internal.processors.cache.CacheNearReaderUpdateTest; import org.apache.ignite.internal.processors.cache.CacheRebalancingSelfTest; @@ -75,6 +76,8 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(ExternalizableKeyValueTest.class); + suite.addTestSuite(GridCachePartitionExchangeManagerHistSizeTest.class); + return suite; } } diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java index 60bc483576db5..77c1df526150f 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java @@ -116,6 +116,17 @@ public class DmlStatementsProcessor { this.indexing = indexing; } + /** + * Handle cache stop. + * + * @param spaceName Cache name. + */ + public void onCacheStop(String spaceName) { + spaceName = F.isEmpty(spaceName) ? "default" : spaceName; + + planCache.remove(spaceName); + } + /** * Execute DML statement, possibly with few re-attempts in case of concurrent data modifications. * From 84ebd019e514c3015fd82d5658e81fcfdaf9a5d4 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Mon, 23 Oct 2017 19:39:26 +0300 Subject: [PATCH 328/357] Backported IGNITE-5116: Fixed stale DML plan caching in DmlStatementsProcessor. (cherry picked from commit b7ba1d4) --- .../ignite/internal/processors/query/h2/IgniteH2Indexing.java | 1 + .../cache/IgniteCacheAbstractInsertSqlQuerySelfTest.java | 2 +- .../processors/cache/IgniteCacheInsertSqlQuerySelfTest.java | 1 + 3 files changed, 3 insertions(+), 1 deletion(-) 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 4080c1ff06bfc..14d185091fb4d 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 @@ -2251,6 +2251,7 @@ private void createSqlFunctions(String schema, Class[] clss) throws IgniteChe if (rmv != null) { space2schema.remove(emptyIfNull(rmv.spaceName)); mapQryExec.onCacheStop(ccfg.getName()); + dmlProc.onCacheStop(rmv.spaceName); rmv.onDrop(); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractInsertSqlQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractInsertSqlQuerySelfTest.java index 4470a48e4df09..2bc8c33306cbd 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractInsertSqlQuerySelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractInsertSqlQuerySelfTest.java @@ -361,7 +361,7 @@ Object createPerson2(int id, String name, int valFld) { * @param idxTypes Indexed types. * @return Cache configuration. */ - private static CacheConfiguration cacheConfig(String name, boolean partitioned, boolean escapeSql, Class... idxTypes) { + static CacheConfiguration cacheConfig(String name, boolean partitioned, boolean escapeSql, Class... idxTypes) { return new CacheConfiguration() .setName(name) .setCacheMode(partitioned ? CacheMode.PARTITIONED : CacheMode.REPLICATED) diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInsertSqlQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInsertSqlQuerySelfTest.java index 471b791bf5dfb..2dbc3aa1603bf 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInsertSqlQuerySelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInsertSqlQuerySelfTest.java @@ -216,4 +216,5 @@ public void testUuidHandling() { assertEquals(1, (int)p.get(id)); } + } From f69532d36a01b2f6665e94be085a46d1a22b664b Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Tue, 4 Apr 2017 11:03:40 +0300 Subject: [PATCH 329/357] GG-12957: Backport of IGNITE-4625 .NET: Fix java-only node stop Solves "Ignite instance with this name has already been started" test failures (cherry picked from commit 7696dd2) --- .../platform/PlatformStopIgniteTask.java | 11 ++++++- .../Compute/AbstractTaskTest.cs | 28 ++--------------- .../Compute/MixedClusterTest.cs | 31 +++---------------- .../DeploymentTest.cs | 10 ++++++ 4 files changed, 26 insertions(+), 54 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/platform/PlatformStopIgniteTask.java b/modules/core/src/test/java/org/apache/ignite/platform/PlatformStopIgniteTask.java index 238f0584cb738..92e3bc5f1b480 100644 --- a/modules/core/src/test/java/org/apache/ignite/platform/PlatformStopIgniteTask.java +++ b/modules/core/src/test/java/org/apache/ignite/platform/PlatformStopIgniteTask.java @@ -38,7 +38,16 @@ public class PlatformStopIgniteTask extends ComputeTaskAdapter /** {@inheritDoc} */ @Nullable @Override public Map map(List subgrid, @Nullable String arg) throws IgniteException { - return Collections.singletonMap(new PlatformStopIgniteJob(arg), F.first(subgrid)); + ClusterNode node = subgrid.get(0); + + for (ClusterNode n : subgrid) { + if (n.isLocal()) { + node = n; + break; + } + } + + return Collections.singletonMap(new PlatformStopIgniteJob(arg), node); } /** {@inheritDoc} */ diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/AbstractTaskTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/AbstractTaskTest.cs index 6bcd01042e598..65bb369f002d4 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/AbstractTaskTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/AbstractTaskTest.cs @@ -129,32 +129,8 @@ public void BeforeTest() [TestFixtureTearDown] public void StopClient() { - if (Grid1 != null) - Ignition.Stop(Grid1.Name, true); - - if (_fork) - { - if (_proc2 != null) { - _proc2.Kill(); - - _proc2.Join(); - } - - if (_proc3 != null) - { - _proc3.Kill(); - - _proc3.Join(); - } - } - else - { - if (_grid2 != null) - Ignition.Stop(_grid2.Name, true); - - if (_grid3 != null) - Ignition.Stop(_grid3.Name, true); - } + Ignition.StopAll(true); + IgniteProcess.KillAll(); } /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/MixedClusterTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/MixedClusterTest.cs index ab4e6ab88201a..41817eb676561 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/MixedClusterTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/MixedClusterTest.cs @@ -21,8 +21,6 @@ namespace Apache.Ignite.Core.Tests.Compute using System.Collections; using System.Linq; using Apache.Ignite.Core.Cache; - using Apache.Ignite.Core.Cache.Query; - using Apache.Ignite.Core.Common; using Apache.Ignite.Core.Compute; using NUnit.Framework; @@ -31,8 +29,12 @@ namespace Apache.Ignite.Core.Tests.Compute /// public class MixedClusterTest { + /** */ private IIgnite _ignite; + + /** */ private string _javaNodeName; + /** */ private const string SpringConfig = @"Config\Compute\compute-grid1.xml"; @@ -88,18 +90,6 @@ public void TestJavaTask() Assert.AreEqual(2, res.Count); } - /// - /// Tests the scan query. - /// - [Test] - public void TestScanQuery() - { - var cache = GetCache(); - - // Scan query does not work in the mixed cluster. - Assert.Throws(() => cache.Query(new ScanQuery(new ScanFilter())).GetAll()); - } - /// /// Tests the cache invoke. /// @@ -153,19 +143,6 @@ public int Invoke() } } - /// - /// Test filter. - /// - [Serializable] - private class ScanFilter : ICacheEntryFilter - { - /** */ - public bool Invoke(ICacheEntry entry) - { - return entry.Key < 100; - } - } - /// /// Test processor. /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/DeploymentTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/DeploymentTest.cs index ab5a1a6aea116..20a5ffdca1a88 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/DeploymentTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/DeploymentTest.cs @@ -107,6 +107,16 @@ public void TestCustomDeployment() } } + /// + /// Fixture tear down. + /// + [TestFixtureTearDown] + public void TestFixtureTearDown() + { + Ignition.StopAll(true); + IgniteProcess.KillAll(); + } + /// /// Verifies that custom-deployed node has started. /// From ad49c5e66ec9a15c3996cc137afc44cc626e9f27 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Tue, 31 Oct 2017 18:50:46 +0300 Subject: [PATCH 330/357] Fixed javadoc. --- .../main/java/org/apache/ignite/IgniteSystemProperties.java | 3 ++- 1 file changed, 2 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 4bae00e4b739c..16ae83a1ee6d3 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -604,7 +604,8 @@ public final class IgniteSystemProperties { /** Ignite marshaller cache reread pause. */ public static final String IGNITE_MARSHALLER_CACHE_REREAD_PAUSE = "IGNITE_MARSHALLER_CACHE_REREAD_PAUSE"; - /** Class name of the closure {@link IgniteClosure} that + /** + * Class name of the closure {@link IgniteClosure} that * will be invoked once per topology and validates cache. If it returns or throws exception, * that means cache is invalid.*/ public static final String IGNITE_CACHE_VALIDATOR = "IGNITE_CACHE_VALIDATOR"; From b35ff01af99594f4fe397340f1e0561ea771264a Mon Sep 17 00:00:00 2001 From: Yakov Zhdanov Date: Tue, 11 Apr 2017 19:47:10 +0300 Subject: [PATCH 331/357] GG-12991: Backport of IGNITE-4828 Improve the distribution of keys within partitions. (cherry picked from commit 55ab10e) --- .../RendezvousAffinityFunction.java | 26 ++++++++-- ...ityFunctionFastPowerOfTwoHashSelfTest.java | 50 +++++++++++++++++++ ...sAffinityFunctionStandardHashSelfTest.java | 50 +++++++++++++++++++ .../testsuites/IgniteCacheTestSuite2.java | 4 ++ 4 files changed, 126 insertions(+), 4 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunctionFastPowerOfTwoHashSelfTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunctionStandardHashSelfTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java b/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java index 3d21dd5b154fe..54df331948165 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java @@ -101,6 +101,9 @@ public class RendezvousAffinityFunction implements AffinityFunction, Externaliza /** Number of partitions. */ private int parts; + /** Mask to use in calculation when partitions count is power of 2. */ + private transient int mask = -1; + /** Exclude neighbors flag. */ private boolean exclNeighbors; @@ -188,7 +191,9 @@ private RendezvousAffinityFunction(boolean exclNeighbors, int parts, A.ensure(parts > 0, "parts > 0"); this.exclNeighbors = exclNeighbors; - this.parts = parts; + + setPartitions(parts); + this.backupFilter = backupFilter; try { @@ -216,14 +221,20 @@ public int getPartitions() { } /** - * Sets total number of partitions. + * Sets total number of partitions.If the number of partitions is a power of two, + * the PowerOfTwo hashing method will be used. Otherwise the Standard hashing + * method will be applied. * * @param parts Total number of partitions. */ public void setPartitions(int parts) { - A.ensure(parts <= CacheConfiguration.MAX_PARTITIONS_COUNT, "parts <= " + CacheConfiguration.MAX_PARTITIONS_COUNT); + A.ensure(parts <= CacheConfiguration.MAX_PARTITIONS_COUNT, + "parts <= " + CacheConfiguration.MAX_PARTITIONS_COUNT); + A.ensure(parts > 0, "parts > 0"); this.parts = parts; + + mask = (parts & (parts - 1)) == 0 ? parts - 1 : -1; } /** @@ -490,6 +501,12 @@ else if (affinityBackupFilter == null && backupFilter == null) throw new IllegalArgumentException("Null key is passed for a partition calculation. " + "Make sure that an affinity key that is used is initialized properly."); + if (mask >= 0) { + int h; + + return ((h = key.hashCode()) ^ (h >>> 16)) & mask; + } + return U.safeAbs(key.hashCode() % parts); } @@ -536,7 +553,8 @@ else if (affinityBackupFilter == null && backupFilter == null) /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { - parts = in.readInt(); + setPartitions(in.readInt()); + exclNeighbors = in.readBoolean(); hashIdRslvr = (AffinityNodeHashResolver)in.readObject(); backupFilter = (IgniteBiPredicate)in.readObject(); diff --git a/modules/core/src/test/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunctionFastPowerOfTwoHashSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunctionFastPowerOfTwoHashSelfTest.java new file mode 100644 index 0000000000000..683ffa2542b86 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunctionFastPowerOfTwoHashSelfTest.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.cache.affinity.rendezvous; + +import org.apache.ignite.Ignite; +import org.apache.ignite.cache.affinity.AbstractAffinityFunctionSelfTest; +import org.apache.ignite.cache.affinity.AffinityFunction; +import org.apache.ignite.testframework.GridTestUtils; + +/** + * Tests for {@link RendezvousAffinityFunction}. + */ +public class RendezvousAffinityFunctionFastPowerOfTwoHashSelfTest extends AbstractAffinityFunctionSelfTest { + /** Ignite. */ + private static Ignite ignite; + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + ignite = startGrid(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + } + + /** {@inheritDoc} */ + @Override protected AffinityFunction affinityFunction() { + AffinityFunction aff = new RendezvousAffinityFunction(512, null); + + GridTestUtils.setFieldValue(aff, "ignite", ignite); + + return aff; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunctionStandardHashSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunctionStandardHashSelfTest.java new file mode 100644 index 0000000000000..ed47c57eb545d --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunctionStandardHashSelfTest.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.cache.affinity.rendezvous; + +import org.apache.ignite.Ignite; +import org.apache.ignite.cache.affinity.AbstractAffinityFunctionSelfTest; +import org.apache.ignite.cache.affinity.AffinityFunction; +import org.apache.ignite.testframework.GridTestUtils; + +/** + * Tests for {@link RendezvousAffinityFunction}. + */ +public class RendezvousAffinityFunctionStandardHashSelfTest extends AbstractAffinityFunctionSelfTest { + /** Ignite. */ + private static Ignite ignite; + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + ignite = startGrid(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + } + + /** {@inheritDoc} */ + @Override protected AffinityFunction affinityFunction() { + AffinityFunction aff = new RendezvousAffinityFunction(513, null); + + GridTestUtils.setFieldValue(aff, "ignite", ignite); + + return aff; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java index 6d0745dea8a74..5304cd4bd92bc 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java @@ -22,6 +22,8 @@ import org.apache.ignite.cache.affinity.fair.FairAffinityFunctionExcludeNeighborsSelfTest; import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunctionBackupFilterSelfTest; import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunctionExcludeNeighborsSelfTest; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunctionFastPowerOfTwoHashSelfTest; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunctionStandardHashSelfTest; import org.apache.ignite.internal.processors.cache.CacheConcurrentReadThroughTest; import org.apache.ignite.internal.processors.cache.CacheConfigurationLeakTest; import org.apache.ignite.internal.processors.cache.CacheDhtLocalPartitionAfterRemoveSelfTest; @@ -182,6 +184,8 @@ public static TestSuite suite() throws Exception { suite.addTest(new TestSuite(GridCacheAtomicNearReadersSelfTest.class)); suite.addTest(new TestSuite(GridCachePartitionedAffinitySelfTest.class)); suite.addTest(new TestSuite(RendezvousAffinityFunctionExcludeNeighborsSelfTest.class)); + suite.addTest(new TestSuite(RendezvousAffinityFunctionFastPowerOfTwoHashSelfTest.class)); + suite.addTest(new TestSuite(RendezvousAffinityFunctionStandardHashSelfTest.class)); suite.addTest(new TestSuite(FairAffinityFunctionExcludeNeighborsSelfTest.class)); suite.addTest(new TestSuite(GridCacheRendezvousAffinityClientSelfTest.class)); suite.addTest(new TestSuite(GridCachePartitionedProjectionAffinitySelfTest.class)); From f1236288e229e373c9b18191c75c6f1ac7346aab Mon Sep 17 00:00:00 2001 From: Alexey Popov Date: Tue, 31 Oct 2017 14:33:58 +0300 Subject: [PATCH 332/357] Backported IGNITE-6690 DiscoverySpi: Clientmode Ignite should not fail on handshake errors --- .../ignite/spi/discovery/tcp/ClientImpl.java | 14 +- .../ignite/spi/discovery/tcp/ServerImpl.java | 16 +- .../spi/discovery/tcp/TcpDiscoverySpi.java | 22 ++ .../tcp/TcpDiscoveryWithWrongServerTest.java | 332 ++++++++++++++++++ .../IgniteSpiDiscoverySelfTestSuite.java | 5 +- 5 files changed, 383 insertions(+), 6 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryWithWrongServerTest.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 3657b514d2780..c7c2ffa73a1e4 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 @@ -702,11 +702,17 @@ else if (addrs.isEmpty()) { } if (X.hasCause(e, StreamCorruptedException.class)) { - if (--sslConnectAttempts == 0) - throw new IgniteSpiException("Unable to establish plain connection. " + - "Was remote cluster configured with SSL? [rmtAddr=" + addr + ", errMsg=\"" + e.getMessage() + "\"]", e); + // StreamCorruptedException could be caused by remote node failover + if (connectAttempts < 2) { + connectAttempts++; - continue; + continue; + } + + if (log.isDebugEnabled()) + log.debug("Connect failed with StreamCorruptedException, skip address: " + addr); + + break; } if (timeoutHelper.checkFailureTimeoutReached(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 0f85e0612b942..a89eb08b2e2f8 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 @@ -1250,7 +1250,7 @@ else if (U.currentTimeMillis() - noResStart > spi.joinTimeout) errs.add(e); - if (X.hasCause(e, SSLException.class, StreamCorruptedException.class)) { + if (X.hasCause(e, SSLException.class)) { if (--sslConnectAttempts == 0) throw new IgniteException("Unable to establish secure connection. " + "Was remote cluster configured with SSL? [rmtAddr=" + addr + ", errMsg=\"" + e.getMessage() + "\"]", e); @@ -1258,6 +1258,20 @@ else if (U.currentTimeMillis() - noResStart > spi.joinTimeout) continue; } + if (X.hasCause(e, StreamCorruptedException.class)) { + // StreamCorruptedException could be caused by remote node failover + if (connectAttempts < 2) { + connectAttempts++; + + continue; + } + + if (log.isDebugEnabled()) + log.debug("Connect failed with StreamCorruptedException, skip address: " + addr); + + break; + } + if (timeoutHelper.checkFailureTimeoutReached(e)) break; 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 ac423363184ea..5b94a8d403c5e 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 @@ -23,6 +23,7 @@ import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; +import java.io.StreamCorruptedException; import java.net.InetAddress; import java.net.InetSocketAddress; import java.net.Socket; @@ -40,7 +41,9 @@ import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.regex.Pattern; import javax.net.ssl.SSLContext; +import javax.net.ssl.SSLException; import javax.net.ssl.SSLServerSocketFactory; import javax.net.ssl.SSLSocketFactory; import org.apache.ignite.Ignite; @@ -281,6 +284,9 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T /** Maximum ack timeout value for receiving message acknowledgement in milliseconds (value is 600,000ms). */ public static final long DFLT_MAX_ACK_TIMEOUT = 10 * 60 * 1000; + /** Ssl message pattern for StreamCorruptedException. */ + private static Pattern sslMsgPattern = Pattern.compile("invalid stream header: 150\\d0\\d00"); + /** Local address. */ protected String locAddr; @@ -1507,6 +1513,22 @@ protected T readMessage(Socket sock, @Nullable InputStream in, long timeout) "long GC pauses on remote node) [curTimeout=" + timeout + ", rmtAddr=" + sock.getRemoteSocketAddress() + ", rmtPort=" + sock.getPort() + ']'); + StreamCorruptedException streamCorruptedCause = X.cause(e, StreamCorruptedException.class); + + if (streamCorruptedCause != null) { + // Lets check StreamCorruptedException for SSL Alert message + // Sample SSL Alert message: 15:03:03:00:02:02:0a + // 15 = Alert + // 03:03 = SSL version + // 00:02 = payload length + // 02:0a = critical (02) / unexpected message (0a) + // So, check message for "invalid stream header: 150X0X00" + + String msg = streamCorruptedCause.getMessage(); + + if (msg != null && sslMsgPattern.matcher(msg).matches()) + streamCorruptedCause.initCause(new SSLException("Detected SSL alert in StreamCorruptedException")); + } throw e; } finally { diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryWithWrongServerTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryWithWrongServerTest.java new file mode 100644 index 0000000000000..768f5f7f3d281 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryWithWrongServerTest.java @@ -0,0 +1,332 @@ +/* + * 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.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.net.ServerSocket; +import java.net.Socket; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import org.apache.ignite.Ignite; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.spi.IgniteSpiException; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.GridTestThread; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Client-based discovery SPI test with non-Ignite servers. + */ +public class TcpDiscoveryWithWrongServerTest extends GridCommonAbstractTest { + /** Non-Ignite Server port. */ + private final static int SERVER_PORT = 47500; + + /** Non-Ignite Server socket. */ + private ServerSocket srvSock; + + /** Count of accepted connections to non-Ignite Server. */ + private int connCnt; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + TcpDiscoveryVmIpFinder ipFinder = new TcpDiscoveryVmIpFinder(); + + ipFinder.setAddresses(Collections.singleton("127.0.0.1:" + Integer.toString(SERVER_PORT) + ".." + + Integer.toString(SERVER_PORT + 2))); + + cfg.setDiscoverySpi(new TcpDiscoverySpiWithOrderedIps().setIpFinder(ipFinder)); + + if (igniteInstanceName.startsWith("client")) + cfg.setClientMode(true); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopTcpThread(); + + stopAllGrids(); + + super.afterTest(); + } + + /** + * Starts tcp test thread + * @param workerFactory one of WorkerFactory + */ + private void startTcpThread(final WorkerFactory workerFactory) { + connCnt = 0; + + try { + srvSock = new ServerSocket(SERVER_PORT, 10, InetAddress.getByName("127.0.0.1")); + } + catch (Exception e) { + fail("Unexpected TcpServer exception " + e.getMessage()); + } + + new GridTestThread(new Runnable() { + @Override public void run() { + try { + while(!Thread.currentThread().isInterrupted()) { + Socket clientSock = srvSock.accept(); + + connCnt++; + + // Create a new thread for socket connection. + new GridTestThread(workerFactory.newWorker(clientSock)).start(); + } + } + catch (Exception e) { + if (!srvSock.isClosed()) + e.printStackTrace(); + } + } + }).start(); + } + + /** + * Stops tcp test thread + * @throws IOException IOException + */ + private void stopTcpThread() throws IOException { + if (srvSock != null && !srvSock.isClosed()) + srvSock.close(); + } + + /** + * Test that Client successfully ignores wrong responses during Discovery Handshake Procedure. + * + * @throws Exception in case of error. + */ + public void testWrongHandshakeResponse() throws Exception { + startTcpThread(new SomeResponseWorker()); + + simpleTest(); + } + + /** + * Test that Client successfully ignores wrong responses during Discovery Handshake Procedure. + * + * @throws Exception in case of error. + */ + public void testNoHandshakeResponse() throws Exception { + startTcpThread(new NoResponseWorker()); + + simpleTest(); + } + + /** + * Test that Client successfully ignores when server closes sockets after Discovery Handshake Request. + * + * @throws Exception in case of error. + */ + public void testDisconnectOnRequest() throws Exception { + startTcpThread(new DisconnectOnRequestWorker()); + + simpleTest(); + } + + /** + * Test that Client successfully ignores when server closes sockets immediately. + * + * @throws Exception in case of error. + */ + public void testEarlyDisconnect() throws Exception { + startTcpThread(new EarlyDisconnectWorker()); + + simpleTest(); + } + + /** + * Some simple sanity check with the Server and Client + * It is expected that both client and server could successfully perform Discovery Procedure when there is + * unknown (test) server in the ipFinder list. + */ + private void simpleTest() { + try { + Ignite srv = startGrid("server"); + Ignite client = startGrid("client"); + + awaitPartitionMapExchange(); + + assertEquals(2, srv.cluster().nodes().size()); + assertEquals(2, client.cluster().nodes().size()); + assertTrue(connCnt >= 2); + + srv.getOrCreateCache(DEFAULT_CACHE_NAME).put(1, 1); + + assertEquals(1, client.getOrCreateCache(DEFAULT_CACHE_NAME).get(1)); + } + catch (Exception e) { + fail("Failed with unexpected exception: " + e.getMessage()); + } + } + + /** + * Just a factory for runnable workers + */ + private interface WorkerFactory { + /** + * Creates a new worker for socket + * @param clientSock socket for worker + * @return runnable Worker + */ + Runnable newWorker(Socket clientSock); + } + + /** + * SocketWorker + */ + private abstract class SocketWorker implements Runnable { + /** Client socket. */ + Socket clientSock; + + /** + * @param clientSock Client socket. + */ + SocketWorker(Socket clientSock) { + this.clientSock = clientSock; + } + + /** {@inheritDoc} */ + @Override public void run() { + try { + InputStream input = clientSock.getInputStream(); + OutputStream output = clientSock.getOutputStream(); + byte[] buf = new byte[1024]; + + while (!clientSock.isClosed() && input.read(buf) > 0) + action(input, output); + + if (!clientSock.isClosed()) + clientSock.close(); + } + catch (IOException e) { + log.error("Unexpected error", e); + } + } + + /** + * @param input socket input stream + * @param output socket output stream + * @throws IOException IOException + */ + public abstract void action(InputStream input, OutputStream output) throws IOException; + } + + /** + * SomeResponseWorker. + */ + private class SomeResponseWorker implements WorkerFactory { + /** {@inheritDoc} */ + @Override public Runnable newWorker(Socket clientSock) { + return new SocketWorker(clientSock) { + @Override public void action(InputStream input, OutputStream output) throws IOException { + output.write("Some response".getBytes()); + + log.error("TEST: Some response was sent to " + clientSock.getRemoteSocketAddress()); + } + }; + } + } + + /** + * NoResponseWorker. + */ + private class NoResponseWorker implements WorkerFactory { + /** {@inheritDoc} */ + @Override public Runnable newWorker(Socket clientSock) { + return new SocketWorker(clientSock) { + @Override public void action(InputStream input, OutputStream output) throws IOException { + log.error("TEST: No response was sent to " + clientSock.getRemoteSocketAddress()); + } + }; + } + } + + /** + * DisconnectOnRequestWorker. + */ + private class DisconnectOnRequestWorker implements WorkerFactory { + /** {@inheritDoc} */ + @Override public Runnable newWorker(Socket clientSock) { + return new SocketWorker(clientSock) { + @Override public void action(InputStream input, OutputStream output) throws IOException { + clientSock.close(); + + log.error("TEST: Socket closed for " + clientSock.getRemoteSocketAddress()); + } + }; + } + } + + /** + * EarlyDisconnectWorker. + */ + private class EarlyDisconnectWorker implements WorkerFactory { + /** {@inheritDoc} */ + @Override public Runnable newWorker(Socket clientSock) { + return new SocketWorker(clientSock) { + @Override public void action(InputStream input, OutputStream output) throws IOException { + // No-op + } + + @Override public void run() { + try { + clientSock.close(); + + log.error("TEST: Socket closed for " + clientSock.getRemoteSocketAddress()); + } + catch (IOException e) { + log.error("Unexpected error", e); + } + } + }; + } + } + + /** + * TcpDiscoverySpi with non-shuffled resolved IP addresses. We should ensure that in this test non-Ignite server + * is the first element of the addresses list + */ + class TcpDiscoverySpiWithOrderedIps extends TcpDiscoverySpi { + /** {@inheritDoc} */ + @Override protected Collection resolvedAddresses() throws IgniteSpiException { + Collection shuffled = super.resolvedAddresses(); + List res = new ArrayList<>(shuffled); + + Collections.sort(res, new Comparator() { + @Override public int compare(InetSocketAddress o1, InetSocketAddress o2) { + return o1.toString().compareTo(o2.toString()); + } + }); + + return res; + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java index aa9ac93078947..43e9153cf3084 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java @@ -41,6 +41,7 @@ import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpiStartStopSelfTest; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySslSecuredUnsecuredTest; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySslSelfTest; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoveryWithWrongServerTest; import org.apache.ignite.spi.discovery.tcp.ipfinder.jdbc.TcpDiscoveryJdbcIpFinderSelfTest; import org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinderSelfTest; import org.apache.ignite.spi.discovery.tcp.ipfinder.sharedfs.TcpDiscoverySharedFsIpFinderSelfTest; @@ -93,7 +94,9 @@ public static TestSuite suite() throws Exception { suite.addTest(new TestSuite(TcpDiscoveryNodeAttributesUpdateOnReconnectTest.class)); suite.addTest(new TestSuite(AuthenticationRestartTest.class)); - //Client connect + suite.addTest(new TestSuite(TcpDiscoveryWithWrongServerTest.class)); + + // Client connect. suite.addTest(new TestSuite(IgniteClientConnectTest.class)); suite.addTest(new TestSuite(IgniteClientReconnectMassiveShutdownTest.class)); From 4d51f4716b5c206d1bc70511f8c3ec480a29008d Mon Sep 17 00:00:00 2001 From: Alexey Popov Date: Thu, 2 Nov 2017 13:12:03 +0300 Subject: [PATCH 333/357] Fixed TcpDiscoveryWithWrongServerTest. Signed-off-by: nikolay_tikhonov --- .../tcp/TcpDiscoveryWithWrongServerTest.java | 52 ++++++++++--------- 1 file changed, 28 insertions(+), 24 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryWithWrongServerTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryWithWrongServerTest.java index 768f5f7f3d281..ffd0d030c643e 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryWithWrongServerTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryWithWrongServerTest.java @@ -29,6 +29,7 @@ import java.util.Collections; import java.util.Comparator; import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.ignite.Ignite; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.spi.IgniteSpiException; @@ -40,14 +41,17 @@ * Client-based discovery SPI test with non-Ignite servers. */ public class TcpDiscoveryWithWrongServerTest extends GridCommonAbstractTest { - /** Non-Ignite Server port. */ + /** Non-Ignite Server port #1. */ private final static int SERVER_PORT = 47500; - /** Non-Ignite Server socket. */ - private ServerSocket srvSock; + /** Non-Ignite Server port #2. */ + private final static int LAST_SERVER_PORT = SERVER_PORT + 5; + + /** Non-Ignite Server sockets. */ + private List srvSocks = new ArrayList<>(); /** Count of accepted connections to non-Ignite Server. */ - private int connCnt; + private AtomicInteger connCnt = new AtomicInteger(0); /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { @@ -56,7 +60,7 @@ public class TcpDiscoveryWithWrongServerTest extends GridCommonAbstractTest { TcpDiscoveryVmIpFinder ipFinder = new TcpDiscoveryVmIpFinder(); ipFinder.setAddresses(Collections.singleton("127.0.0.1:" + Integer.toString(SERVER_PORT) + ".." + - Integer.toString(SERVER_PORT + 2))); + Integer.toString(LAST_SERVER_PORT))); cfg.setDiscoverySpi(new TcpDiscoverySpiWithOrderedIps().setIpFinder(ipFinder)); @@ -68,7 +72,7 @@ public class TcpDiscoveryWithWrongServerTest extends GridCommonAbstractTest { /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { - stopTcpThread(); + stopTcpThreads(); stopAllGrids(); @@ -79,15 +83,10 @@ public class TcpDiscoveryWithWrongServerTest extends GridCommonAbstractTest { * Starts tcp test thread * @param workerFactory one of WorkerFactory */ - private void startTcpThread(final WorkerFactory workerFactory) { - connCnt = 0; + private void startTcpThread(final WorkerFactory workerFactory, final int port) throws Exception { + final ServerSocket srvSock = new ServerSocket(port, 10, InetAddress.getByName("127.0.0.1")); - try { - srvSock = new ServerSocket(SERVER_PORT, 10, InetAddress.getByName("127.0.0.1")); - } - catch (Exception e) { - fail("Unexpected TcpServer exception " + e.getMessage()); - } + srvSocks.add(srvSock); new GridTestThread(new Runnable() { @Override public void run() { @@ -95,7 +94,7 @@ private void startTcpThread(final WorkerFactory workerFactory) { while(!Thread.currentThread().isInterrupted()) { Socket clientSock = srvSock.accept(); - connCnt++; + connCnt.getAndIncrement(); // Create a new thread for socket connection. new GridTestThread(workerFactory.newWorker(clientSock)).start(); @@ -103,7 +102,7 @@ private void startTcpThread(final WorkerFactory workerFactory) { } catch (Exception e) { if (!srvSock.isClosed()) - e.printStackTrace(); + log.error("Unexpected error", e); } } }).start(); @@ -113,9 +112,10 @@ private void startTcpThread(final WorkerFactory workerFactory) { * Stops tcp test thread * @throws IOException IOException */ - private void stopTcpThread() throws IOException { - if (srvSock != null && !srvSock.isClosed()) - srvSock.close(); + private void stopTcpThreads() throws IOException { + for (ServerSocket srvSock: srvSocks) + if (!srvSock.isClosed()) + srvSock.close(); } /** @@ -124,7 +124,8 @@ private void stopTcpThread() throws IOException { * @throws Exception in case of error. */ public void testWrongHandshakeResponse() throws Exception { - startTcpThread(new SomeResponseWorker()); + startTcpThread(new SomeResponseWorker(), SERVER_PORT); + startTcpThread(new SomeResponseWorker(), LAST_SERVER_PORT); simpleTest(); } @@ -135,7 +136,8 @@ public void testWrongHandshakeResponse() throws Exception { * @throws Exception in case of error. */ public void testNoHandshakeResponse() throws Exception { - startTcpThread(new NoResponseWorker()); + startTcpThread(new NoResponseWorker(), SERVER_PORT); + startTcpThread(new NoResponseWorker(), LAST_SERVER_PORT); simpleTest(); } @@ -146,7 +148,8 @@ public void testNoHandshakeResponse() throws Exception { * @throws Exception in case of error. */ public void testDisconnectOnRequest() throws Exception { - startTcpThread(new DisconnectOnRequestWorker()); + startTcpThread(new DisconnectOnRequestWorker(), SERVER_PORT); + startTcpThread(new DisconnectOnRequestWorker(), LAST_SERVER_PORT); simpleTest(); } @@ -157,7 +160,8 @@ public void testDisconnectOnRequest() throws Exception { * @throws Exception in case of error. */ public void testEarlyDisconnect() throws Exception { - startTcpThread(new EarlyDisconnectWorker()); + startTcpThread(new EarlyDisconnectWorker(), SERVER_PORT); + startTcpThread(new EarlyDisconnectWorker(), LAST_SERVER_PORT); simpleTest(); } @@ -176,7 +180,7 @@ private void simpleTest() { assertEquals(2, srv.cluster().nodes().size()); assertEquals(2, client.cluster().nodes().size()); - assertTrue(connCnt >= 2); + assertTrue(connCnt.get() >= 2); srv.getOrCreateCache(DEFAULT_CACHE_NAME).put(1, 1); From b083133a4710441ad25a434234fa0419dc084a23 Mon Sep 17 00:00:00 2001 From: apopov Date: Fri, 3 Nov 2017 09:43:48 +0300 Subject: [PATCH 334/357] Backported IGNITE-6690 Fixed build error --- .../spi/discovery/tcp/TcpDiscoveryWithWrongServerTest.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryWithWrongServerTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryWithWrongServerTest.java index ffd0d030c643e..77db8711993d3 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryWithWrongServerTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryWithWrongServerTest.java @@ -41,6 +41,9 @@ * Client-based discovery SPI test with non-Ignite servers. */ public class TcpDiscoveryWithWrongServerTest extends GridCommonAbstractTest { + /** */ + public static final String DEFAULT_CACHE_NAME = "myCache"; + /** Non-Ignite Server port #1. */ private final static int SERVER_PORT = 47500; From 0677aaf8763af75325306390c0e81f8406f72b89 Mon Sep 17 00:00:00 2001 From: nikolay_tikhonov Date: Wed, 8 Nov 2017 17:17:56 +0300 Subject: [PATCH 335/357] GG-12992 Moved queries execution to sys-pool. Signed-off-by: nikolay_tikhonov --- .../cache/query/GridCacheDistributedQueryManager.java | 2 +- .../processors/cache/query/GridCacheQueryRequest.java | 10 +++++++++- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java index 8457180e7b3a7..b97000693b6f8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java @@ -283,7 +283,7 @@ protected void removeQueryFuture(long reqId) { false, null, req.keyValueFilter(), - req.partition() == -1 ? null : req.partition(), + req.partitionId() == -1 ? null : req.partitionId(), req.className(), req.clause(), req.includeMetaData(), diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java index 3baafec54ebf0..4a7e14f9f8789 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java @@ -477,10 +477,18 @@ public int taskHash() { return taskHash; } + /** {@inheritDoc} */ + @Override public int partition() { + return Integer.MIN_VALUE; + } + /** + * This method needed for scan query. {@link #GridCacheQueryRequest#partition()} method related with IO policy and cannot be used + * for this role. Otherwise Query will be processed in striped pool. + * * @return partition. */ - public int partition() { + public int partitionId() { return part; } From eb73b9df193d1724f45739759d5f4ac545401d1d Mon Sep 17 00:00:00 2001 From: apopov Date: Fri, 27 Oct 2017 14:53:31 +0300 Subject: [PATCH 336/357] Backport IGNITE-4477 into 1.8.x --- .../cache/IgniteCacheFutureImpl.java | 8 +- .../util/future/AsyncFutureListener.java | 57 +++ .../util/future/IgniteFutureImpl.java | 33 +- .../org/apache/ignite/lang/IgniteFuture.java | 24 +- .../future/IgniteCacheFutureImplTest.java | 46 ++ .../util/future/IgniteFutureImplTest.java | 474 ++++++++++++++++-- .../testsuites/IgniteLangSelfTestSuite.java | 3 + .../schedule/ScheduleFutureImpl.java | 49 +- .../schedule/GridScheduleSelfTest.java | 84 +++- 9 files changed, 723 insertions(+), 55 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/util/future/AsyncFutureListener.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/util/future/IgniteCacheFutureImplTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheFutureImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheFutureImpl.java index 74cccc14cdaa8..c861be8894675 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheFutureImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheFutureImpl.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.processors.cache; +import java.util.concurrent.Executor; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.IgniteFutureCancelledCheckedException; import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException; @@ -43,7 +44,12 @@ public IgniteCacheFutureImpl(IgniteInternalFuture fut) { /** {@inheritDoc} */ @Override public IgniteFuture chain(IgniteClosure, T> doneCb) { - return new IgniteCacheFutureImpl<>(chainInternal(doneCb)); + return new IgniteCacheFutureImpl<>(chainInternal(doneCb, null)); + } + + /** {@inheritDoc} */ + @Override public IgniteFuture chainAsync(IgniteClosure, T> doneCb, Executor exec) { + return new IgniteCacheFutureImpl<>(chainInternal(doneCb, exec)); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/future/AsyncFutureListener.java b/modules/core/src/main/java/org/apache/ignite/internal/util/future/AsyncFutureListener.java new file mode 100644 index 0000000000000..460ce8bca1076 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/future/AsyncFutureListener.java @@ -0,0 +1,57 @@ +/* + * 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.util.future; + +import java.util.concurrent.Executor; +import org.apache.ignite.lang.IgniteFuture; +import org.apache.ignite.lang.IgniteInClosure; + +/** + * Wraps listener and executes it in specified executor. + */ +public class AsyncFutureListener implements IgniteInClosure> { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + + /** */ + private final IgniteInClosure> lsnr; + + /** */ + private final Executor exec; + + /** + * @param lsnr Listener to be called asynchronously. + * @param exec Executor to process listener. + */ + public AsyncFutureListener(IgniteInClosure> lsnr, Executor exec) { + assert lsnr != null; + assert exec != null; + + this.lsnr = lsnr; + this.exec = exec; + } + + /** {@inheritDoc} */ + @Override public void apply(final IgniteFuture fut) { + exec.execute(new Runnable() { + @Override public void run() { + lsnr.apply(fut); + } + }); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/future/IgniteFutureImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/util/future/IgniteFutureImpl.java index 7d741548f5ef7..5427ea7d06895 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/future/IgniteFutureImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/future/IgniteFutureImpl.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.util.future; +import java.util.concurrent.Executor; import java.util.concurrent.TimeUnit; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; @@ -28,6 +29,7 @@ import org.apache.ignite.lang.IgniteClosure; import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.lang.IgniteInClosure; +import org.jetbrains.annotations.Nullable; /** * Implementation of public API future. @@ -79,17 +81,35 @@ public IgniteInternalFuture internalFuture() { fut.listen(new InternalFutureListener(lsnr)); } + /** {@inheritDoc} */ + @Override public void listenAsync(IgniteInClosure> lsnr, Executor exec) { + A.notNull(lsnr, "lsnr"); + A.notNull(exec, "exec"); + + fut.listen(new InternalFutureListener(new AsyncFutureListener<>(lsnr, exec))); + } + /** {@inheritDoc} */ @Override public IgniteFuture chain(final IgniteClosure, T> doneCb) { - return new IgniteFutureImpl<>(chainInternal(doneCb)); + return new IgniteFutureImpl<>(chainInternal(doneCb, null)); + } + + /** {@inheritDoc} */ + @Override public IgniteFuture chainAsync(final IgniteClosure, T> doneCb, + Executor exec) { + A.notNull(doneCb, "doneCb"); + A.notNull(exec, "exec"); + + return new IgniteFutureImpl<>(chainInternal(doneCb, exec)); } /** * @param doneCb Done callback. * @return Internal future */ - protected IgniteInternalFuture chainInternal(final IgniteClosure, T> doneCb) { - return fut.chain(new C1, T>() { + protected IgniteInternalFuture chainInternal(final IgniteClosure, T> doneCb, + @Nullable Executor exec) { + C1, T> clos = new C1, T>() { @Override public T apply(IgniteInternalFuture fut) { assert IgniteFutureImpl.this.fut == fut; @@ -100,7 +120,12 @@ protected IgniteInternalFuture chainInternal(final IgniteClosure { /** * Registers listener closure to be asynchronously notified whenever future completes. + * Closure will be processed in thread that completes this future or (if future already + * completed) immediately in current thread. * - * @param lsnr Listener closure to register. If not provided - this method is no-op. + * @param lsnr Listener closure to register. Cannot be {@code null}. */ public void listen(IgniteInClosure> lsnr); + /** + * Registers listener closure to be asynchronously notified whenever future completes. + * Closure will be processed in specified executor. + * + * @param lsnr Listener closure to register. Cannot be {@code null}. + * @param exec Executor to run listener. Cannot be {@code null}. + */ + public void listenAsync(IgniteInClosure> lsnr, Executor exec); + /** * Make a chained future to convert result of this future (when complete) into a new format. * It is guaranteed that done callback will be called only ONCE. @@ -124,4 +136,14 @@ public interface IgniteFuture { * @return Chained future that finishes after this future completes and done callback is called. */ public IgniteFuture chain(IgniteClosure, T> doneCb); + + /** + * Make a chained future to convert result of this future (when complete) into a new format. + * It is guaranteed that done callback will be called only ONCE. + * + * @param doneCb Done callback that is applied to this future when it finishes to produce chained future result. + * @param exec Executor to run done callback. Cannot be {@code null}. + * @return Chained future that finishes after this future completes and done callback is called. + */ + public IgniteFuture chainAsync(IgniteClosure, T> doneCb, Executor exec); } \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/future/IgniteCacheFutureImplTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/future/IgniteCacheFutureImplTest.java new file mode 100644 index 0000000000000..46f1706f21d39 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/util/future/IgniteCacheFutureImplTest.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.util.future; + +import javax.cache.CacheException; +import org.apache.ignite.IgniteException; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.processors.cache.IgniteCacheFutureImpl; + +/** + * Tests IgniteCacheFutureImpl. + */ +public class IgniteCacheFutureImplTest extends IgniteFutureImplTest { + /** {@inheritDoc} */ + @Override protected IgniteFutureImpl createFuture(IgniteInternalFuture fut) { + return new IgniteCacheFutureImpl<>(fut); + } + + /** {@inheritDoc} */ + @Override protected Class expectedException() { + return CacheException.class; + } + + /** {@inheritDoc} */ + @Override protected void assertExpectedException(Exception e, Exception exp) { + if (exp instanceof IgniteException) + assertEquals(exp, e.getCause().getCause()); + else + assertEquals(exp, e.getCause()); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/future/IgniteFutureImplTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/future/IgniteFutureImplTest.java index c1cc51ec60fd9..f97f1bcbfd4e9 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/util/future/IgniteFutureImplTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/util/future/IgniteFutureImplTest.java @@ -18,10 +18,17 @@ package org.apache.ignite.internal.util.future; import java.util.concurrent.Callable; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import javax.cache.CacheException; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; +import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.util.typedef.C1; import org.apache.ignite.internal.util.typedef.CI1; import org.apache.ignite.internal.util.typedef.internal.U; @@ -29,18 +36,47 @@ import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.jetbrains.annotations.NotNull; /** * */ public class IgniteFutureImplTest extends GridCommonAbstractTest { + /** Context thread name. */ + private static final String CTX_THREAD_NAME = "test-async"; + + /** Custom thread name. */ + private static final String CUSTOM_THREAD_NAME = "test-custom-async"; + + /** Test executor. */ + private ExecutorService ctxExec; + + /** Custom executor. */ + private ExecutorService customExec; + + /** {@inheritDoc} */ + @SuppressWarnings("ExternalizableWithoutPublicNoArgConstructor") + @Override protected void beforeTest() throws Exception { + ctxExec = createExecutor(CTX_THREAD_NAME); + customExec = createExecutor(CUSTOM_THREAD_NAME); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + U.shutdownNow(getClass(), ctxExec, log); + U.shutdownNow(getClass(), customExec, log); + + ctxExec = null; + customExec = null; + } + /** * @throws Exception If failed. */ public void testFutureGet() throws Exception { GridFutureAdapter fut0 = new GridFutureAdapter<>(); - IgniteFutureImpl fut = new IgniteFutureImpl<>(fut0); + IgniteFutureImpl fut = createFuture(fut0); assertFalse(fut.isDone()); @@ -73,7 +109,7 @@ public void testFutureGet() throws Exception { public void testFutureException() throws Exception { GridFutureAdapter fut0 = new GridFutureAdapter<>(); - final IgniteFutureImpl fut = new IgniteFutureImpl<>(fut0); + final IgniteFutureImpl fut = createFuture(fut0); assertFalse(fut.isDone()); @@ -87,35 +123,27 @@ public void testFutureException() throws Exception { fut0.onDone(err0); - IgniteException err = (IgniteException)GridTestUtils.assertThrows(log, new Callable() { + Exception err = (Exception)GridTestUtils.assertThrows(log, new Callable() { @Override public Void call() throws Exception { fut.get(); return null; } - }, IgniteException.class, "test error"); + }, expectedException(), "test error"); - assertEquals(err0, err.getCause()); + assertExpectedException(err, err0); assertTrue(fut.isDone()); - assertTrue(fut.duration() > 0); - - long dur0 = fut.duration(); - - U.sleep(100); - - assertEquals(dur0, fut.duration()); - - err = (IgniteException)GridTestUtils.assertThrows(log, new Callable() { + err = (Exception)GridTestUtils.assertThrows(log, new Callable() { @Override public Void call() throws Exception { fut.get(); return null; } - }, IgniteException.class, null); + }, expectedException(), null); - assertEquals(err0, err.getCause()); + assertExpectedException(err, err0); } /** @@ -124,21 +152,21 @@ public void testFutureException() throws Exception { public void testFutureIgniteException() throws Exception { GridFutureAdapter fut0 = new GridFutureAdapter<>(); - final IgniteFutureImpl fut = new IgniteFutureImpl<>(fut0); + final IgniteFutureImpl fut = createFuture(fut0); IgniteException err0 = new IgniteException("test error"); fut0.onDone(err0); - IgniteException err = (IgniteException)GridTestUtils.assertThrows(log, new Callable() { + Exception err = (Exception)GridTestUtils.assertThrows(log, new Callable() { @Override public Void call() throws Exception { fut.get(); return null; } - }, IgniteException.class, "test error"); + }, expectedException(), "test error"); - assertEquals(err0, err); + assertExpectedException(err, err0); } /** @@ -147,7 +175,7 @@ public void testFutureIgniteException() throws Exception { public void testListeners() throws Exception { GridFutureAdapter fut0 = new GridFutureAdapter<>(); - IgniteFutureImpl fut = new IgniteFutureImpl<>(fut0); + IgniteFutureImpl fut = createFuture(fut0); final AtomicInteger lsnr1Cnt = new AtomicInteger(); @@ -192,7 +220,7 @@ public void testListenersOnError() throws Exception { { GridFutureAdapter fut0 = new GridFutureAdapter<>(); - IgniteFutureImpl fut = new IgniteFutureImpl<>(fut0); + IgniteFutureImpl fut = createFuture(fut0); final IgniteException err0 = new IgniteException("test error"); @@ -205,8 +233,8 @@ public void testListenersOnError() throws Exception { fail(); } - catch (IgniteException err) { - assertEquals(err0, err); + catch (IgniteException | CacheException err) { + assertExpectedException(err, err0); passed.set(true); } @@ -223,7 +251,7 @@ public void testListenersOnError() throws Exception { { GridFutureAdapter fut0 = new GridFutureAdapter<>(); - IgniteFutureImpl fut = new IgniteFutureImpl<>(fut0); + IgniteFutureImpl fut = createFuture(fut0); final IgniteCheckedException err0 = new IgniteCheckedException("test error"); @@ -236,8 +264,8 @@ public void testListenersOnError() throws Exception { fail(); } - catch (IgniteException err) { - assertEquals(err0, err.getCause()); + catch (IgniteException | CacheException err) { + assertExpectedException(err, err0); passed.set(true); } @@ -252,13 +280,133 @@ public void testListenersOnError() throws Exception { } } + /** + * @throws Exception If failed. + */ + public void testAsyncListeners() throws Exception { + GridFutureAdapter fut0 = new GridFutureAdapter<>(); + + IgniteFutureImpl fut = createFuture(fut0); + + final CountDownLatch latch1 = new CountDownLatch(1); + + IgniteInClosure> lsnr1 = createAsyncListener(latch1, CUSTOM_THREAD_NAME, null); + + assertFalse(fut.isDone()); + + fut.listenAsync(lsnr1, customExec); + + U.sleep(100); + + assertEquals(1, latch1.getCount()); + + fut0.onDone("test"); + + assert latch1.await(1, TimeUnit.SECONDS) : latch1.getCount(); + + checkAsyncListener(fut); + checkAsyncListener(createFuture(new GridFinishedFuture<>("test"))); + } + + /** + * @param fut Future. + */ + private void checkAsyncListener(IgniteFutureImpl fut) throws InterruptedException { + final CountDownLatch latch = new CountDownLatch(1); + + IgniteInClosure> lsnr = createAsyncListener(latch, CUSTOM_THREAD_NAME, null); + + fut.listenAsync(lsnr, customExec); + + assert latch.await(1, TimeUnit.SECONDS) : latch.getCount(); + } + + /** + * @throws Exception If failed. + */ + public void testAsyncListenersOnError() throws Exception { + checkAsyncListenerOnError(new IgniteException("Test exception")); + checkAsyncListenerOnError(new IgniteCheckedException("Test checked exception")); + } + + /** + * @param err0 Test exception. + */ + private void checkAsyncListenerOnError(Exception err0) throws InterruptedException { + GridFutureAdapter fut0 = new GridFutureAdapter<>(); + + IgniteFutureImpl fut = createFuture(fut0); + + final CountDownLatch latch1 = new CountDownLatch(1); + + IgniteInClosure> lsnr1 = createAsyncListener(latch1, CUSTOM_THREAD_NAME, err0); + + fut.listenAsync(lsnr1, customExec); + + assertEquals(1, latch1.getCount()); + + fut0.onDone(err0); + + assert latch1.await(1, TimeUnit.SECONDS); + + checkAsyncListenerOnError(err0, fut); + checkAsyncListenerOnError(err0, createFuture(new GridFinishedFuture(err0))); + } + + /** + * @param err0 Err 0. + * @param fut Future. + */ + private void checkAsyncListenerOnError(Exception err0, IgniteFutureImpl fut) throws InterruptedException { + final CountDownLatch latch = new CountDownLatch(1); + + IgniteInClosure> lsnr = createAsyncListener(latch, CUSTOM_THREAD_NAME, err0); + + fut.listenAsync(lsnr, customExec); + + assert latch.await(1, TimeUnit.SECONDS); + } + + /** + * @param latch Latch. + */ + @NotNull private CI1> createAsyncListener( + final CountDownLatch latch, + final String threadName, + final Exception err + ) { + return new CI1>() { + @Override public void apply(IgniteFuture fut) { + try { + String tname = Thread.currentThread().getName(); + + assert tname.contains(threadName) : tname; + + assertEquals("test", fut.get()); + + if (err != null) + fail(); + } + catch (IgniteException | CacheException e) { + if (err != null) + assertExpectedException(e, err); + else + throw e; + } + finally { + latch.countDown(); + } + } + }; + } + /** * @throws Exception If failed. */ public void testChain() throws Exception { GridFutureAdapter fut0 = new GridFutureAdapter<>(); - IgniteFutureImpl fut = new IgniteFutureImpl<>(fut0); + IgniteFutureImpl fut = createFuture(fut0); IgniteFuture chained = fut.chain(new C1, Integer>() { @Override public Integer apply(IgniteFuture fut) { @@ -312,7 +460,7 @@ public void testChainError() throws Exception { { GridFutureAdapter fut0 = new GridFutureAdapter<>(); - IgniteFutureImpl fut = new IgniteFutureImpl<>(fut0); + IgniteFutureImpl fut = createFuture(fut0); final IgniteException err0 = new IgniteException("test error"); @@ -327,8 +475,8 @@ public void testChainError() throws Exception { return -1; } - catch (IgniteException err) { - assertEquals(err0, err); + catch (IgniteException | CacheException err) { + assertExpectedException(err, err0); chainedPassed.set(true); @@ -346,8 +494,8 @@ public void testChainError() throws Exception { fail(); } - catch (IgniteException err) { - assertEquals(err0, err); + catch (IgniteException | CacheException err) { + assertExpectedException(err, err0); lsnrPassed.set(true); } @@ -367,8 +515,8 @@ public void testChainError() throws Exception { fail(); } - catch (IgniteException err) { - assertEquals(err0, err); + catch (IgniteException | CacheException err) { + assertExpectedException(err, err0); } try { @@ -376,15 +524,15 @@ public void testChainError() throws Exception { fail(); } - catch (IgniteException err) { - assertEquals(err0, err); + catch (IgniteException | CacheException err) { + assertExpectedException(err, err0); } } { GridFutureAdapter fut0 = new GridFutureAdapter<>(); - IgniteFutureImpl fut = new IgniteFutureImpl<>(fut0); + IgniteFutureImpl fut = createFuture(fut0); final IgniteCheckedException err0 = new IgniteCheckedException("test error"); @@ -399,8 +547,8 @@ public void testChainError() throws Exception { return -1; } - catch (IgniteException err) { - assertEquals(err0, err.getCause()); + catch (IgniteException | CacheException err) { + assertExpectedException(err, err0); chainedPassed.set(true); @@ -418,8 +566,8 @@ public void testChainError() throws Exception { fail(); } - catch (IgniteException err) { - assertEquals(err0, err.getCause()); + catch (IgniteException | CacheException err) { + assertExpectedException(err, err0); lsnrPassed.set(true); } @@ -439,8 +587,8 @@ public void testChainError() throws Exception { fail(); } - catch (IgniteException err) { - assertEquals(err0, err.getCause()); + catch (IgniteException | CacheException err) { + assertExpectedException(err, err0); } try { @@ -448,9 +596,243 @@ public void testChainError() throws Exception { fail(); } - catch (IgniteException err) { - assertEquals(err0, err.getCause()); + catch (IgniteException | CacheException err) { + assertExpectedException(err, err0); } } } + + /** + * @throws Exception If failed. + */ + public void testChainAsync() throws Exception { + GridFutureAdapter fut0 = new GridFutureAdapter<>(); + + IgniteFuture fut = createFuture(fut0); + + C1, Integer> chainClos = new C1, Integer>() { + @Override public Integer apply(IgniteFuture fut) { + assertEquals(CUSTOM_THREAD_NAME, Thread.currentThread().getName()); + + return Integer.valueOf(fut.get()); + } + }; + + IgniteFuture chained1 = fut.chainAsync(chainClos, customExec); + + assertFalse(chained1.isDone()); + + final CountDownLatch latch = new CountDownLatch(1); + + class TestClosure implements CI1> { + private final CountDownLatch latch; + + private TestClosure(CountDownLatch latch) { + this.latch = latch; + } + + @Override public void apply(IgniteFuture fut) { + assertEquals(CUSTOM_THREAD_NAME, Thread.currentThread().getName()); + assertEquals(10, (int)fut.get()); + + latch.countDown(); + } + } + + chained1.listen(new TestClosure(latch)); + + fut0.onDone("10"); + + // Chained future will be completed asynchronously. + chained1.get(100, TimeUnit.MILLISECONDS); + + assertTrue(chained1.isDone()); + + assertEquals(10, (int)chained1.get()); + + assert latch.await(100, TimeUnit.MILLISECONDS); + + assertTrue(fut.isDone()); + + assertEquals("10", fut.get()); + + // Test finished future + GridFinishedFuture ffut0 = new GridFinishedFuture<>("10"); + + CountDownLatch latch1 = new CountDownLatch(1); + + IgniteFuture chained2 = createFuture(ffut0).chainAsync(chainClos, customExec); + + chained2.listen(new TestClosure(latch1)); + + chained2.get(100, TimeUnit.MILLISECONDS); + + assertTrue(chained2.isDone()); + + assertEquals(10, (int)chained2.get()); + + assert latch1.await(100, TimeUnit.MILLISECONDS); + } + + /** + * @throws Exception If failed. + */ + public void testChainAsyncOnError() throws Exception { + checkChainedOnError(new IgniteException("Test exception")); + checkChainedOnError(new IgniteCheckedException("Test checked exception")); + checkChainedOnErrorFinishedFuture(new IgniteException("Test exception")); + checkChainedOnErrorFinishedFuture(new IgniteCheckedException("Test checked exception")); + } + + /** + * @param err Exception. + * @throws Exception If failed. + */ + private void checkChainedOnError(final Exception err) throws Exception { + GridFutureAdapter fut0 = new GridFutureAdapter<>(); + + IgniteFutureImpl fut = createFuture(fut0); + + // Chain callback will be invoked in specific executor. + IgniteFuture chained1 = fut.chainAsync(new C1, Integer>() { + @Override public Integer apply(IgniteFuture fut) { + assertEquals(CUSTOM_THREAD_NAME, Thread.currentThread().getName()); + + try { + fut.get(); + + fail(); + } + catch (IgniteException | CacheException e) { + assertExpectedException(e, err); + + throw e; + } + + return -1; + } + }, customExec); + + assertFalse(chained1.isDone()); + assertFalse(fut.isDone()); + + final CountDownLatch latch = new CountDownLatch(1); + + chained1.listen(new CI1>() { + @Override public void apply(IgniteFuture fut) { + try { + assertEquals(CUSTOM_THREAD_NAME, Thread.currentThread().getName()); + + fut.get(); + + fail(); + } + catch (IgniteException | CacheException e) { + assertExpectedException(e, err); + } + finally { + latch.countDown(); + } + } + }); + + fut0.onDone(err); + + assertExceptionThrown(err, chained1); + assertExceptionThrown(err, fut); + + assertTrue(chained1.isDone()); + assertTrue(fut.isDone()); + + assert latch.await(100, TimeUnit.MILLISECONDS); + } + + /** + * @param err Err. + */ + private void checkChainedOnErrorFinishedFuture(final Exception err) throws Exception { + IgniteFutureImpl fut = createFuture(new GridFinishedFuture(err)); + + // Chain callback will be invoked in specific executor. + IgniteFuture chained1 = fut.chainAsync(new C1, Integer>() { + @Override public Integer apply(IgniteFuture fut) { + assertEquals(CUSTOM_THREAD_NAME, Thread.currentThread().getName()); + + try { + fut.get(); + + fail(); + } + catch (IgniteException e) { + assertExpectedException(e, err); + + throw e; + } + + return -1; + } + }, customExec); + + + assertExceptionThrown(err, chained1); + assertExceptionThrown(err, fut); + + assertTrue(chained1.isDone()); + assertTrue(fut.isDone()); + } + + /** + * @param err Expected exception. + * @param fut Future. + */ + private void assertExceptionThrown(Exception err, IgniteFuture fut) { + try { + fut.get(); + + fail(); + } + catch (IgniteException | CacheException e) { + assertExpectedException(e, err); + } + } + + /** + * @param e Actual exception. + * @param exp Expected exception. + */ + protected void assertExpectedException(Exception e, Exception exp) { + if (exp instanceof IgniteException) + assertEquals(exp, e); + else + assertEquals(exp, e.getCause()); + } + + /** + * @param name Name. + */ + @NotNull private ExecutorService createExecutor(final String name) { + return Executors.newSingleThreadExecutor(new ThreadFactory() { + @Override public Thread newThread(@NotNull Runnable r) { + Thread t = new Thread(r); + + t.setName(name); + + return t; + } + }); + } + + /** + * @param fut Future. + */ + protected IgniteFutureImpl createFuture(IgniteInternalFuture fut) { + return new IgniteFutureImpl<>(fut); + } + + /** + * + */ + protected Class expectedException() { + return IgniteException.class; + } } \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteLangSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteLangSelfTestSuite.java index cfec1ec417fb1..70aec7270d06d 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteLangSelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteLangSelfTestSuite.java @@ -21,6 +21,7 @@ import org.apache.ignite.internal.util.future.GridCompoundFutureSelfTest; import org.apache.ignite.internal.util.future.GridEmbeddedFutureSelfTest; import org.apache.ignite.internal.util.future.GridFutureAdapterSelfTest; +import org.apache.ignite.internal.util.future.IgniteCacheFutureImplTest; import org.apache.ignite.internal.util.future.IgniteFutureImplTest; import org.apache.ignite.internal.util.future.nio.GridNioEmbeddedFutureSelfTest; import org.apache.ignite.internal.util.future.nio.GridNioFutureSelfTest; @@ -80,7 +81,9 @@ public static TestSuite suite() throws Exception { suite.addTest(new TestSuite(GridEmbeddedFutureSelfTest.class)); suite.addTest(new TestSuite(GridNioFutureSelfTest.class)); suite.addTest(new TestSuite(GridNioEmbeddedFutureSelfTest.class)); + suite.addTest(new TestSuite(IgniteFutureImplTest.class)); + suite.addTest(new TestSuite(IgniteCacheFutureImplTest.class)); // Consistent hash tests. suite.addTest(new TestSuite(GridConsistentHashSelfTest.class)); diff --git a/modules/schedule/src/main/java/org/apache/ignite/internal/processors/schedule/ScheduleFutureImpl.java b/modules/schedule/src/main/java/org/apache/ignite/internal/processors/schedule/ScheduleFutureImpl.java index ac23a7952d077..3536229dc9f49 100644 --- a/modules/schedule/src/main/java/org/apache/ignite/internal/processors/schedule/ScheduleFutureImpl.java +++ b/modules/schedule/src/main/java/org/apache/ignite/internal/processors/schedule/ScheduleFutureImpl.java @@ -25,6 +25,7 @@ import java.util.Collection; import java.util.concurrent.Callable; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.regex.Matcher; @@ -35,6 +36,7 @@ import org.apache.ignite.IgniteLogger; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.processors.timeout.GridTimeoutObjectAdapter; +import org.apache.ignite.internal.util.future.AsyncFutureListener; import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.future.IgniteFutureImpl; import org.apache.ignite.internal.util.lang.GridClosureException; @@ -588,16 +590,43 @@ private void parsePatternParameters() throws IgniteCheckedException { notifyListener(lsnr, res, err); } + /** {@inheritDoc} */ + @Override public void listenAsync(IgniteInClosure> lsnr, Executor exec) { + A.notNull(lsnr, "lsnr"); + A.notNull(exec, "exec"); + + listen(new AsyncFutureListener<>(lsnr, exec)); + } + /** {@inheritDoc} */ @SuppressWarnings("ExternalizableWithoutPublicNoArgConstructor") @Override public IgniteFuture chain(final IgniteClosure, T> doneCb) { + A.notNull(doneCb, "doneCb"); + + return chain(doneCb, null); + } + + /** {@inheritDoc} */ + @Override public IgniteFuture chainAsync(IgniteClosure, T> doneCb, Executor exec) { + A.notNull(doneCb, ""); + A.notNull(exec, "exec"); + + return chain(doneCb, exec); + } + + /** + * @param doneCb Done callback. + * @param exec Executor. + * @return Chained future. + */ + private IgniteFuture chain(final IgniteClosure, T> doneCb, @Nullable Executor exec) { final GridFutureAdapter fut = new GridFutureAdapter() { @Override public String toString() { return "ChainFuture[orig=" + ScheduleFutureImpl.this + ", doneCb=" + doneCb + ']'; } }; - listen(new CI1>() { + IgniteInClosure> lsnr = new CI1>() { @Override public void apply(IgniteFuture fut0) { try { fut.onDone(doneCb.apply(fut0)); @@ -617,7 +646,12 @@ private void parsePatternParameters() throws IgniteCheckedException { throw e; } } - }); + }; + + if (exec != null) + lsnr = new AsyncFutureListener<>(lsnr, exec); + + listen(lsnr); return new IgniteFutureImpl<>(fut); } @@ -880,10 +914,21 @@ private static class ScheduleFutureSnapshot implements SchedulerFuture { ref.listen(lsnr); } + /** {@inheritDoc} */ + @Override public void listenAsync(IgniteInClosure> lsnr, Executor exec) { + ref.listenAsync(lsnr, exec); + } + /** {@inheritDoc} */ @Override public IgniteFuture chain(IgniteClosure, T> doneCb) { return ref.chain(doneCb); } + + /** {@inheritDoc} */ + @Override public IgniteFuture chainAsync(IgniteClosure, T> doneCb, + Executor exec) { + return ref.chainAsync(doneCb, exec); + } } /** {@inheritDoc} */ diff --git a/modules/schedule/src/test/java/org/apache/ignite/internal/processors/schedule/GridScheduleSelfTest.java b/modules/schedule/src/test/java/org/apache/ignite/internal/processors/schedule/GridScheduleSelfTest.java index f0860f219f9ed..122d98ddcbe3a 100644 --- a/modules/schedule/src/test/java/org/apache/ignite/internal/processors/schedule/GridScheduleSelfTest.java +++ b/modules/schedule/src/test/java/org/apache/ignite/internal/processors/schedule/GridScheduleSelfTest.java @@ -19,21 +19,29 @@ import java.util.concurrent.Callable; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadFactory; import java.util.concurrent.atomic.AtomicInteger; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.internal.util.lang.GridTuple; import org.apache.ignite.internal.util.typedef.CI1; +import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteCallable; +import org.apache.ignite.lang.IgniteClosure; import org.apache.ignite.lang.IgniteFuture; +import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.lang.IgniteRunnable; import org.apache.ignite.resources.IgniteInstanceResource; import org.apache.ignite.resources.LoggerResource; import org.apache.ignite.scheduler.SchedulerFuture; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.jetbrains.annotations.NotNull; import static java.util.concurrent.TimeUnit.SECONDS; +import static org.apache.ignite.testframework.GridTestUtils.assertThrows; /** * Test for task scheduler. @@ -43,9 +51,15 @@ public class GridScheduleSelfTest extends GridCommonAbstractTest { /** */ private static final int NODES_CNT = 2; + /** Custom thread name. */ + private static final String CUSTOM_THREAD_NAME = "custom-async-test"; + /** */ private static AtomicInteger execCntr = new AtomicInteger(0); + /** Custom executor. */ + private ExecutorService exec; + /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { startGrids(NODES_CNT); @@ -61,6 +75,22 @@ public class GridScheduleSelfTest extends GridCommonAbstractTest { /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { execCntr.set(0); + exec = Executors.newSingleThreadExecutor(new ThreadFactory() { + @Override public Thread newThread(@NotNull Runnable r) { + Thread t = new Thread(r); + + t.setName(CUSTOM_THREAD_NAME); + + return t; + } + }); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + U.shutdownNow(getClass(), exec, log); + + exec = null; } /** @@ -124,6 +154,54 @@ public void testScheduleRunnable() throws Exception { } }); + final SchedulerFuture fut0 = fut; + + //noinspection ThrowableNotThrown + assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + fut0.listenAsync(new IgniteInClosure>() { + @Override public void apply(IgniteFuture fut) { + // No-op + } + }, null); + + return null; + } + }, NullPointerException.class, null); + + fut.listenAsync(new IgniteInClosure>() { + @Override public void apply(IgniteFuture fut) { + assertEquals(Thread.currentThread().getName(), CUSTOM_THREAD_NAME); + + notifyCnt.incrementAndGet(); + } + }, exec); + + //noinspection ThrowableNotThrown + assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + fut0.chainAsync(new IgniteClosure, String>() { + @Override public String apply(IgniteFuture fut) { + // No-op + + return null; + } + }, null); + + return null; + } + }, NullPointerException.class, null); + + IgniteFuture chained1 = fut.chainAsync(new IgniteClosure, String>() { + @Override public String apply(IgniteFuture fut) { + assertEquals(Thread.currentThread().getName(), CUSTOM_THREAD_NAME); + + fut.get(); + + return "done-custom"; + } + }, exec); + long timeTillRun = freq + delay; info("Going to wait for the first run: " + timeTillRun); @@ -135,6 +213,7 @@ public void testScheduleRunnable() throws Exception { assert !fut.isDone(); assert !fut.isCancelled(); assert fut.last() == null; + assertFalse(chained1.isDone()); info("Going to wait for 2nd run: " + timeTillRun); @@ -142,10 +221,13 @@ public void testScheduleRunnable() throws Exception { Thread.sleep(timeTillRun * 1000); assert fut.isDone(); - assert notifyCnt.get() == 2; + assert notifyCnt.get() == 2 * 2; assert !fut.isCancelled(); assert fut.last() == null; + assertEquals("done-custom", chained1.get()); + + assertTrue(chained1.isDone()); } finally { assert fut != null; From cae22a3324f90c739c1cdf51504188f164096396 Mon Sep 17 00:00:00 2001 From: apopov Date: Thu, 9 Nov 2017 11:13:53 +0300 Subject: [PATCH 337/357] GG-13042 Backport GG-13035 changes to 1.9.x to keep libs in sync --- examples/pom-standalone-lgpl.xml | 8 +------- examples/pom-standalone.xml | 8 +------- examples/pom.xml | 14 +------------- modules/clients/pom.xml | 8 +------- modules/ignored-tests/pom.xml | 10 ++-------- 5 files changed, 6 insertions(+), 42 deletions(-) diff --git a/examples/pom-standalone-lgpl.xml b/examples/pom-standalone-lgpl.xml index f664514d5c83f..e73c416a39e63 100644 --- a/examples/pom-standalone-lgpl.xml +++ b/examples/pom-standalone-lgpl.xml @@ -74,13 +74,7 @@ com.google.code.simple-spring-memcached spymemcached - 2.7.3 - - - commons-codec - commons-codec - - + 2.8.4 diff --git a/examples/pom-standalone.xml b/examples/pom-standalone.xml index 029ca777ae242..aa93273234201 100644 --- a/examples/pom-standalone.xml +++ b/examples/pom-standalone.xml @@ -74,13 +74,7 @@ com.google.code.simple-spring-memcached spymemcached - 2.7.3 - - - commons-codec - commons-codec - - + 2.8.4 diff --git a/examples/pom.xml b/examples/pom.xml index e7fd67d197dac..7bc96f570fb08 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -83,13 +83,7 @@ com.google.code.simple-spring-memcached spymemcached - 2.7.3 - - - commons-codec - commons-codec - - + 2.8.4 @@ -152,12 +146,6 @@ ignite-spark ${project.version} - - - org.jboss.netty - netty - 3.2.9.Final - diff --git a/modules/clients/pom.xml b/modules/clients/pom.xml index 022edb78f3b66..e3804355f503f 100644 --- a/modules/clients/pom.xml +++ b/modules/clients/pom.xml @@ -44,14 +44,8 @@ com.google.code.simple-spring-memcached spymemcached - 2.7.3 + 2.8.4 test - - - commons-codec - commons-codec - - diff --git a/modules/ignored-tests/pom.xml b/modules/ignored-tests/pom.xml index 00c7d554d7650..8e3993b29022f 100644 --- a/modules/ignored-tests/pom.xml +++ b/modules/ignored-tests/pom.xml @@ -164,14 +164,8 @@ com.google.code.simple-spring-memcached spymemcached - 2.7.3 - test - - - commons-codec - commons-codec - - + 2.8.4 + test From 02b13a5d2f266bc28147b79e65ade7246a70f7d0 Mon Sep 17 00:00:00 2001 From: Slava Koptilin Date: Thu, 9 Nov 2017 18:10:31 +0300 Subject: [PATCH 338/357] ignite-6669 Do not call CacheStoreSessionListener if store operation is not executed --- .../store/GridCacheStoreManagerAdapter.java | 98 +++++- ...tenerRWThroughDisabledAtomicCacheTest.java | 33 ++ ...ThroughDisabledTransactionalCacheTest.java | 138 ++++++++ ...nListenerReadWriteThroughDisabledTest.java | 296 +++++++++++++++++ ...SessionListenerWriteBehindEnabledTest.java | 309 ++++++++++++++++++ .../testsuites/IgniteCacheTestSuite4.java | 8 +- .../Cache/Store/CacheStoreSessionTest.cs | 15 +- 7 files changed, 869 insertions(+), 28 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreListenerRWThroughDisabledAtomicCacheTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreListenerRWThroughDisabledTransactionalCacheTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerReadWriteThroughDisabledTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerWriteBehindEnabledTest.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 e8ce7236052f5..3dbec4d21a56a 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 @@ -31,16 +31,12 @@ import javax.cache.integration.CacheLoaderException; import javax.cache.integration.CacheWriterException; import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.IgniteException; -import org.apache.ignite.cache.store.CacheStore; import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.cache.store.CacheStore; -import org.apache.ignite.cache.store.CacheStore; import org.apache.ignite.cache.store.CacheStoreSession; import org.apache.ignite.cache.store.CacheStoreSessionListener; import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStore; import org.apache.ignite.cluster.ClusterNode; -import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStore; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.processors.cache.CacheEntryImpl; @@ -112,6 +108,9 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt /** */ private boolean writeThrough; + /** */ + private boolean readThrough; + /** */ private Collection sesLsnrs; @@ -129,6 +128,8 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt writeThrough = cfg.isWriteThrough(); + readThrough = cfg.isReadThrough(); + this.cfgStore = cfgStore; store = cacheStoreWrapper(ctx, cfgStore, cfg); @@ -329,7 +330,7 @@ private CacheStore cacheStoreWrapper(GridKernalContext ctx, log.debug(S.toString("Loading value from store for key", "key", storeKey, true)); - sessionInit0(tx); + sessionInit0(tx, StoreOperation.READ, false); boolean threwEx = true; @@ -463,7 +464,7 @@ private void loadAllFromStore(@Nullable IgniteInternalTx tx, if (log.isDebugEnabled()) log.debug("Loading values from store for keys: " + keys0); - sessionInit0(tx); + sessionInit0(tx, StoreOperation.READ, false); boolean threwEx = true; @@ -522,7 +523,7 @@ private void loadAllFromStore(@Nullable IgniteInternalTx tx, if (log.isDebugEnabled()) log.debug("Loading all values from store."); - sessionInit0(null); + sessionInit0(null, StoreOperation.READ, false); boolean threwEx = true; @@ -588,7 +589,7 @@ private void loadAllFromStore(@Nullable IgniteInternalTx tx, "val", val, true)); } - sessionInit0(tx); + sessionInit0(tx, StoreOperation.WRITE, false); boolean threwEx = true; @@ -640,7 +641,7 @@ private void loadAllFromStore(@Nullable IgniteInternalTx tx, if (log.isDebugEnabled()) log.debug("Storing values in cache store [entries=" + entries + ']'); - sessionInit0(tx); + sessionInit0(tx, StoreOperation.WRITE, false); boolean threwEx = true; @@ -693,7 +694,7 @@ private void loadAllFromStore(@Nullable IgniteInternalTx tx, if (log.isDebugEnabled()) log.debug(S.toString("Removing value from cache store", "key", key, true)); - sessionInit0(tx); + sessionInit0(tx, StoreOperation.WRITE, false); boolean threwEx = true; @@ -742,7 +743,7 @@ private void loadAllFromStore(@Nullable IgniteInternalTx tx, log.debug(S.toString("Removing values from cache store", "keys", keys0, true)); - sessionInit0(tx); + sessionInit0(tx, StoreOperation.WRITE, false); boolean threwEx = true; @@ -793,10 +794,10 @@ private void loadAllFromStore(@Nullable IgniteInternalTx tx, boolean storeSessionEnded) throws IgniteCheckedException { assert store != null; - sessionInit0(tx); + sessionInit0(tx, commit? StoreOperation.COMMIT: StoreOperation.ROLLBACK, false); try { - if (sesLsnrs != null) { + if (sesLsnrs != null && sesHolder.get().contains(store)) { for (CacheStoreSessionListener lsnr : sesLsnrs) lsnr.onSessionEnd(locSes, commit); } @@ -835,7 +836,7 @@ private void handleClassCastException(ClassCastException e) throws IgniteChecked /** {@inheritDoc} */ @Override public void writeBehindSessionInit() throws IgniteCheckedException { - sessionInit0(null); + sessionInit0(null, null, true); } /** {@inheritDoc} */ @@ -845,9 +846,12 @@ private void handleClassCastException(ClassCastException e) throws IgniteChecked /** * @param tx Current transaction. + * @param op Store operation. + * @param writeBehindStoreInitiator {@code true} if method call is initiated by {@link GridCacheWriteBehindStore}. * @throws IgniteCheckedException If failed. */ - private void sessionInit0(@Nullable IgniteInternalTx tx) throws IgniteCheckedException { + private void sessionInit0(@Nullable IgniteInternalTx tx, @Nullable StoreOperation op, + boolean writeBehindStoreInitiator) throws IgniteCheckedException { assert sesHolder != null; SessionData ses; @@ -869,8 +873,45 @@ private void sessionInit0(@Nullable IgniteInternalTx tx) throws IgniteCheckedExc sesHolder.set(ses); + notifyCacheStoreSessionListeners(ses, op, writeBehindStoreInitiator); + } + + /** + * @param ses Current session. + * @param op Store operation. + * @param writeBehindStoreInitiator {@code True} if method call is initiated by {@link GridCacheWriteBehindStore}. + * @throws IgniteCheckedException If failed. + */ + private void notifyCacheStoreSessionListeners(SessionData ses, @Nullable StoreOperation op, + boolean writeBehindStoreInitiator) throws IgniteCheckedException { try { - if (!ses.started(store) && sesLsnrs != null) { + boolean notifyLsnrs = false; + + if (writeBehindStoreInitiator) + notifyLsnrs = !ses.started(store) && sesLsnrs != null; + else { + assert op != null; + + switch (op) { + case READ: + notifyLsnrs = readThrough && !ses.started(store) && sesLsnrs != null; + break; + + case WRITE: + notifyLsnrs = !cacheConfiguration().isWriteBehindEnabled() && writeThrough + && !ses.started(store) && sesLsnrs != null; + break; + + case COMMIT: + case ROLLBACK: + // No needs to start the session (if not started yet) and notify listeners. + break; + + default: + assert false : "Unexpected operation: " + op.toString(); + } + } + if (notifyLsnrs) { for (CacheStoreSessionListener lsnr : sesLsnrs) lsnr.onSessionStart(locSes); } @@ -886,7 +927,7 @@ private void sessionInit0(@Nullable IgniteInternalTx tx) throws IgniteCheckedExc private void sessionEnd0(@Nullable IgniteInternalTx tx, boolean threwEx) throws IgniteCheckedException { try { if (tx == null) { - if (sesLsnrs != null) { + if (sesLsnrs != null && sesHolder.get().contains(store)) { for (CacheStoreSessionListener lsnr : sesLsnrs) lsnr.onSessionEnd(locSes, !threwEx); } @@ -1010,6 +1051,14 @@ private boolean ended(CacheStore store) { return !started.remove(store); } + /** + * @param store Cache store. + * @return {@code True} if session started. + */ + private boolean contains(CacheStore store) { + return started.contains(store); + } + /** {@inheritDoc} */ @Override public String toString() { return S.toString(SessionData.class, this, "tx", CU.txString(tx)); @@ -1312,4 +1361,19 @@ public String toString() { } } } + + /** Enumeration that represents possible operations on the underlying store. */ + private enum StoreOperation { + /** Read key-value pair from the underlying store. */ + READ, + + /** Update or remove key from the underlying store. */ + WRITE, + + /** Commit changes to the underlying store. */ + COMMIT, + + /** Rollback changes to the underlying store. */ + ROLLBACK + } } diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreListenerRWThroughDisabledAtomicCacheTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreListenerRWThroughDisabledAtomicCacheTest.java new file mode 100644 index 0000000000000..6e28a52b30cc0 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreListenerRWThroughDisabledAtomicCacheTest.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.cache.store; + +import org.apache.ignite.cache.CacheAtomicityMode; + +import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC; + +/** + * This class tests that redundant calls of {@link CacheStoreSessionListener#onSessionStart(CacheStoreSession)} + * and {@link CacheStoreSessionListener#onSessionEnd(CacheStoreSession, boolean)} are not executed. + */ +public class CacheStoreListenerRWThroughDisabledAtomicCacheTest extends CacheStoreSessionListenerReadWriteThroughDisabledTest { + /** {@inheritDoc} */ + @Override protected CacheAtomicityMode atomicityMode() { + return ATOMIC; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreListenerRWThroughDisabledTransactionalCacheTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreListenerRWThroughDisabledTransactionalCacheTest.java new file mode 100644 index 0000000000000..fd784a364eca4 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreListenerRWThroughDisabledTransactionalCacheTest.java @@ -0,0 +1,138 @@ +/* + * 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.store; + +import java.util.Random; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.transactions.Transaction; +import org.apache.ignite.transactions.TransactionConcurrency; +import org.apache.ignite.transactions.TransactionIsolation; + +import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; +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; +import static org.apache.ignite.transactions.TransactionIsolation.SERIALIZABLE; + +/** + * This class tests that redundant calls of {@link CacheStoreSessionListener#onSessionStart(CacheStoreSession)} + * and {@link CacheStoreSessionListener#onSessionEnd(CacheStoreSession, boolean)} are not executed. + */ +public class CacheStoreListenerRWThroughDisabledTransactionalCacheTest extends CacheStoreSessionListenerReadWriteThroughDisabledTest { + /** {@inheritDoc} */ + @Override protected CacheAtomicityMode atomicityMode() { + return TRANSACTIONAL; + } + + /** + * Tests {@link IgniteCache#get(Object)} with disabled read-through and write-through modes. + */ + public void testTransactionalLookup() { + testTransactionalLookup(OPTIMISTIC, READ_COMMITTED); + testTransactionalLookup(OPTIMISTIC, REPEATABLE_READ); + testTransactionalLookup(OPTIMISTIC, SERIALIZABLE); + + testTransactionalLookup(PESSIMISTIC, READ_COMMITTED); + testTransactionalLookup(PESSIMISTIC, REPEATABLE_READ); + testTransactionalLookup(PESSIMISTIC, SERIALIZABLE); + } + + /** + * @param concurrency Transaction concurrency level. + * @param isolation Transaction isolation level. + */ + private void testTransactionalLookup(TransactionConcurrency concurrency, TransactionIsolation isolation) { + IgniteCache cache = grid(0).getOrCreateCache(DEFAULT_CACHE_NAME); + + Random r = new Random(); + + try (Transaction tx = grid(0).transactions().txStart(concurrency, isolation)) { + for (int i = 0; i < CNT; ++i) + cache.get(r.nextInt()); + + tx.commit(); + } + } + + /** + * Tests {@link IgniteCache#put(Object, Object)} with disabled read-through and write-through modes. + */ + public void testTransactionalUpdate() { + testTransactionalUpdate(OPTIMISTIC, READ_COMMITTED); + testTransactionalUpdate(OPTIMISTIC, REPEATABLE_READ); + testTransactionalUpdate(OPTIMISTIC, SERIALIZABLE); + + testTransactionalUpdate(PESSIMISTIC, READ_COMMITTED); + testTransactionalUpdate(PESSIMISTIC, REPEATABLE_READ); + testTransactionalUpdate(PESSIMISTIC, SERIALIZABLE); + } + + /** + * @param concurrency Transaction concurrency level. + * @param isolation Transaction isolation level. + */ + private void testTransactionalUpdate(TransactionConcurrency concurrency, TransactionIsolation isolation) { + IgniteCache cache = grid(0).getOrCreateCache(DEFAULT_CACHE_NAME); + + Random r = new Random(); + + try (Transaction tx = grid(0).transactions().txStart(concurrency, isolation)) { + for (int i = 0; i < CNT; ++i) + cache.put(r.nextInt(), "test-value"); + + tx.commit(); + } + } + + /** + * Tests {@link IgniteCache#remove(Object)} with disabled read-through and write-through modes. + */ + public void testTransactionalRemove() { + testTransactionalRemove(OPTIMISTIC, READ_COMMITTED); + testTransactionalRemove(OPTIMISTIC, REPEATABLE_READ); + testTransactionalRemove(OPTIMISTIC, SERIALIZABLE); + + testTransactionalRemove(PESSIMISTIC, READ_COMMITTED); + testTransactionalRemove(PESSIMISTIC, REPEATABLE_READ); + testTransactionalRemove(PESSIMISTIC, SERIALIZABLE); + } + + /** + * @param concurrency Transaction concurrency level. + * @param isolation Transaction isolation level. + */ + private void testTransactionalRemove(TransactionConcurrency concurrency, TransactionIsolation isolation) { + IgniteCache cache = grid(0).getOrCreateCache(DEFAULT_CACHE_NAME); + + Random r = new Random(); + + try (Transaction tx = grid(0).transactions().txStart(concurrency, isolation)) { + for (int i = 0; i < CNT; ++i) { + int key = r.nextInt(); + + cache.put(key, "test-value"); + + cache.remove(key, "test-value"); + } + + tx.commit(); + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerReadWriteThroughDisabledTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerReadWriteThroughDisabledTest.java new file mode 100644 index 0000000000000..22691213227a5 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerReadWriteThroughDisabledTest.java @@ -0,0 +1,296 @@ +/* + * 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.store; + +import java.io.PrintWriter; +import java.io.Serializable; +import java.sql.Connection; +import java.sql.SQLException; +import java.sql.SQLFeatureNotSupportedException; +import java.util.HashSet; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.TreeMap; +import java.util.logging.Logger; +import javax.cache.Cache; +import javax.cache.configuration.Factory; +import javax.cache.configuration.FactoryBuilder; +import javax.cache.integration.CacheLoaderException; +import javax.cache.integration.CacheWriterException; +import javax.sql.DataSource; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.store.jdbc.CacheJdbcStoreSessionListener; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.NearCacheConfiguration; +import org.apache.ignite.internal.processors.cache.GridCacheAbstractSelfTest; + +/** + * This class tests that redundant calls of {@link CacheStoreSessionListener#onSessionStart(CacheStoreSession)} + * and {@link CacheStoreSessionListener#onSessionEnd(CacheStoreSession, boolean)} are not executed. + */ +public abstract class CacheStoreSessionListenerReadWriteThroughDisabledTest extends GridCacheAbstractSelfTest { + /** {@inheritDoc} */ + protected int gridCount() { + return 2; + } + + /** */ + protected final int CNT = 100; + + /** */ + protected final String DEFAULT_CACHE_NAME = null; + + /** {@inheritDoc} */ + protected CacheConfiguration cacheConfiguration(String igniteInstanceName) throws Exception { + CacheConfiguration cacheCfg = super.cacheConfiguration(igniteInstanceName); + + cacheCfg.setName(DEFAULT_CACHE_NAME); + + cacheCfg.setCacheStoreFactory(FactoryBuilder.factoryOf(EmptyCacheStore.class)); + + cacheCfg.setCacheStoreSessionListenerFactories(new CacheStoreSessionFactory()); + + cacheCfg.setReadThrough(false); + cacheCfg.setWriteThrough(false); + + cacheCfg.setBackups(0); + + return cacheCfg; + } + + /** {@inheritDoc} */ + protected NearCacheConfiguration nearConfiguration() { + return null; + } + + /** + * Tests that there are no calls of {@link CacheStoreSessionListener#onSessionStart(CacheStoreSession)} and + * {@link CacheStoreSessionListener#onSessionStart(CacheStoreSession)} + * while {@link IgniteCache#get(Object)} performed. + * + * @throws Exception If failed. + */ + public void testLookup() throws Exception { + IgniteCache cache = grid(0).getOrCreateCache(DEFAULT_CACHE_NAME); + + Random r = new Random(); + + for (int i = 0; i < CNT; ++i) + cache.get(r.nextInt()); + } + + /** + * Tests that there are no calls of {@link CacheStoreSessionListener#onSessionStart(CacheStoreSession)} and + * {@link CacheStoreSessionListener#onSessionStart(CacheStoreSession)} + * while {@link IgniteCache#getAll(Set)} performed. + * + * @throws Exception If failed. + */ + public void testBatchLookup() throws Exception { + IgniteCache cache = grid(0).getOrCreateCache(DEFAULT_CACHE_NAME); + + Random r = new Random(); + + Set values = new HashSet<>(); + + for (int i = 0; i < CNT; ++i) + values.add(r.nextInt()); + + cache.getAll(values); + } + + /** + * Tests that there are no calls of {@link CacheStoreSessionListener#onSessionStart(CacheStoreSession)} and + * {@link CacheStoreSessionListener#onSessionStart(CacheStoreSession)} + * while {@link IgniteCache#put(Object, Object)} performed. + * + * @throws Exception If failed. + */ + public void testUpdate() throws Exception { + IgniteCache cache = grid(0).getOrCreateCache(DEFAULT_CACHE_NAME); + + Random r = new Random(); + + for (int i = 0; i < CNT; ++i) + cache.put(r.nextInt(), "test-value"); + } + + /** + * Tests that there are no calls of {@link CacheStoreSessionListener#onSessionStart(CacheStoreSession)} and + * {@link CacheStoreSessionListener#onSessionStart(CacheStoreSession)} + * while {@link IgniteCache#putAll(Map)} performed. + * + * @throws Exception If failed. + */ + public void testBatchUpdate() throws Exception { + IgniteCache cache = grid(0).getOrCreateCache(DEFAULT_CACHE_NAME); + + Random r = new Random(); + + Map values = new TreeMap<>(); + + for (int i = 0; i < CNT; ++i) + values.put(r.nextInt(), "test-value"); + + cache.putAll(values); + } + + /** + * Tests that there are no calls of {@link CacheStoreSessionListener#onSessionStart(CacheStoreSession)} and + * {@link CacheStoreSessionListener#onSessionStart(CacheStoreSession)} + * while {@link IgniteCache#remove(Object)} performed. + * + * @throws Exception If failed. + */ + public void testRemove() throws Exception { + IgniteCache cache = grid(0).getOrCreateCache(DEFAULT_CACHE_NAME); + + Random r = new Random(); + + for (int i = 0; i < CNT; ++i) { + int key = r.nextInt(); + + cache.put(key, "test-value"); + + cache.remove(key); + } + } + + /** + * Tests that there are no calls of {@link CacheStoreSessionListener#onSessionStart(CacheStoreSession)} and + * {@link CacheStoreSessionListener#onSessionStart(CacheStoreSession)} + * while {@link IgniteCache#removeAll(Set)} performed. + * + * @throws Exception If failed. + */ + public void testBatchRemove() throws Exception { + IgniteCache cache = grid(0).getOrCreateCache(DEFAULT_CACHE_NAME); + + Random r = new Random(); + + Set values = new HashSet<>(); + + for (int i = 0; i < CNT; ++i) { + int key = r.nextInt(); + + cache.put(key, "test-value"); + + values.add(key); + } + + cache.removeAll(values); + } + + /** + * Cache store session factory. + */ + public static class CacheStoreSessionFactory implements Factory { + /** {@inheritDoc} */ + @Override public TestCacheStoreSessionListener create() { + TestCacheStoreSessionListener lsnr = new TestCacheStoreSessionListener(); + lsnr.setDataSource(new DataSourceStub()); + return lsnr; + } + } + + /** + * Test cache store session listener. + */ + public static class TestCacheStoreSessionListener extends CacheJdbcStoreSessionListener { + /** {@inheritDoc} */ + @Override public void onSessionStart(CacheStoreSession ses) { + fail("TestCacheStoreSessionListener.onSessionStart(CacheStoreSession) should not be called."); + } + + /** {@inheritDoc} */ + @Override public void onSessionEnd(CacheStoreSession ses, boolean commit) { + fail("TestCacheStoreSessionListener.onSessionEnd(CacheStoreSession, boolean) should not be called."); + } + } + + /** Empty cache store implementation. All overridden methods should not be called while the test is running. */ + public static class EmptyCacheStore extends CacheStoreAdapter { + /** {@inheritDoc} */ + @Override public Object load(Object key) throws CacheLoaderException { + fail("EmptyCacheStore.load(Object) should not be called."); + + return null; + } + + /** {@inheritDoc} */ + @Override public void write(Cache.Entry entry) throws CacheWriterException { + fail("EmptyCacheStore.write(Cache.Entry) should not be called."); + } + + /** {@inheritDoc} */ + @Override public void delete(Object key) throws CacheWriterException { + fail("EmptyCacheStore.delete(Object) should not be called."); + } + } + + /** + * Data source stub which should not be called. + */ + public static class DataSourceStub implements DataSource, Serializable { + /** {@inheritDoc} */ + @Override public Connection getConnection() throws SQLException { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public Connection getConnection(String username, String password) throws SQLException { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public T unwrap(Class iface) throws SQLException { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public boolean isWrapperFor(Class iface) throws SQLException { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public PrintWriter getLogWriter() throws SQLException { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public void setLogWriter(PrintWriter out) throws SQLException { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public void setLoginTimeout(int seconds) throws SQLException { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public int getLoginTimeout() throws SQLException { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public Logger getParentLogger() throws SQLFeatureNotSupportedException { + throw new UnsupportedOperationException(); + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerWriteBehindEnabledTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerWriteBehindEnabledTest.java new file mode 100644 index 0000000000000..2b04309a40422 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerWriteBehindEnabledTest.java @@ -0,0 +1,309 @@ +/* + * 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.store; + +import java.io.PrintWriter; +import java.io.Serializable; +import java.sql.Connection; +import java.sql.SQLException; +import java.sql.SQLFeatureNotSupportedException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.logging.Logger; +import javax.cache.Cache; +import javax.cache.configuration.Factory; +import javax.cache.configuration.FactoryBuilder; +import javax.cache.integration.CacheLoaderException; +import javax.cache.integration.CacheWriterException; +import javax.sql.DataSource; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteException; +import org.apache.ignite.cache.store.jdbc.CacheJdbcStoreSessionListener; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.processors.cache.GridCacheAbstractSelfTest; +import org.apache.ignite.internal.processors.cache.store.GridCacheWriteBehindStore; +import org.apache.ignite.resources.IgniteInstanceResource; + +/** + * This class tests that calls of {@link CacheStoreSessionListener#onSessionStart(CacheStoreSession)} + * and {@link CacheStoreSessionListener#onSessionEnd(CacheStoreSession, boolean)} are executed from + * {@link GridCacheWriteBehindStore} only. + */ +public class CacheStoreSessionListenerWriteBehindEnabledTest extends GridCacheAbstractSelfTest { + /** */ + protected final static int CNT = 100; + + /** */ + protected final String DEFAULT_CACHE_NAME = null; + + /** */ + private final static int WRITE_BEHIND_FLUSH_FREQUENCY = 1000; + + /** */ + private static final List operations = Collections.synchronizedList(new ArrayList()); + + /** */ + private static final AtomicInteger entryCnt = new AtomicInteger(); + + /** {@inheritDoc} */ + @Override protected int gridCount() { + return 1; + } + + /** {@inheritDoc} */ + @Override protected CacheConfiguration cacheConfiguration(String igniteInstanceName) throws Exception { + CacheConfiguration cacheCfg = super.cacheConfiguration(igniteInstanceName); + + cacheCfg.setName(DEFAULT_CACHE_NAME); + + cacheCfg.setCacheStoreFactory(FactoryBuilder.factoryOf(EmptyCacheStore.class)); + + cacheCfg.setCacheStoreSessionListenerFactories(new CacheStoreSessionFactory()); + + cacheCfg.setReadThrough(true); + cacheCfg.setWriteThrough(true); + + cacheCfg.setWriteBehindEnabled(true); + cacheCfg.setWriteBehindBatchSize(CNT * 2); + cacheCfg.setWriteBehindFlushFrequency(WRITE_BEHIND_FLUSH_FREQUENCY); + + cacheCfg.setBackups(0); + + return cacheCfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + operations.clear(); + + entryCnt.set(0); + } + + /** + * Tests that there are no redundant calls of {@link CacheStoreSessionListener#onSessionStart(CacheStoreSession)} + * while {@link IgniteCache#get(Object)} performed. + */ + public void testLookup() { + IgniteCache cache = grid(0).getOrCreateCache(DEFAULT_CACHE_NAME); + + for (int i = 0; i < CNT; ++i) + cache.get(i); + + checkSessionCounters(CNT); + } + + /** + * Tests that there are no redundant calls of {@link CacheStoreSessionListener#onSessionStart(CacheStoreSession)} + * while {@link IgniteCache#put(Object, Object)} performed. + */ + public void testUpdate() { + IgniteCache cache = grid(0).getOrCreateCache(DEFAULT_CACHE_NAME); + + for (int i = 0; i < CNT; ++i) + cache.put(i, i); + + checkSessionCounters(1); + } + + /** + * Tests that there are no redundant calls of {@link CacheStoreSessionListener#onSessionStart(CacheStoreSession)} + * while {@link IgniteCache#remove(Object)} performed. + */ + public void testRemove() { + IgniteCache cache = grid(0).getOrCreateCache(DEFAULT_CACHE_NAME); + + for (int i = 0; i < CNT; ++i) { + cache.remove(i); + } + + checkSessionCounters(1); + } + + /** + * @param startedSessions Number of expected sessions. + */ + private void checkSessionCounters(int startedSessions) { + try { + // Wait for GridCacheWriteBehindStore + Thread.sleep(WRITE_BEHIND_FLUSH_FREQUENCY * 4); + + assertEquals(CNT, entryCnt.get()); + + checkOpCount(operations, OperationType.SESSION_START, startedSessions); + + checkOpCount(operations, OperationType.SESSION_END, startedSessions); + } + catch (InterruptedException e) { + throw new IgniteException("Failed to wait for the GridCacheWriteBehindStore due to interruption.", e); + } + } + + /** + * @param operations List of {@link OperationType}. + * @param op Operation. + * @param expected Expected number of operations for the given {@code op}. + */ + private void checkOpCount(List operations, OperationType op, int expected) { + int n = 0; + + for (OperationType o : operations) { + if (op.equals(o)) + ++n; + } + + assertEquals("Operation=" + op.name(), expected, n); + } + + /** + * Operation type. + */ + public enum OperationType { + /** + * Cache store session started. + */ + SESSION_START, + + /** + * Cache store session ended. + */ + SESSION_END, + } + + /** + * Cache store session factory. + */ + public static class CacheStoreSessionFactory implements Factory { + /** {@inheritDoc} */ + @Override public TestCacheStoreSessionListener create() { + TestCacheStoreSessionListener lsnr = new TestCacheStoreSessionListener(); + lsnr.setDataSource(new DataSourceStub()); + return lsnr; + } + } + + /** + * Test cache store session listener. + */ + public static class TestCacheStoreSessionListener extends CacheJdbcStoreSessionListener { + /** */ + @IgniteInstanceResource + private Ignite ignite; + + /** {@inheritDoc} */ + @Override public void onSessionStart(CacheStoreSession ses) { + operations.add(OperationType.SESSION_START); + } + + /** {@inheritDoc} */ + @Override public void onSessionEnd(CacheStoreSession ses, boolean commit) { + operations.add(OperationType.SESSION_END); + } + } + + /** + * Test cache store. + * + * {@link EmptyCacheStore#writeAll(Collection)} and {@link EmptyCacheStore#deleteAll(Collection)} should be called + * by {@link GridCacheWriteBehindStore}. + */ + public static class EmptyCacheStore extends CacheStoreAdapter { + /** */ + @IgniteInstanceResource + private Ignite ignite; + + /** {@inheritDoc} */ + @Override public Object load(Object key) throws CacheLoaderException { + entryCnt.getAndIncrement(); + return null; + } + + /** {@inheritDoc} */ + @Override public void writeAll(Collection> entries) { + entryCnt.addAndGet(entries.size()); + } + + /** {@inheritDoc} */ + @Override public void write(Cache.Entry entry) throws CacheWriterException { + } + + /** {@inheritDoc} */ + @Override public void deleteAll(Collection keys) { + entryCnt.addAndGet(keys.size()); + } + + /** {@inheritDoc} */ + @Override public void delete(Object key) throws CacheWriterException { + } + } + + /** + * Data source stub which should not be called. + */ + public static class DataSourceStub implements DataSource, Serializable { + /** {@inheritDoc} */ + @Override public Connection getConnection() throws SQLException { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public Connection getConnection(String username, String password) throws SQLException { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public T unwrap(Class iface) throws SQLException { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public boolean isWrapperFor(Class iface) throws SQLException { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public PrintWriter getLogWriter() throws SQLException { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public void setLogWriter(PrintWriter out) throws SQLException { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public void setLoginTimeout(int seconds) throws SQLException { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public int getLoginTimeout() throws SQLException { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public Logger getParentLogger() throws SQLFeatureNotSupportedException { + throw new UnsupportedOperationException(); + } + } +} 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 c5667ba9a97e8..94492fa9e0b2c 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 @@ -18,6 +18,9 @@ package org.apache.ignite.testsuites; import junit.framework.TestSuite; +import org.apache.ignite.cache.store.CacheStoreListenerRWThroughDisabledAtomicCacheTest; +import org.apache.ignite.cache.store.CacheStoreListenerRWThroughDisabledTransactionalCacheTest; +import org.apache.ignite.cache.store.CacheStoreSessionListenerWriteBehindEnabledTest; import org.apache.ignite.cache.store.jdbc.CacheJdbcStoreSessionListenerSelfTest; import org.apache.ignite.internal.processors.GridCacheTxLoadFromStoreOnLockSelfTest; import org.apache.ignite.internal.processors.cache.CacheClientStoreSelfTest; @@ -283,6 +286,9 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(CacheOffheapMapEntrySelfTest.class); suite.addTestSuite(CacheJdbcStoreSessionListenerSelfTest.class); + suite.addTestSuite(CacheStoreListenerRWThroughDisabledAtomicCacheTest.class); + suite.addTestSuite(CacheStoreListenerRWThroughDisabledTransactionalCacheTest.class); + suite.addTestSuite(CacheStoreSessionListenerWriteBehindEnabledTest.class); suite.addTestSuite(CacheClientStoreSelfTest.class); suite.addTestSuite(CacheStoreUsageMultinodeStaticStartAtomicTest.class); @@ -350,4 +356,4 @@ public static TestSuite suite() throws Exception { return suite; } -} \ No newline at end of file +} diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreSessionTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreSessionTest.cs index 54e0414c23f47..8ab3b25c65b94 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreSessionTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreSessionTest.cs @@ -74,7 +74,7 @@ public void AfterTests() { Ignition.StopAll(true); } - + /// /// Test basic session API. /// @@ -97,13 +97,8 @@ public void TestSession() tx.Rollback(); } - Assert.AreEqual(1, _dumps.Count); - var ops = _dumps.First(); - Assert.AreEqual(1, ops.Count); - - Assert.AreEqual(1, ops.Count(op => op.Type == OperationType.SesEnd && !op.Commit)); - - _dumps = new ConcurrentBag>(); + // SessionEnd should not be called. + Assert.AreEqual(0, _dumps.Count); // 2. Test puts. using (var tx = ignite.GetTransactions().TxStart()) @@ -209,7 +204,7 @@ private ICollection GetOperations() } return (ICollection) ops; - } + } } /// @@ -245,7 +240,7 @@ public Operation(string cacheName, OperationType type, int key, int val) : this( /// Cache name. /// public string CacheName { get; private set; } - + /// /// Operation type. /// From 5372606f28a97de169691ba34f354b106048b6b1 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Fri, 10 Nov 2017 17:33:52 +0300 Subject: [PATCH 339/357] Backport of IGNITE-6649: Added eviction policy factory to cache configuration. Signed-off-by: Andrey Gura (cherry picked from commit 6579e69) --- .../org/apache/ignite/cache/CacheMode.java | 2 +- .../AbstractEvictionPolicyFactory.java | 104 ++ .../fifo/FifoEvictionPolicyFactory.java | 72 ++ .../lru/LruEvictionPolicyFactory.java | 72 ++ .../sorted/SortedEvictionPolicyFactory.java | 98 ++ .../configuration/CacheConfiguration.java | 42 +- .../configuration/NearCacheConfiguration.java | 41 + .../processors/cache/GridCacheAttributes.java | 19 + .../cache/GridCacheEvictionManager.java | 15 +- .../processors/cache/GridCacheProcessor.java | 31 +- .../processors/cache/GridCacheUtils.java | 1 + .../processors/igfs/IgfsHelperImpl.java | 8 +- .../internal/processors/igfs/IgfsImpl.java | 4 +- ...CacheConfigurationConsistencySelfTest.java | 50 + .../GridCacheNearEvictionEventSelfTest.java | 5 - .../EvictionPolicyFactoryAbstractTest.java | 1074 +++++++++++++++++ .../FifoEvictionPolicyFactorySelfTest.java | 261 ++++ .../lru/LruEvictionPolicyFactorySelfTest.java | 352 ++++++ .../SortedEvictionPolicyFactorySelfTest.java | 264 ++++ .../IgniteCacheEvictionSelfTestSuite.java | 6 + .../apache/ignite/yardstick/IgniteNode.java | 6 +- 21 files changed, 2506 insertions(+), 21 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/cache/eviction/AbstractEvictionPolicyFactory.java create mode 100644 modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicyFactory.java create mode 100644 modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicyFactory.java create mode 100644 modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicyFactory.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/EvictionPolicyFactoryAbstractTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/fifo/FifoEvictionPolicyFactorySelfTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/LruEvictionPolicyFactorySelfTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/SortedEvictionPolicyFactorySelfTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/cache/CacheMode.java b/modules/core/src/main/java/org/apache/ignite/cache/CacheMode.java index e75fa0c9c3b24..4171b1ac28074 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/CacheMode.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheMode.java @@ -55,7 +55,7 @@ public enum CacheMode { *

    * Note that partitioned cache is always fronted by local * {@code 'near'} cache which stores most recent data. You - * can configure the size of near cache via {@link NearCacheConfiguration#getNearEvictionPolicy()} + * can configure the size of near cache via {@link NearCacheConfiguration#getNearEvictionPolicyFactory()} * configuration property. */ PARTITIONED; diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/AbstractEvictionPolicyFactory.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/AbstractEvictionPolicyFactory.java new file mode 100644 index 0000000000000..012c7ee522ea9 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/AbstractEvictionPolicyFactory.java @@ -0,0 +1,104 @@ +/* + * 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.eviction; + +import javax.cache.configuration.Factory; +import org.apache.ignite.internal.util.typedef.internal.A; + +/** + * Common functionality implementation for eviction policies factories. + */ +public abstract class AbstractEvictionPolicyFactory implements Factory { + /** */ + private int maxSize; + + /** */ + private int batchSize = 1; + + /** */ + private long maxMemSize; + + /** + * Sets maximum allowed size of cache before entry will start getting evicted. + * + * @param max Maximum allowed size of cache before entry will start getting evicted. + * @return {@code this} for chaining. + */ + public AbstractEvictionPolicyFactory setMaxSize(int max) { + A.ensure(max >= 0, "max >= 0"); + + this.maxSize = max; + + return this; + } + + /** + * Gets maximum allowed size of cache before entry will start getting evicted. + * + * @return Maximum allowed size of cache before entry will start getting evicted. + */ + public int getMaxSize() { + return maxSize; + } + + /** + * Sets batch size. + * + * @param batchSize Batch size. + * @return {@code this} for chaining. + */ + public AbstractEvictionPolicyFactory setBatchSize(int batchSize) { + A.ensure(batchSize > 0, "batchSize > 0"); + + this.batchSize = batchSize; + + return this; + } + + /** + * Gets batch size. + * + * @return batch size. + */ + public int getBatchSize() { + return batchSize; + } + + /** + * Sets maximum allowed cache size in bytes. + * + * @return {@code this} for chaining. + */ + public AbstractEvictionPolicyFactory setMaxMemorySize(long maxMemSize) { + A.ensure(maxMemSize >= 0, "maxMemSize >= 0"); + + this.maxMemSize = maxMemSize; + + return this; + } + + /** + * Gets maximum allowed cache size in bytes. + * + * @return maximum allowed cache size in bytes. + */ + public long getMaxMemorySize() { + return maxMemSize; + } + +} diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicyFactory.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicyFactory.java new file mode 100644 index 0000000000000..856865af6aa3b --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicyFactory.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.eviction.fifo; + +import org.apache.ignite.cache.eviction.AbstractEvictionPolicyFactory; + +/** + * Factory class for {@link FifoEvictionPolicy}. + * + * Creates cache Eviction policy based on {@code First In First Out (FIFO)} algorithm and supports batch eviction. + *

    + * The eviction starts in the following cases: + *

      + *
    • The cache size becomes {@code batchSize} elements greater than the maximum size.
    • + *
    • + * The size of cache entries in bytes becomes greater than the maximum memory size. + * The size of cache entry calculates as sum of key size and value size. + *
    • + *
    + * Note:Batch eviction is enabled only if maximum memory limit isn't set ({@code maxMemSize == 0}). + * {@code batchSize} elements will be evicted in this case. The default {@code batchSize} value is {@code 1}. + *

    + * {@link FifoEvictionPolicy} implementation is very efficient since it does not create any additional + * table-like data structures. The {@code FIFO} ordering information is + * maintained by attaching ordering metadata to cache entries. + */ +public class FifoEvictionPolicyFactory extends AbstractEvictionPolicyFactory> { + /** */ + private static final long serialVersionUID = 0L; + + /** Constructor. */ + public FifoEvictionPolicyFactory() { + } + + /** Constructor. */ + public FifoEvictionPolicyFactory(int maxSize) { + setMaxSize(maxSize); + } + + /** */ + public FifoEvictionPolicyFactory(int maxSize, int batchSize, long maxMemSize) { + setMaxSize(maxSize); + setBatchSize(batchSize); + setMaxMemorySize(maxMemSize); + } + + /** {@inheritDoc} */ + @Override public FifoEvictionPolicy create() { + FifoEvictionPolicy policy = new FifoEvictionPolicy<>(); + + policy.setBatchSize(getBatchSize()); + policy.setMaxMemorySize(getMaxMemorySize()); + policy.setMaxSize(getMaxSize()); + + return policy; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicyFactory.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicyFactory.java new file mode 100644 index 0000000000000..8f7fbc5c7b95d --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicyFactory.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.eviction.lru; + +import org.apache.ignite.cache.eviction.AbstractEvictionPolicyFactory; + +/** + * Factory class for {@link LruEvictionPolicy}. + * + * Creates cache Eviction policy based on {@code Least Recently Used (LRU)} algorithm and supports batch eviction. + *

    + * The eviction starts in the following cases: + *

      + *
    • The cache size becomes {@code batchSize} elements greater than the maximum size.
    • + *
    • + * The size of cache entries in bytes becomes greater than the maximum memory size. + * The size of cache entry calculates as sum of key size and value size. + *
    • + *
    + * Note:Batch eviction is enabled only if maximum memory limit isn't set ({@code maxMemSize == 0}). + * {@code batchSize} elements will be evicted in this case. The default {@code batchSize} value is {@code 1}. + + * {@link LruEvictionPolicy} implementation is very efficient since it is lock-free and does not create any additional table-like + * data structures. The {@code LRU} ordering information is maintained by attaching ordering metadata to cache entries. + */ +public class LruEvictionPolicyFactory extends AbstractEvictionPolicyFactory> { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + public LruEvictionPolicyFactory() { + } + + /** */ + public LruEvictionPolicyFactory(int maxSize) { + setMaxSize(maxSize); + } + + /** */ + public LruEvictionPolicyFactory(int maxSize, int batchSize, long maxMemSize) { + setMaxSize(maxSize); + setBatchSize(batchSize); + setMaxMemorySize(maxMemSize); + } + + /** {@inheritDoc} */ + @Override public LruEvictionPolicy create() { + LruEvictionPolicy policy = new LruEvictionPolicy<>(); + + policy.setBatchSize(getBatchSize()); + policy.setMaxMemorySize(getMaxMemorySize()); + policy.setMaxSize(getMaxSize()); + + return policy; + } + +} diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicyFactory.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicyFactory.java new file mode 100644 index 0000000000000..a88c277485eb7 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicyFactory.java @@ -0,0 +1,98 @@ +/* + * 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.eviction.sorted; + +import java.io.Serializable; +import java.util.Comparator; +import org.apache.ignite.cache.eviction.AbstractEvictionPolicyFactory; +import org.apache.ignite.cache.eviction.EvictableEntry; + +/** + * Factory class for {@link SortedEvictionPolicy}. + * + * Creates cache Eviction policy which will select the minimum cache entry for eviction. + *

    + * The eviction starts in the following cases: + *

      + *
    • The cache size becomes {@code batchSize} elements greater than the maximum size.
    • + *
    • + * The size of cache entries in bytes becomes greater than the maximum memory size. + * The size of cache entry calculates as sum of key size and value size. + *
    • + *
    + * Note:Batch eviction is enabled only if maximum memory limit isn't set ({@code maxMemSize == 0}). + * {@code batchSize} elements will be evicted in this case. The default {@code batchSize} value is {@code 1}. + *

    + * Entries comparison based on {@link Comparator} instance if provided. + * Default {@code Comparator} behaviour is use cache entries keys for comparison that imposes a requirement for keys + * to implement {@link Comparable} interface. + *

    + * User defined comparator should implement {@link Serializable} interface. + */ +public class SortedEvictionPolicyFactory extends AbstractEvictionPolicyFactory> { + /** */ + private static final long serialVersionUID = 0L; + + /** Comparator. */ + private Comparator> comp; + + /** */ + public SortedEvictionPolicyFactory() { + } + + /** */ + public SortedEvictionPolicyFactory(int maxSize) { + setMaxSize(maxSize); + } + + /** */ + public SortedEvictionPolicyFactory(int maxSize, int batchSize, long maxMemSize) { + setMaxSize(maxSize); + setBatchSize(batchSize); + setMaxMemorySize(maxMemSize); + } + + /** + * Gets entries comparator. + * @return entry comparator. + */ + public Comparator> getComp() { + return comp; + } + + /** + * Sets entries comparator. + * + * @param comp entry comparator. + */ + public void setComp(Comparator> comp) { + this.comp = comp; + } + + /** {@inheritDoc} */ + @Override public SortedEvictionPolicy create() { + SortedEvictionPolicy policy = new SortedEvictionPolicy<>(comp); + + policy.setBatchSize(getBatchSize()); + policy.setMaxMemorySize(getMaxMemorySize()); + policy.setMaxSize(getMaxSize()); + + return policy; + } + +} diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java index 31806e73e69e7..cf6cd4e1ba28b 100644 --- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java @@ -242,9 +242,13 @@ public class CacheConfiguration extends MutableConfiguration { /** Rebalance timeout. */ private long rebalanceTimeout = DFLT_REBALANCE_TIMEOUT; - /** Cache expiration policy. */ + /** Cache eviction policy. */ + @Deprecated private EvictionPolicy evictPlc; + /** Cache eviction policy factory. */ + private Factory evictPlcFactory; + /** Flag indicating whether eviction is synchronized. */ private boolean evictSync = DFLT_EVICT_SYNCHRONIZED; @@ -466,6 +470,7 @@ public CacheConfiguration(CompleteConfiguration cfg) { evictKeyBufSize = cc.getEvictSynchronizedKeyBufferSize(); evictMaxOverflowRatio = cc.getEvictMaxOverflowRatio(); evictPlc = cc.getEvictionPolicy(); + evictPlcFactory = cc.getEvictionPolicyFactory(); evictSync = cc.isEvictSynchronized(); evictSyncConcurrencyLvl = cc.getEvictSynchronizedConcurrencyLevel(); evictSyncTimeout = cc.getEvictSynchronizedTimeout(); @@ -549,7 +554,10 @@ public CacheConfiguration setName(String name) { * which means that evictions are disabled for cache. * * @return Cache eviction policy or {@code null} if evictions should be disabled. + * + * @deprecated Use {@link #getEvictionPolicyFactory()} instead. */ + @Deprecated @SuppressWarnings({"unchecked"}) @Nullable public EvictionPolicy getEvictionPolicy() { return evictPlc; @@ -558,15 +566,43 @@ public CacheConfiguration setName(String name) { /** * Sets cache eviction policy. * - * @param evictPlc Cache expiration policy. + * @param evictPlc Cache eviction policy. * @return {@code this} for chaining. + * + * @deprecated Use {@link #setEvictionPolicyFactory(Factory)} instead. */ + @Deprecated public CacheConfiguration setEvictionPolicy(@Nullable EvictionPolicy evictPlc) { this.evictPlc = evictPlc; return this; } + /** + * Gets cache eviction policy factory. By default, returns {@code null} + * which means that evictions are disabled for cache. + * + * @return Cache eviction policy factory or {@code null} if evictions should be disabled + * or if {@link #getEvictionPolicy()} should be used instead. + */ + @Nullable public Factory> getEvictionPolicyFactory() { + return evictPlcFactory; + } + + /** + * Sets cache eviction policy factory. + * Note: Eviction policy factory should be {@link Serializable}. + * + * @param evictPlcFactory Cache eviction policy factory. + * @return {@code this} for chaining. + */ + public CacheConfiguration setEvictionPolicyFactory( + @Nullable Factory> evictPlcFactory) { + this.evictPlcFactory = evictPlcFactory; + + return this; + } + /** * @return Near enabled flag. */ @@ -786,7 +822,7 @@ public CacheConfiguration setEvictMaxOverflowRatio(float evictMaxOverflowR * never be evicted. *

    * If not provided, any entry may be evicted depending on - * {@link #getEvictionPolicy() eviction policy} configuration. + * {@link #getEvictionPolicyFactory()} eviction policy} configuration. * * @return Eviction filter or {@code null}. */ diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/NearCacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/NearCacheConfiguration.java index 3c59bd6d7db07..e9c71f941ff73 100644 --- a/modules/core/src/main/java/org/apache/ignite/configuration/NearCacheConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/configuration/NearCacheConfiguration.java @@ -17,10 +17,13 @@ package org.apache.ignite.configuration; +import java.io.Serializable; import javax.cache.configuration.MutableConfiguration; +import javax.cache.configuration.Factory; import org.apache.ignite.cache.CacheMode; import org.apache.ignite.cache.eviction.EvictionPolicy; import org.apache.ignite.internal.util.typedef.internal.S; +import org.jetbrains.annotations.Nullable; import static org.apache.ignite.configuration.CacheConfiguration.DFLT_NEAR_START_SIZE; @@ -37,8 +40,12 @@ public class NearCacheConfiguration extends MutableConfiguration { private static final long serialVersionUID = 0L; /** Near cache eviction policy. */ + @Deprecated private EvictionPolicy nearEvictPlc; + /** Near cache eviction policy factory. */ + private Factory nearEvictPlcFactory; + /** Default near cache start size. */ private int nearStartSize = DFLT_NEAR_START_SIZE; @@ -57,6 +64,7 @@ public NearCacheConfiguration() { public NearCacheConfiguration(NearCacheConfiguration ccfg) { super(ccfg); + nearEvictPlcFactory = ccfg.getNearEvictionPolicyFactory(); nearEvictPlc = ccfg.getNearEvictionPolicy(); nearStartSize = ccfg.getNearStartSize(); } @@ -68,7 +76,10 @@ public NearCacheConfiguration(NearCacheConfiguration ccfg) { * @return Near eviction policy. * @see CacheConfiguration#getEvictionPolicy() * @see CacheConfiguration#isEvictSynchronized() + * + * @deprecated Use {@link #getNearEvictionPolicyFactory()} instead. */ + @Deprecated public EvictionPolicy getNearEvictionPolicy() { return nearEvictPlc; } @@ -78,13 +89,43 @@ public EvictionPolicy getNearEvictionPolicy() { * * @param nearEvictPlc Near eviction policy. * @return {@code this} for chaining. + * + * @deprecated Use {@link #setNearEvictionPolicyFactory(Factory)} instead. */ + @Deprecated public NearCacheConfiguration setNearEvictionPolicy(EvictionPolicy nearEvictPlc) { this.nearEvictPlc = nearEvictPlc; return this; } + /** + * Gets cache eviction policy factory. By default, returns {@code null} + * which means that evictions are disabled for cache. + * + * @return Cache eviction policy or {@code null} if evictions should be disabled. + */ + @Nullable public Factory> getNearEvictionPolicyFactory() { + return nearEvictPlcFactory; + } + + /** + * Sets cache eviction policy factory. + * Note: Eviction policy factory should be {@link Serializable}. + * + * @see CacheConfiguration#getEvictionPolicyFactory() + * @see CacheConfiguration#isEvictSynchronized() + * + * @param nearEvictPlcFactory Cache eviction policy. + * @return {@code this} for chaining. + */ + public NearCacheConfiguration setNearEvictionPolicyFactory( + @Nullable Factory> nearEvictPlcFactory) { + this.nearEvictPlcFactory = nearEvictPlcFactory; + + return this; + } + /** * Gets initial cache size for near cache which will be used to pre-create internal * hash table after start. Default value is defined by {@link CacheConfiguration#DFLT_NEAR_START_SIZE}. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java index 57f12f635489b..568886b0dfde4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java @@ -170,13 +170,25 @@ public String evictionFilterClassName() { /** * @return Eviction policy class name. + * + * @deprecated Use evictionPolicyFactoryClassName() instead. */ + @Deprecated public String evictionPolicyClassName() { return className(ccfg.getEvictionPolicy()); } + /** + * @return Eviction policy factory class name. + */ + public String evictionPolicyFactoryClassName() { + return className(ccfg.getEvictionPolicyFactory()); + } + /** * @return Near eviction policy class name. + * + * @deprecated Use nearEvictionPolicyFactoryClassName() instead. */ public String nearEvictionPolicyClassName() { NearCacheConfiguration nearCfg = ccfg.getNearConfiguration(); @@ -187,6 +199,13 @@ public String nearEvictionPolicyClassName() { return className(nearCfg.getNearEvictionPolicy()); } + /** + * @return Near eviction policy factory class name. + */ + public String nearEvictionPolicyFactoryClassName() { + return className(ccfg.getEvictionPolicyFactory()); + } + /** * @return Store class name. */ 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 f8722d6a4f338..ed2bd6ff3f548 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 @@ -170,7 +170,15 @@ public class GridCacheEvictionManager extends GridCacheManagerAdapter { @Override public void start0() throws IgniteCheckedException { CacheConfiguration cfg = cctx.config(); - plc = cctx.isNear() ? cfg.getNearConfiguration().getNearEvictionPolicy() : cfg.getEvictionPolicy(); + if (cctx.isNear()) { + plc = (cfg.getNearConfiguration().getNearEvictionPolicyFactory() != null) ? + (EvictionPolicy)cfg.getNearConfiguration().getNearEvictionPolicyFactory().create() : + cfg.getNearConfiguration().getNearEvictionPolicy(); + } + else if (cfg.getEvictionPolicyFactory() != null) + plc = (EvictionPolicy)cfg.getEvictionPolicyFactory().create(); + else + plc = cfg.getEvictionPolicy(); memoryMode = cctx.config().getMemoryMode(); @@ -2119,4 +2127,9 @@ Collection> evictedReaders(KeyCacheObject key) return S.toString(EvictionFuture.class, this); } } + + /** For test purposes. */ + public EvictionPolicy getEvictionPolicy() { + return plc; + } } 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 8a041b2a1a4ab..e9065603ed5ff 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 @@ -356,7 +356,7 @@ private void suggestOptimizations(CacheConfiguration cfg, boolean hasStore) { String msg = "Disable eviction policy (remove from configuration)"; - if (cfg.getEvictionPolicy() != null) { + if (cfg.getEvictionPolicyFactory() != null || cfg.getEvictionPolicy() != null) { perf.add(msg, false); perf.add("Disable synchronized evictions (set 'evictSynchronized' to false)", !cfg.isEvictSynchronized()); @@ -482,7 +482,8 @@ else if (cc.getRebalanceMode() == SYNC) { switch (cc.getMemoryMode()) { case OFFHEAP_VALUES: { - if (cacheType.userCache() && cc.getEvictionPolicy() == null && cc.getOffHeapMaxMemory() >= 0) + if (cacheType.userCache() && cc.getEvictionPolicyFactory() == null + && cc.getEvictionPolicy() == null && cc.getOffHeapMaxMemory() >= 0) U.quietAndWarn(log, "Off heap maximum memory configuration property will be ignored for the " + "cache working in OFFHEAP_VALUES mode (memory usage will be unlimited): " + U.maskName(cc.getName()) + ". Consider configuring eviction policy or switching to " + @@ -502,7 +503,8 @@ else if (cc.getRebalanceMode() == SYNC) { } case ONHEAP_TIERED: - if (cacheType.userCache() && cc.getEvictionPolicy() == null && cc.getOffHeapMaxMemory() >= 0) + if (cacheType.userCache() && cc.getEvictionPolicyFactory() == null + && cc.getEvictionPolicy() == null && cc.getOffHeapMaxMemory() >= 0) U.quietAndWarn(log, "Eviction policy not enabled with ONHEAP_TIERED mode for cache " + "(entries will not be moved to off-heap store): " + U.maskName(cc.getName())); @@ -551,6 +553,7 @@ private Collection dhtExcludes(GridCacheContext ctx) { * @throws IgniteCheckedException If failed to inject. */ private void prepare(CacheConfiguration cfg, Collection objs) throws IgniteCheckedException { + prepare(cfg, cfg.getEvictionPolicyFactory(), false); prepare(cfg, cfg.getEvictionPolicy(), false); prepare(cfg, cfg.getAffinity(), false); prepare(cfg, cfg.getAffinityMapper(), false); @@ -560,8 +563,10 @@ private void prepare(CacheConfiguration cfg, Collection objs) throws Ign NearCacheConfiguration nearCfg = cfg.getNearConfiguration(); - if (nearCfg != null) + if (nearCfg != null) { + prepare(cfg, nearCfg.getNearEvictionPolicyFactory(), true); prepare(cfg, nearCfg.getNearEvictionPolicy(), true); + } for (Object obj : objs) prepare(cfg, obj, false); @@ -589,6 +594,7 @@ private void prepare(CacheConfiguration cfg, @Nullable Object rsrc, boolean near private void cleanup(GridCacheContext cctx) { CacheConfiguration cfg = cctx.config(); + cleanup(cfg, cfg.getEvictionPolicyFactory(), false); cleanup(cfg, cfg.getEvictionPolicy(), false); cleanup(cfg, cfg.getAffinity(), false); cleanup(cfg, cfg.getAffinityMapper(), false); @@ -604,8 +610,10 @@ private void cleanup(GridCacheContext cctx) { NearCacheConfiguration nearCfg = cfg.getNearConfiguration(); - if (nearCfg != null) + if (nearCfg != null) { + cleanup(cfg, nearCfg.getNearEvictionPolicyFactory(), true); cleanup(cfg, nearCfg.getNearEvictionPolicy(), true); + } cctx.cleanup(); } @@ -2983,6 +2991,10 @@ private void checkCache(CacheConfiguration locCfg, CacheConfiguration rmtCfg, Cl CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "evictionPolicy", "Eviction policy", locAttr.evictionPolicyClassName(), rmtAttr.evictionPolicyClassName(), true); + CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "evictionPolicyFactory", + "Eviction policy factory", locAttr.evictionPolicyFactoryClassName(), + rmtAttr.evictionPolicyFactoryClassName(), true); + CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "transactionManagerLookup", "Transaction manager lookup", locAttr.transactionManagerLookupClassName(), rmtAttr.transactionManagerLookupClassName(), false); @@ -3036,6 +3048,10 @@ private void checkCache(CacheConfiguration locCfg, CacheConfiguration rmtCfg, Cl "Near eviction policy", locAttr.nearEvictionPolicyClassName(), rmtAttr.nearEvictionPolicyClassName(), false); + CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "nearEvictionPolicyFactory", + "Near eviction policy factory", locAttr.nearEvictionPolicyFactoryClassName(), + rmtAttr.nearEvictionPolicyFactoryClassName(), false); + CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "affinityIncludeNeighbors", "Affinity include neighbors", locAttr.affinityIncludeNeighbors(), rmtAttr.affinityIncludeNeighbors(), true); @@ -3671,14 +3687,17 @@ private Iterable lifecycleAwares(CacheConfiguration ccfg, Object... objs ret.add(ccfg.getAffinity()); ret.add(ccfg.getAffinityMapper()); ret.add(ccfg.getEvictionFilter()); + ret.add(ccfg.getEvictionPolicyFactory()); ret.add(ccfg.getEvictionPolicy()); ret.add(ccfg.getInterceptor()); ret.add(ccfg.getTopologyValidator()); NearCacheConfiguration nearCfg = ccfg.getNearConfiguration(); - if (nearCfg != null) + if (nearCfg != null) { + ret.add(nearCfg.getNearEvictionPolicyFactory()); ret.add(nearCfg.getNearEvictionPolicy()); + } Collections.addAll(ret, objs); 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 e274485ec45f6..2a976ecbe230c 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 @@ -1145,6 +1145,7 @@ public static CacheConfiguration hadoopSystemCache() { cache.setAtomicityMode(TRANSACTIONAL); cache.setWriteSynchronizationMode(FULL_SYNC); + cache.setEvictionPolicyFactory(null); cache.setEvictionPolicy(null); cache.setSwapEnabled(false); cache.setCacheStoreFactory(null); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsHelperImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsHelperImpl.java index 29e75a5a122d2..f20b787aa7c17 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsHelperImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsHelperImpl.java @@ -31,7 +31,9 @@ public class IgfsHelperImpl implements IgfsHelper { /** {@inheritDoc} */ @Override public void preProcessCacheConfiguration(CacheConfiguration cfg) { - EvictionPolicy evictPlc = cfg.getEvictionPolicy(); + EvictionPolicy evictPlc = cfg.getEvictionPolicyFactory() != null ? + (EvictionPolicy)cfg.getEvictionPolicyFactory().create() + : cfg.getEvictionPolicy(); if (evictPlc instanceof IgfsPerBlockLruEvictionPolicy && cfg.getEvictionFilter() == null) cfg.setEvictionFilter(new IgfsEvictionFilter()); @@ -39,7 +41,9 @@ public class IgfsHelperImpl implements IgfsHelper { /** {@inheritDoc} */ @Override public void validateCacheConfiguration(CacheConfiguration cfg) throws IgniteCheckedException { - EvictionPolicy evictPlc = cfg.getEvictionPolicy(); + EvictionPolicy evictPlc = cfg.getEvictionPolicyFactory() != null ? + (EvictionPolicy)cfg.getEvictionPolicyFactory().create() + : cfg.getEvictionPolicy(); if (evictPlc != null && evictPlc instanceof IgfsPerBlockLruEvictionPolicy) { EvictionFilter evictFilter = cfg.getEvictionFilter(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java index 6b23e8084a46b..98313bad4dbbe 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java @@ -263,7 +263,9 @@ public final class IgfsImpl implements IgfsEx { for (CacheConfiguration cacheCfg : igfsCtx.kernalContext().config().getCacheConfiguration()) { if (F.eq(dataCacheName, cacheCfg.getName())) { - EvictionPolicy evictPlc = cacheCfg.getEvictionPolicy(); + EvictionPolicy evictPlc = cacheCfg.getEvictionPolicyFactory() != null ? + (EvictionPolicy)cacheCfg.getEvictionPolicyFactory().create() + : cacheCfg.getEvictionPolicy(); if (evictPlc != null & evictPlc instanceof IgfsPerBlockLruEvictionPolicy) this.evictPlc = (IgfsPerBlockLruEvictionPolicy)evictPlc; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationConsistencySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationConsistencySelfTest.java index a1f917fccb8a4..07072f7b1c5e3 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationConsistencySelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationConsistencySelfTest.java @@ -30,8 +30,11 @@ import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; import org.apache.ignite.cache.eviction.EvictionFilter; import org.apache.ignite.cache.eviction.fifo.FifoEvictionPolicy; +import org.apache.ignite.cache.eviction.fifo.FifoEvictionPolicyFactory; import org.apache.ignite.cache.eviction.lru.LruEvictionPolicy; +import org.apache.ignite.cache.eviction.lru.LruEvictionPolicyFactory; import org.apache.ignite.cache.eviction.random.RandomEvictionPolicy; +import org.apache.ignite.cache.eviction.sorted.SortedEvictionPolicyFactory; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.DeploymentMode; @@ -374,6 +377,27 @@ public void testDifferentEvictionEnabled() throws Exception { ); } + /** + * @throws Exception If failed. + */ + public void testDifferentEvictionPolicyEnabled() throws Exception { + checkSecondGridStartFails( + new C1() { + /** {@inheritDoc} */ + @Override public Void apply(CacheConfiguration cfg) { + cfg.setEvictionPolicyFactory(new FifoEvictionPolicyFactory<>()); + return null; + } + }, + new C1() { + /** {@inheritDoc} */ + @Override public Void apply(CacheConfiguration cfg) { + return null; + } + } + ); + } + /** * @throws Exception If failed. */ @@ -396,6 +420,28 @@ public void testDifferentEvictionPolicies() throws Exception { ); } + /** + * @throws Exception If failed. + */ + public void testDifferentEvictionPolicyFactories() throws Exception { + checkSecondGridStartFails( + new C1() { + /** {@inheritDoc} */ + @Override public Void apply(CacheConfiguration cfg) { + cfg.setEvictionPolicyFactory(new SortedEvictionPolicyFactory()); + return null; + } + }, + new C1() { + /** {@inheritDoc} */ + @Override public Void apply(CacheConfiguration cfg) { + cfg.setEvictionPolicyFactory(new FifoEvictionPolicyFactory<>()); + return null; + } + } + ); + } + /** * @throws Exception If failed. */ @@ -632,6 +678,7 @@ public void testPartitionedOnlyAttributesIgnoredForReplicated() throws Exception @Override public Void apply(CacheConfiguration cfg) { NearCacheConfiguration nearCfg = new NearCacheConfiguration(); + nearCfg.setNearEvictionPolicyFactory(new LruEvictionPolicyFactory<>()); nearCfg.setNearEvictionPolicy(new RandomEvictionPolicy()); cfg.setNearConfiguration(nearCfg); @@ -646,6 +693,7 @@ public void testPartitionedOnlyAttributesIgnoredForReplicated() throws Exception @Override public Void apply(CacheConfiguration cfg) { NearCacheConfiguration nearCfg = new NearCacheConfiguration(); + nearCfg.setNearEvictionPolicyFactory(new FifoEvictionPolicyFactory<>()); nearCfg.setNearEvictionPolicy(new FifoEvictionPolicy()); cfg.setNearConfiguration(nearCfg); @@ -671,6 +719,7 @@ public void testIgnoreMismatchForLocalCaches() throws Exception { @Override public Void apply(CacheConfiguration cfg) { cfg.setAffinity(new TestRendezvousAffinityFunction()); + cfg.setEvictionPolicyFactory(new FifoEvictionPolicyFactory<>()); cfg.setEvictionPolicy(new FifoEvictionPolicy()); cfg.setCacheStoreFactory(new IgniteCacheAbstractTest.TestStoreFactory()); @@ -689,6 +738,7 @@ public void testIgnoreMismatchForLocalCaches() throws Exception { @Override public Void apply(CacheConfiguration cfg) { cfg.setAffinity(new RendezvousAffinityFunction()); + cfg.setEvictionPolicyFactory(new FifoEvictionPolicyFactory<>()); cfg.setEvictionPolicy(new LruEvictionPolicy()); cfg.setCacheStoreFactory(null); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearEvictionEventSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearEvictionEventSelfTest.java index 7088ad7c43fc8..0d36a5a7fde6b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearEvictionEventSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearEvictionEventSelfTest.java @@ -36,9 +36,4 @@ public class GridCacheNearEvictionEventSelfTest extends GridCacheEvictionEventAb @Override protected CacheAtomicityMode atomicityMode() { return TRANSACTIONAL; } - - /** {@inheritDoc} */ - @Override public void testEvictionEvent() throws Exception { - super.testEvictionEvent(); - } } \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/EvictionPolicyFactoryAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/EvictionPolicyFactoryAbstractTest.java new file mode 100644 index 0000000000000..89d7f0b42ef6d --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/EvictionPolicyFactoryAbstractTest.java @@ -0,0 +1,1074 @@ +/* + * 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.eviction; + +import java.lang.reflect.InvocationTargetException; +import java.util.Collection; +import java.util.List; +import java.util.Random; +import java.util.concurrent.Callable; +import java.util.concurrent.atomic.AtomicInteger; +import javax.cache.Cache; +import javax.cache.configuration.Factory; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.eviction.EvictableEntry; +import org.apache.ignite.cache.eviction.EvictionFilter; +import org.apache.ignite.cache.eviction.EvictionPolicy; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.configuration.NearCacheConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.processors.cache.GridCacheEvictionManager; +import org.apache.ignite.internal.processors.cache.distributed.dht.colocated.GridDhtColocatedCache; +import org.apache.ignite.internal.util.typedef.C2; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.S; +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.common.GridCommonAbstractTest; +import org.apache.ignite.transactions.Transaction; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; +import static org.apache.ignite.cache.CacheMode.LOCAL; +import static org.apache.ignite.cache.CacheMode.PARTITIONED; +import static org.apache.ignite.cache.CacheMode.REPLICATED; +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_ASYNC; +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; +import static org.apache.ignite.events.EventType.EVT_JOB_MAPPED; +import static org.apache.ignite.events.EventType.EVT_TASK_FAILED; +import static org.apache.ignite.events.EventType.EVT_TASK_FINISHED; +import static org.apache.ignite.internal.processors.cache.eviction.EvictionPolicyFactoryAbstractTest.EvictionPolicyProxy.proxy; +import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; +import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ; + +/** + * Base class for eviction tests. + */ +public abstract class EvictionPolicyFactoryAbstractTest> + extends GridCommonAbstractTest { + /** IP finder. */ + protected static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); + + /** */ + private static final String DEFAULT_CACHE_NAME = null; + + /** Put entry size. */ + protected static final int PUT_ENTRY_SIZE = 10; + + /** Replicated cache. */ + protected CacheMode mode = REPLICATED; + + /** Near enabled flag. */ + protected boolean nearEnabled; + + /** Policy max. */ + protected int plcMax = 10; + + /** Policy batch size. */ + protected int plcBatchSize = 1; + + /** Policy max memory size. */ + protected long plcMaxMemSize = 0; + + protected Factory policyFactory; + + /** Near policy max. */ + protected int nearMax = 3; + + /** Synchronous commit. */ + protected boolean syncCommit; + + /** */ + protected int gridCnt = 2; + + /** */ + protected EvictionFilter filter; + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + super.afterTest(); + + policyFactory = null; + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + assert policyFactory != null; + + IgniteConfiguration c = super.getConfiguration(igniteInstanceName); + + CacheConfiguration cc = defaultCacheConfiguration(); + + cc.setCacheMode(mode); + cc.setEvictionPolicyFactory(policyFactory); + cc.setWriteSynchronizationMode(syncCommit ? FULL_SYNC : FULL_ASYNC); + cc.setAtomicityMode(TRANSACTIONAL); + + if (nearEnabled) { + NearCacheConfiguration nearCfg = new NearCacheConfiguration(); + + nearCfg.setNearEvictionPolicyFactory(createNearPolicyFactory(nearMax)); + + cc.setNearConfiguration(nearCfg); + } + else + cc.setNearConfiguration(null); + + if (mode == PARTITIONED) + cc.setBackups(1); + + if (filter != null) + cc.setEvictionFilter(filter); + + c.setCacheConfiguration(cc); + + TcpDiscoverySpi disco = new TcpDiscoverySpi(); + + disco.setIpFinder(ipFinder); + + c.setDiscoverySpi(disco); + + c.setIncludeEventTypes(EVT_TASK_FAILED, EVT_TASK_FINISHED, EVT_JOB_MAPPED); + + c.setIncludeProperties(); + + return c; + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + filter = null; + + super.afterTestsStopped(); + } + + /** + * @throws Exception If failed. + */ + public void testMaxSizePolicy() throws Exception { + plcMax = 3; + plcMaxMemSize = 0; + plcBatchSize = 1; + + doTestPolicy(); + } + + /** + * @throws Exception If failed. + */ + public void testMaxSizePolicyWithBatch() throws Exception { + plcMax = 3; + plcMaxMemSize = 0; + plcBatchSize = 2; + + doTestPolicyWithBatch(); + } + + /** + * @throws Exception If failed. + */ + public void testMaxMemSizePolicy() throws Exception { + plcMax = 0; + plcMaxMemSize = 3 * MockEntry.ENTRY_SIZE; + plcBatchSize = 1; + + doTestPolicy(); + } + + /** + * Batch ignored when {@code maxSize > 0} and {@code maxMemSize > 0}. + * + * @throws Exception If failed. + */ + public void testMaxMemSizePolicyWithBatch() throws Exception { + plcMax = 3; + plcMaxMemSize = 10 * MockEntry.ENTRY_SIZE; + plcBatchSize = 2; + + doTestPolicy(); + } + + /** + * @throws Exception If failed. + */ + public void testMaxSizeMemory() throws Exception { + int max = 10; + + plcMax = max; + plcMaxMemSize = 0; + plcBatchSize = 1; + + doTestMemory(max); + } + + /** + * @throws Exception If failed. + */ + public void testMaxSizeMemoryWithBatch() throws Exception { + int max = 10; + + plcMax = max; + plcMaxMemSize = 0; + plcBatchSize = 2; + + doTestMemory(max); + } + + /** + * @throws Exception If failed. + */ + public void testMaxMemSizeMemory() throws Exception { + int max = 10; + + plcMax = 0; + plcMaxMemSize = max * MockEntry.ENTRY_SIZE; + plcBatchSize = 1; + + doTestMemory(max); + } + + /** + * @throws Exception If failed. + */ + public void testMaxSizeRandom() throws Exception { + plcMax = 10; + plcMaxMemSize = 0; + plcBatchSize = 1; + + doTestRandom(); + } + + /** + * @throws Exception If failed. + */ + public void testMaxSizeRandomWithBatch() throws Exception { + plcMax = 10; + plcMaxMemSize = 0; + plcBatchSize = 2; + + doTestRandom(); + } + + /** + * @throws Exception If failed. + */ + public void testMaxMemSizeRandom() throws Exception { + plcMax = 0; + plcMaxMemSize = 10 * MockEntry.KEY_SIZE; + plcBatchSize = 1; + + doTestRandom(); + } + + /** + * @throws Exception If failed. + */ + public void testMaxSizeAllowEmptyEntries() throws Exception { + plcMax = 10; + plcMaxMemSize = 0; + plcBatchSize = 1; + + doTestAllowEmptyEntries(); + } + + /** + * @throws Exception If failed. + */ + public void testMaxSizeAllowEmptyEntriesWithBatch() throws Exception { + plcMax = 10; + plcMaxMemSize = 0; + plcBatchSize = 2; + + doTestAllowEmptyEntries(); + } + + /** + * @throws Exception If failed. + */ + public void testMaxMemSizeAllowEmptyEntries() throws Exception { + plcMax = 0; + plcMaxMemSize = 10 * MockEntry.KEY_SIZE; + plcBatchSize = 1; + + doTestAllowEmptyEntries(); + } + + /** + * @throws Exception If failed. + */ + public void testMaxSizePut() throws Exception { + plcMax = 100; + plcBatchSize = 1; + plcMaxMemSize = 0; + + doTestPut(plcMax); + } + + /** + * @throws Exception If failed. + */ + public void testMaxSizePutWithBatch() throws Exception { + plcMax = 100; + plcBatchSize = 2; + plcMaxMemSize = 0; + + doTestPut(plcMax); + } + + /** + * @throws Exception If failed. + */ + public void testMaxMemSizePut() throws Exception { + int max = 100; + + plcMax = 0; + plcBatchSize = 2; + plcMaxMemSize = max * PUT_ENTRY_SIZE; + + doTestPut(max); + } + + /** + * Tests policy behaviour. + * + * @throws Exception If failed. + */ + protected abstract void doTestPolicy() throws Exception; + + /** + * Tests policy behaviour with batch enabled. + * + * @throws Exception If failed. + */ + protected abstract void doTestPolicyWithBatch() throws Exception; + + /** + * @throws Exception If failed. + */ + protected void doTestAllowEmptyEntries() throws Exception { + policyFactory = createPolicyFactory(); + + try { + startGrid(); + + MockEntry e1 = new MockEntry("1"); + MockEntry e2 = new MockEntry("2"); + MockEntry e3 = new MockEntry("3"); + MockEntry e4 = new MockEntry("4"); + MockEntry e5 = new MockEntry("5"); + + EvictionPolicyProxy p = proxy(policy()); + + p.onEntryAccessed(false, e1); + + assertFalse(e1.isEvicted()); + + check(p.queue().size(), MockEntry.KEY_SIZE); + + p.onEntryAccessed(false, e2); + + assertFalse(e1.isEvicted()); + assertFalse(e2.isEvicted()); + + check(p.queue().size(), MockEntry.KEY_SIZE); + + p.onEntryAccessed(false, e3); + + assertFalse(e1.isEvicted()); + assertFalse(e3.isEvicted()); + + check(p.queue().size(), MockEntry.KEY_SIZE); + + p.onEntryAccessed(false, e4); + + assertFalse(e1.isEvicted()); + assertFalse(e3.isEvicted()); + assertFalse(e4.isEvicted()); + + check(p.queue().size(), MockEntry.KEY_SIZE); + + p.onEntryAccessed(false, e5); + + assertFalse(e1.isEvicted()); + assertFalse(e3.isEvicted()); + assertFalse(e5.isEvicted()); + + check(p.queue().size(), MockEntry.KEY_SIZE); + } + finally { + stopAllGrids(); + } + } + + /** + * @throws Exception If failed. + */ + protected void doTestMemory(int max) throws Exception { + policyFactory = createPolicyFactory(); + + try { + startGrid(); + + EvictionPolicyProxy p = proxy(policy()); + + int cnt = max + plcBatchSize; + + for (int i = 0; i < cnt; i++) + p.onEntryAccessed(false, new MockEntry(Integer.toString(i), Integer.toString(i))); + + info(p); + + check(max, MockEntry.ENTRY_SIZE); + } + finally { + stopAllGrids(); + } + } + + /** + * @throws Exception If failed. + */ + protected void doTestRandom() throws Exception { + policyFactory = createPolicyFactory(); + + try { + startGrid(); + + EvictionPolicyProxy p = proxy(policy()); + + int max = 10; + + Random rand = new Random(); + + int keys = 31; + + MockEntry[] entries = new MockEntry[keys]; + + for (int i = 0; i < entries.length; i++) + entries[i] = new MockEntry(Integer.toString(i)); + + int runs = 5000000; + + for (int i = 0; i < runs; i++) { + boolean rmv = rand.nextBoolean(); + + int j = rand.nextInt(entries.length); + + MockEntry e = entry(entries, j); + + if (rmv) + entries[j] = new MockEntry(Integer.toString(j)); + + p.onEntryAccessed(rmv, e); + } + + info(p); + + assertTrue(p.getCurrentSize() <= (plcMaxMemSize > 0 ? max : max + plcBatchSize)); + assertTrue(p.getCurrentMemorySize() <= (plcMaxMemSize > 0 ? max : max + plcBatchSize) * MockEntry.KEY_SIZE); + } + finally { + stopAllGrids(); + } + } + + /** + * @throws Exception If failed. + */ + protected void doTestPut(int max) throws Exception { + mode = LOCAL; + syncCommit = true; + + policyFactory = createPolicyFactory(); + + try { + Ignite ignite = startGrid(); + + IgniteCache cache = ignite.cache(DEFAULT_CACHE_NAME); + + int cnt = 500; + + int min = Integer.MAX_VALUE; + + int minIdx = 0; + + for (int i = 0; i < cnt; i++) { + cache.put(i, i); + + int cacheSize = cache.size(); + + if (i > max && cacheSize < min) { + min = cacheSize; + minIdx = i; + } + } + + assertTrue("Min cache size is too small: " + min, min >= max); + + check(max, PUT_ENTRY_SIZE); + + info("Min cache size [min=" + min + ", idx=" + minIdx + ']'); + info("Current cache size " + cache.size()); + info("Current cache key size " + cache.size()); + + min = Integer.MAX_VALUE; + + minIdx = 0; + + // Touch. + for (int i = cnt; --i > cnt - max;) { + cache.get(i); + + int cacheSize = cache.size(); + + if (cacheSize < min) { + min = cacheSize; + minIdx = i; + } + } + + info("----"); + info("Min cache size [min=" + min + ", idx=" + minIdx + ']'); + info("Current cache size " + cache.size()); + info("Current cache key size " + cache.size()); + + check(max, PUT_ENTRY_SIZE); + } + finally { + stopAllGrids(); + } + } + + /** + * @param arr Array. + * @param idx Index. + * @return Entry at the index. + */ + protected MockEntry entry(MockEntry[] arr, int idx) { + MockEntry e = arr[idx]; + + if (e.isEvicted()) + e = arr[idx] = new MockEntry(e.getKey()); + + return e; + } + + /** + * @param prefix Prefix. + * @param p Policy. + */ + protected void info(String prefix, EvictionPolicy p) { + info(prefix + ": " + p.toString()); + } + + /** @param p Policy. */ + protected void info(EvictionPolicy p) { + info(p.toString()); + } + + /** + * @param c1 Policy collection. + * @param c2 Expected list. + */ + protected static void check(Collection> c1, MockEntry... c2) { + check(c1, F.asList(c2)); + } + + /** + * @param expSize Expected size. + * @param entrySize Entry size. + */ + protected void check(int expSize, int entrySize) { + EvictionPolicyProxy proxy = proxy(policy()); + + assertEquals(expSize, proxy.getCurrentSize()); + assertEquals(expSize * entrySize, proxy.getCurrentMemorySize()); + } + + /** + * @param entrySize Entry size. + * @param c1 Closure 1. + * @param c2 Closure 2. + */ + protected void check(int entrySize, Collection> c1, MockEntry... c2) { + check(c2.length, entrySize); + + check(c1, c2); + } + + /** @return Policy. */ + protected T policy() { + GridCacheEvictionManager evictMgr = grid().cachex(DEFAULT_CACHE_NAME).context().evicts(); + + assert evictMgr != null; + + return (T)evictMgr.getEvictionPolicy(); + } + + /** + * @param i Grid index. + * @return Policy. + */ + @SuppressWarnings({"unchecked"}) + protected T policy(int i) { + GridCacheEvictionManager evictMgr = grid(i).cachex(DEFAULT_CACHE_NAME).context().evicts(); + + assert evictMgr != null; + + return (T)evictMgr.getEvictionPolicy(); + } + + /** + * @param i Grid index. + * @return Policy. + */ + @SuppressWarnings({"unchecked"}) + protected T nearPolicy(int i) { + GridCacheEvictionManager evictMgr = grid(i).cachex(DEFAULT_CACHE_NAME).context().near().context().evicts(); + + assert evictMgr !=null; + + return (T)evictMgr.getEvictionPolicy(); + } + + /** + * @param c1 Policy collection. + * @param c2 Expected list. + */ + protected static void check(Collection> c1, List c2) { + assert c1.size() == c2.size() : "Mismatch [actual=" + string(c1) + ", expected=" + string(c2) + ']'; + + assert c1.containsAll(c2) : "Mismatch [actual=" + string(c1) + ", expected=" + string(c2) + ']'; + + int i = 0; + + // Check order. + for (Cache.Entry e : c1) + assertEquals(e, c2.get(i++)); + } + + /** + * @param c Collection. + * @return String. + */ + @SuppressWarnings("unchecked") + protected static String string(Iterable c) { + return "[" + + F.fold( + c, + "", + new C2() { + @Override public String apply(Cache.Entry e, String b) { + return b.isEmpty() ? e.getKey().toString() : b + ", " + e.getKey(); + } + }) + + "]]"; + } + + /** @throws Exception If failed. */ + public void testMaxSizePartitionedNearDisabled() throws Exception { + mode = PARTITIONED; + nearEnabled = false; + plcMax = 10; + syncCommit = true; + + gridCnt = 2; + + checkPartitioned(); + } + + /** @throws Exception If failed. */ + public void testMaxSizePartitionedNearDisabledWithBatch() throws Exception { + mode = PARTITIONED; + nearEnabled = false; + plcMax = 10; + plcBatchSize = 2; + syncCommit = true; + + gridCnt = 2; + + checkPartitioned(); + } + + /** @throws Exception If failed. */ + public void testMaxMemSizePartitionedNearDisabled() throws Exception { + mode = PARTITIONED; + nearEnabled = false; + plcMax = 0; + plcMaxMemSize = 100; + syncCommit = true; + + gridCnt = 2; + + checkPartitioned(); + } + + /** @throws Exception If failed. */ + public void testPartitionedNearEnabled() throws Exception { + mode = PARTITIONED; + nearEnabled = true; + nearMax = 3; + plcMax = 10; + syncCommit = true; + + gridCnt = 2; + + checkPartitioned(); // Near size is 0 because of backups present. + } + + /** @throws Exception If failed. */ + public void testPartitionedNearDisabledMultiThreaded() throws Exception { + mode = PARTITIONED; + nearEnabled = false; + plcMax = 100; + + gridCnt = 2; + + checkPartitionedMultiThreaded(); + } + + /** @throws Exception If failed. */ + public void testPartitionedNearEnabledMultiThreaded() throws Exception { + mode = PARTITIONED; + nearEnabled = true; + plcMax = 10; + + gridCnt = 2; + + checkPartitionedMultiThreaded(); + } + + /** + * @throws Exception If failed. + */ + protected void checkPartitioned() throws Exception { + int endSize = nearEnabled ? 0 : plcMax; + + int endPlcSize = nearEnabled ? 0 : plcMax; + + policyFactory = createPolicyFactory(); + + startGridsMultiThreaded(gridCnt); + + try { + Random rand = new Random(); + + int cnt = 500; + + for (int i = 0; i < cnt; i++) { + IgniteCache cache = grid(rand.nextInt(2)).cache(DEFAULT_CACHE_NAME); + + int key = rand.nextInt(100); + String val = Integer.toString(key); + + cache.put(key, val); + + if (i % 100 == 0) + info("Stored cache object for key [key=" + key + ", idx=" + i + ']'); + } + + if (nearEnabled) { + for (int i = 0; i < gridCnt; i++) + assertEquals(endSize, near(i).nearSize()); + + if (endPlcSize >= 0) + checkNearPolicies(endPlcSize); + } + else { + if (plcMaxMemSize > 0) { + for (int i = 0; i < gridCnt; i++) { + GridDhtColocatedCache cache = colocated(i); + + int memSize = 0; + + for (Cache.Entry entry : cache.entrySet()) + memSize += entry.unwrap(EvictableEntry.class).size(); + + EvictionPolicyProxy plc = proxy(policy(i)); + + assertTrue(plc.getCurrentMemorySize() <= memSize); + } + } + + if (plcMax > 0) { + for (int i = 0; i < gridCnt; i++) { + int actual = colocated(i).size(); + + assertTrue("Cache size is greater then policy size [expected=" + endSize + ", actual=" + actual + ']', + actual <= endSize + (plcMaxMemSize > 0 ? 1 : plcBatchSize)); + } + } + + checkPolicies(); + } + } + finally { + stopAllGrids(); + } + } + + /** + * @throws Exception If failed. + */ + protected void checkPartitionedMultiThreaded() throws Exception { + policyFactory = createPolicyFactory(); + + try { + startGridsMultiThreaded(gridCnt); + + final Random rand = new Random(); + + final AtomicInteger cntr = new AtomicInteger(); + + multithreaded(new Callable() { + @Nullable @Override public Object call() throws Exception { + int cnt = 100; + + for (int i = 0; i < cnt && !Thread.currentThread().isInterrupted(); i++) { + IgniteEx grid = grid(rand.nextInt(2)); + + IgniteCache cache = grid.cache(DEFAULT_CACHE_NAME); + + int key = rand.nextInt(1000); + String val = Integer.toString(key); + + try (Transaction tx = grid.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + String v = cache.get(key); + + assert v == null || v.equals(Integer.toString(key)) : "Invalid value for key [key=" + key + + ", val=" + v + ']'; + + cache.put(key, val); + + tx.commit(); + } + + if (cntr.incrementAndGet() % 100 == 0) + info("Stored cache object for key [key=" + key + ", idx=" + i + ']'); + } + + return null; + } + }, 10); + } + finally { + stopAllGrids(); + } + } + + /** + * @return Policy. + * @deprecated replace with getPolicyFactory(); + */ + @Deprecated + protected T createPolicy() { + return null; + } + + /** + * @return Policy. + */ + protected abstract Factory createPolicyFactory(); + + /** + * @param nearMax Near max. + * @return Policy. + */ + protected abstract Factory createNearPolicyFactory(int nearMax); + + /** + * Performs after-test near policy check. + * + * @param nearMax Near max. + */ + protected void checkNearPolicies(int nearMax) { + for (int i = 0; i < gridCnt; i++) { + + EvictionPolicyProxy proxy = proxy(nearPolicy(i)); + + for (EvictableEntry e : proxy.queue()) + assert !e.isCached() : "Invalid near policy size: " + proxy.queue(); + } + } + + /** + * Performs after-test policy check. + */ + protected void checkPolicies() { + for (int i = 0; i < gridCnt; i++) { + if (plcMaxMemSize > 0) { + int size = 0; + + for (EvictableEntry entry : proxy(policy(i)).queue()) + size += entry.size(); + + assertEquals(size, proxy(policy(i)).getCurrentMemorySize()); + } + else + assertTrue(proxy(policy(i)).queue().size() <= plcMax + plcBatchSize); + } + } + + /** + * + */ + @SuppressWarnings({"PublicConstructorInNonPublicClass"}) + protected static class MockEntry extends GridCacheMockEntry { + /** Key size. */ + public static final int KEY_SIZE = 1; + + /** Value size. */ + public static final int VALUE_SIZE = 1; + + /** Entry size. */ + public static final int ENTRY_SIZE = KEY_SIZE + VALUE_SIZE; + + /** */ + private IgniteCache parent; + + /** Entry value. */ + private String val; + + /** @param key Key. */ + public MockEntry(String key) { + super(key); + } + + /** + * @param key Key. + * @param val Value. + */ + public MockEntry(String key, String val) { + super(key); + + this.val = val; + } + + /** + * @param key Key. + * @param parent Parent. + */ + public MockEntry(String key, @Nullable IgniteCache parent) { + super(key); + + this.parent = parent; + } + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Override public T unwrap(Class clazz) { + if (clazz.isAssignableFrom(IgniteCache.class)) + return (T)parent; + + return super.unwrap(clazz); + } + + /** {@inheritDoc} */ + @Override public String getValue() throws IllegalStateException { + return val; + } + + /** {@inheritDoc} */ + @Override public int size() { + return val == null ? KEY_SIZE : ENTRY_SIZE; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(MockEntry.class, this, super.toString()); + } + } + + /** + * Rvicition policy proxy. + */ + public static class EvictionPolicyProxy implements EvictionPolicy { + /** Policy. */ + private final EvictionPolicy plc; + + /** + * @param plc Policy. + */ + private EvictionPolicyProxy(EvictionPolicy plc) { + this.plc = plc; + } + + /** + * @param plc Policy. + * @return Policy proxy. + */ + public static EvictionPolicyProxy proxy(EvictionPolicy plc) { + return new EvictionPolicyProxy(plc); + } + + /** + * @return Get current size. + */ + int getCurrentSize() { + try { + return (Integer)plc.getClass().getDeclaredMethod("getCurrentSize").invoke(plc); + } + catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) { + throw new RuntimeException(e); + } + } + + /** + * @return Current memory size. + */ + long getCurrentMemorySize() { + try { + return (Long)plc.getClass().getMethod("getCurrentMemorySize").invoke(plc); + } + catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) { + throw new RuntimeException(e); + } + } + + /** + * @return Current queue. + */ + public Collection queue() { + try { + return (Collection)plc.getClass().getDeclaredMethod("queue").invoke(plc); + } + catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) { + throw new RuntimeException(e); + } + } + + /** + * @param rmv Remove. + * @param entry Entry. + */ + @Override public void onEntryAccessed(boolean rmv, EvictableEntry entry) { + try { + plc.getClass() + .getMethod("onEntryAccessed", boolean.class, EvictableEntry.class) + .invoke(plc, rmv, entry); + } + catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) { + throw new RuntimeException(e); + } + } + } +} \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/fifo/FifoEvictionPolicyFactorySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/fifo/FifoEvictionPolicyFactorySelfTest.java new file mode 100644 index 0000000000000..472bf41949091 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/fifo/FifoEvictionPolicyFactorySelfTest.java @@ -0,0 +1,261 @@ +/* + * 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.eviction.fifo; + +import javax.cache.configuration.Factory; +import org.apache.ignite.cache.eviction.fifo.FifoEvictionPolicy; +import org.apache.ignite.cache.eviction.fifo.FifoEvictionPolicyFactory; +import org.apache.ignite.internal.processors.cache.eviction.EvictionPolicyFactoryAbstractTest; + +/** + * FIFO eviction policy tests. + */ +public class FifoEvictionPolicyFactorySelfTest extends EvictionPolicyFactoryAbstractTest> { + /** {@inheritDoc} */ + @Override protected Factory> createPolicyFactory() { + return new FifoEvictionPolicyFactory<>(plcMax, plcBatchSize, plcMaxMemSize); + } + + /** {@inheritDoc} */ + @Override protected Factory> createNearPolicyFactory(int nearMax) { + FifoEvictionPolicyFactory plc = new FifoEvictionPolicyFactory<>(); + + plc.setMaxSize(nearMax); + plc.setBatchSize(plcBatchSize); + + return plc; + } + + /** {@inheritDoc} */ + @Override protected void doTestPolicy() throws Exception { + policyFactory = createPolicyFactory(); + + try { + startGrid(); + + MockEntry e1 = new MockEntry("1", "1"); + MockEntry e2 = new MockEntry("2", "2"); + MockEntry e3 = new MockEntry("3", "3"); + MockEntry e4 = new MockEntry("4", "4"); + MockEntry e5 = new MockEntry("5", "5"); + + FifoEvictionPolicy p = policy(); + + p.onEntryAccessed(false, e1); + + check(MockEntry.ENTRY_SIZE, p.queue(), e1); + + p.onEntryAccessed(false, e2); + + check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2); + + p.onEntryAccessed(false, e3); + + check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2, e3); + + assert !e1.isEvicted(); + assert !e2.isEvicted(); + assert !e3.isEvicted(); + + p.onEntryAccessed(false, e4); + + check(MockEntry.ENTRY_SIZE, p.queue(), e2, e3, e4); + + assert e1.isEvicted(); + assert !e2.isEvicted(); + assert !e3.isEvicted(); + assert !e4.isEvicted(); + + p.onEntryAccessed(false, e5); + + check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5); + + assert e2.isEvicted(); + assert !e3.isEvicted(); + assert !e4.isEvicted(); + assert !e5.isEvicted(); + + p.onEntryAccessed(false, e1 = new MockEntry("1", "1")); + + check(MockEntry.ENTRY_SIZE, p.queue(), e4, e5, e1); + + assert e3.isEvicted(); + assert !e1.isEvicted(); + assert !e4.isEvicted(); + assert !e5.isEvicted(); + + p.onEntryAccessed(false, e5); + + check(MockEntry.ENTRY_SIZE, p.queue(), e4, e5, e1); + + assert !e1.isEvicted(); + assert !e4.isEvicted(); + assert !e5.isEvicted(); + + p.onEntryAccessed(false, e1); + + check(MockEntry.ENTRY_SIZE, p.queue(), e4, e5, e1); + + assert !e1.isEvicted(); + assert !e4.isEvicted(); + assert !e5.isEvicted(); + + p.onEntryAccessed(false, e5); + + check(MockEntry.ENTRY_SIZE, p.queue(), e4, e5, e1); + + assert !e1.isEvicted(); + assert !e4.isEvicted(); + assert !e5.isEvicted(); + + p.onEntryAccessed(true, e1); + + check(MockEntry.ENTRY_SIZE, p.queue(), e4, e5); + + assert !e1.isEvicted(); + assert !e4.isEvicted(); + assert !e5.isEvicted(); + + p.onEntryAccessed(true, e4); + + check(MockEntry.ENTRY_SIZE, p.queue(), e5); + + assert !e4.isEvicted(); + assert !e5.isEvicted(); + + p.onEntryAccessed(true, e5); + + check(MockEntry.ENTRY_SIZE, p.queue()); + + assert !e5.isEvicted(); + + info(p); + } + finally { + stopAllGrids(); + } + } + + /** {@inheritDoc} */ + @Override protected void doTestPolicyWithBatch() throws Exception { + policyFactory = createPolicyFactory(); + + try { + startGrid(); + + MockEntry e1 = new MockEntry("1", "1"); + MockEntry e2 = new MockEntry("2", "2"); + MockEntry e3 = new MockEntry("3", "3"); + MockEntry e4 = new MockEntry("4", "4"); + MockEntry e5 = new MockEntry("5", "5"); + + FifoEvictionPolicy p = policy(); + + p.onEntryAccessed(false, e1); + + check(MockEntry.ENTRY_SIZE, p.queue(), e1); + + p.onEntryAccessed(false, e2); + + check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2); + + p.onEntryAccessed(false, e3); + + check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2, e3); + + p.onEntryAccessed(false, e4); + + check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2, e3, e4); + + assertFalse(e1.isEvicted()); + assertFalse(e2.isEvicted()); + assertFalse(e3.isEvicted()); + assertFalse(e4.isEvicted()); + + p.onEntryAccessed(false, e5); + + // Batch evicted. + check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5); + + assertTrue(e1.isEvicted()); + assertTrue(e2.isEvicted()); + assertFalse(e3.isEvicted()); + assertFalse(e4.isEvicted()); + assertFalse(e5.isEvicted()); + + p.onEntryAccessed(false, e1 = new MockEntry("1", "1")); + + check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5, e1); + + assertFalse(e3.isEvicted()); + assertFalse(e4.isEvicted()); + assertFalse(e5.isEvicted()); + assertFalse(e1.isEvicted()); + + p.onEntryAccessed(false, e5); + + check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5, e1); + + assertFalse(e3.isEvicted()); + assertFalse(e4.isEvicted()); + assertFalse(e5.isEvicted()); + assertFalse(e1.isEvicted()); + + p.onEntryAccessed(false, e1); + + check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5, e1); + + assertFalse(e3.isEvicted()); + assertFalse(e4.isEvicted()); + assertFalse(e5.isEvicted()); + assertFalse(e1.isEvicted()); + + p.onEntryAccessed(true, e1); + + check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5); + + assertFalse(e3.isEvicted()); + assertFalse(e4.isEvicted()); + assertFalse(e5.isEvicted()); + + p.onEntryAccessed(true, e4); + + check(MockEntry.ENTRY_SIZE, p.queue(), e3, e5); + + assertFalse(e3.isEvicted()); + assertFalse(e5.isEvicted()); + + p.onEntryAccessed(true, e5); + + check(MockEntry.ENTRY_SIZE, p.queue(), e3); + + assertFalse(e3.isEvicted()); + + p.onEntryAccessed(true, e3); + + check(MockEntry.ENTRY_SIZE, p.queue()); + + assertFalse(e3.isEvicted()); + + info(p); + } + finally { + stopAllGrids(); + } + } +} \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/LruEvictionPolicyFactorySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/LruEvictionPolicyFactorySelfTest.java new file mode 100644 index 0000000000000..d53cb6f36c923 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/LruEvictionPolicyFactorySelfTest.java @@ -0,0 +1,352 @@ +/* + * 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.eviction.lru; + +import javax.cache.configuration.Factory; +import org.apache.ignite.cache.eviction.EvictableEntry; +import org.apache.ignite.cache.eviction.lru.LruEvictionPolicy; +import org.apache.ignite.cache.eviction.lru.LruEvictionPolicyFactory; +import org.apache.ignite.internal.processors.cache.eviction.EvictionPolicyFactoryAbstractTest; + +/** + * LRU Eviction policy tests. + */ +public class LruEvictionPolicyFactorySelfTest extends EvictionPolicyFactoryAbstractTest> { + /** {@inheritDoc} */ + @Override protected Factory> createPolicyFactory() { + return new LruEvictionPolicyFactory<>(plcMax, plcBatchSize, plcMaxMemSize); + } + + /** {@inheritDoc} */ + @Override protected Factory> createNearPolicyFactory(int nearMax) { + LruEvictionPolicyFactory plc = new LruEvictionPolicyFactory<>(); + + plc.setMaxSize(nearMax); + plc.setBatchSize(plcBatchSize); + + return plc; + } + + /** + * @throws Exception If failed. + */ + public void testMiddleAccess() throws Exception { + policyFactory = createPolicyFactory(); + + try { + startGrid(); + + LruEvictionPolicy p = policy(); + + int max = 8; + + p.setMaxSize(max * MockEntry.ENTRY_SIZE); + + MockEntry entry1 = new MockEntry("1", "1"); + MockEntry entry2 = new MockEntry("2", "2"); + MockEntry entry3 = new MockEntry("3", "3"); + + p.onEntryAccessed(false, entry1); + p.onEntryAccessed(false, entry2); + p.onEntryAccessed(false, entry3); + + MockEntry[] freqUsed = new MockEntry[] { + new MockEntry("4", "4"), + new MockEntry("5", "5"), + new MockEntry("6", "6"), + new MockEntry("7", "7"), + new MockEntry("8", "7") + }; + + for (MockEntry e : freqUsed) + p.onEntryAccessed(false, e); + + for (MockEntry e : freqUsed) + assert !e.isEvicted(); + + int cnt = 1001; + + for (int i = 0; i < cnt; i++) + p.onEntryAccessed(false, entry(freqUsed, i % freqUsed.length)); + + info(p); + + check(max, MockEntry.ENTRY_SIZE); + } + finally { + stopGrid(); + } + } + + /** {@inheritDoc} */ + @Override protected void doTestPolicy() throws Exception { + policyFactory = createPolicyFactory(); + + try { + startGrid(); + + MockEntry e1 = new MockEntry("1", "1"); + MockEntry e2 = new MockEntry("2", "2"); + MockEntry e3 = new MockEntry("3", "3"); + MockEntry e4 = new MockEntry("4", "4"); + MockEntry e5 = new MockEntry("5", "5"); + + LruEvictionPolicy p = policy(); + + p.onEntryAccessed(false, e1); + + check(MockEntry.ENTRY_SIZE, p.queue(), e1); + + p.onEntryAccessed(false, e2); + + check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2); + + p.onEntryAccessed(false, e3); + + check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2, e3); + + assertFalse(e1.isEvicted()); + assertFalse(e2.isEvicted()); + assertFalse(e3.isEvicted()); + + p.onEntryAccessed(false, e4); + + check(p.queue(), e2, e3, e4); + check(MockEntry.ENTRY_SIZE, p.queue(), e2, e3, e4); + + assertTrue(e1.isEvicted()); + assertFalse(e2.isEvicted()); + assertFalse(e3.isEvicted()); + assertFalse(e4.isEvicted()); + + p.onEntryAccessed(false, e5); + + check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5); + + assertTrue(e2.isEvicted()); + assertFalse(e3.isEvicted()); + assertFalse(e4.isEvicted()); + assertFalse(e5.isEvicted()); + + p.onEntryAccessed(false, e1 = new MockEntry("1", "1")); + + check(MockEntry.ENTRY_SIZE, p.queue(), e4, e5, e1); + + assertTrue(e3.isEvicted()); + assertFalse(e1.isEvicted()); + assertFalse(e4.isEvicted()); + assertFalse(e5.isEvicted()); + + p.onEntryAccessed(false, e5); + + assertEquals(3, p.getCurrentSize()); + + check(MockEntry.ENTRY_SIZE, p.queue(), e4, e1, e5); + + assertFalse(e1.isEvicted()); + assertFalse(e4.isEvicted()); + assertFalse(e5.isEvicted()); + + p.onEntryAccessed(false, e1); + + check(MockEntry.ENTRY_SIZE, p.queue(), e4, e5, e1); + + assertFalse(e1.isEvicted()); + assertFalse(e4.isEvicted()); + assertFalse(e5.isEvicted()); + + p.onEntryAccessed(false, e5); + + assertEquals(3, p.getCurrentSize()); + + check(MockEntry.ENTRY_SIZE, p.queue(), e4, e1, e5); + + assertFalse(e1.isEvicted()); + assertFalse(e4.isEvicted()); + assertFalse(e5.isEvicted()); + + p.onEntryAccessed(true, e1); + + check(MockEntry.ENTRY_SIZE, p.queue(), e4, e5); + + assertFalse(e1.isEvicted()); + assertFalse(e4.isEvicted()); + assertFalse(e5.isEvicted()); + + p.onEntryAccessed(true, e4); + + check(MockEntry.ENTRY_SIZE, p.queue(), e5); + + assertFalse(e4.isEvicted()); + assertFalse(e5.isEvicted()); + + p.onEntryAccessed(true, e5); + + check(MockEntry.ENTRY_SIZE, p.queue()); + + assertFalse(e5.isEvicted()); + + info(p); + } + finally { + stopGrid(); + } + } + + /** {@inheritDoc} */ + @Override protected void doTestPolicyWithBatch() throws Exception { + policyFactory = createPolicyFactory(); + + try { + startGrid(); + + MockEntry e1 = new MockEntry("1", "1"); + MockEntry e2 = new MockEntry("2", "2"); + MockEntry e3 = new MockEntry("3", "3"); + MockEntry e4 = new MockEntry("4", "4"); + MockEntry e5 = new MockEntry("5", "5"); + + LruEvictionPolicy p = policy(); + + p.onEntryAccessed(false, e1); + + check(MockEntry.ENTRY_SIZE, p.queue(), e1); + + p.onEntryAccessed(false, e2); + + check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2); + + p.onEntryAccessed(false, e3); + + check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2, e3); + + assertFalse(e1.isEvicted()); + assertFalse(e2.isEvicted()); + assertFalse(e3.isEvicted()); + + p.onEntryAccessed(false, e4); + + check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2, e3, e4); + + assertFalse(e1.isEvicted()); + assertFalse(e2.isEvicted()); + assertFalse(e3.isEvicted()); + assertFalse(e4.isEvicted()); + + p.onEntryAccessed(false, e5); + + // Batch evicted + check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5); + + assertTrue(e1.isEvicted()); + assertTrue(e2.isEvicted()); + assertFalse(e3.isEvicted()); + assertFalse(e4.isEvicted()); + assertFalse(e5.isEvicted()); + + p.onEntryAccessed(false, e1 = new MockEntry("1", "1")); + + check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5, e1); + + assertFalse(e3.isEvicted()); + assertFalse(e4.isEvicted()); + assertFalse(e5.isEvicted()); + assertFalse(e1.isEvicted()); + + p.onEntryAccessed(false, e5); + + check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e1, e5); + + assertFalse(e3.isEvicted()); + assertFalse(e4.isEvicted()); + assertFalse(e1.isEvicted()); + assertFalse(e5.isEvicted()); + + p.onEntryAccessed(false, e1); + + check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5, e1); + + assertFalse(e3.isEvicted()); + assertFalse(e4.isEvicted()); + assertFalse(e5.isEvicted()); + assertFalse(e1.isEvicted()); + + p.onEntryAccessed(false, e5); + + check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e1, e5); + + assertFalse(e3.isEvicted()); + assertFalse(e4.isEvicted()); + assertFalse(e1.isEvicted()); + assertFalse(e5.isEvicted()); + + p.onEntryAccessed(true, e1); + + check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5); + + assertFalse(e3.isEvicted()); + assertFalse(e4.isEvicted()); + assertFalse(e5.isEvicted()); + + p.onEntryAccessed(true, e4); + + check(MockEntry.ENTRY_SIZE, p.queue(), e3, e5); + + assertFalse(e3.isEvicted()); + assertFalse(e5.isEvicted()); + + p.onEntryAccessed(true, e5); + + check(MockEntry.ENTRY_SIZE, p.queue(), e3); + + assertFalse(e3.isEvicted()); + + p.onEntryAccessed(true, e3); + + check(MockEntry.ENTRY_SIZE, p.queue()); + + info(p); + } + finally { + stopGrid(); + } + } + + /** {@inheritDoc} */ + @Override protected void checkNearPolicies(int endNearPlcSize) { + for (int i = 0; i < gridCnt; i++) + for (EvictableEntry e : nearPolicy(i).queue()) + assert !e.isCached() : "Invalid near policy size: " + nearPolicy(i).queue(); + } + + /** {@inheritDoc} */ + @Override protected void checkPolicies() { + for (int i = 0; i < gridCnt; i++) { + if (plcMaxMemSize > 0) { + int size = 0; + + for (EvictableEntry entry : policy(i).queue()) + size += entry.size(); + + assertEquals(size, policy(i).getCurrentMemorySize()); + } + else + assertTrue(policy(i).queue().size() <= plcMax + plcBatchSize); + } + } +} \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/SortedEvictionPolicyFactorySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/SortedEvictionPolicyFactorySelfTest.java new file mode 100644 index 0000000000000..a0ab18f0204db --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/SortedEvictionPolicyFactorySelfTest.java @@ -0,0 +1,264 @@ +/* + * 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.eviction.sorted; + +import javax.cache.configuration.Factory; +import org.apache.ignite.cache.eviction.sorted.SortedEvictionPolicy; +import org.apache.ignite.cache.eviction.sorted.SortedEvictionPolicyFactory; +import org.apache.ignite.internal.processors.cache.eviction.EvictionPolicyFactoryAbstractTest; + +/** + * Sorted eviction policy tests. + */ +public class SortedEvictionPolicyFactorySelfTest extends EvictionPolicyFactoryAbstractTest> { + /** {@inheritDoc} */ + @Override protected Factory> createPolicyFactory() { + return new SortedEvictionPolicyFactory<>(plcMax, plcBatchSize, plcMaxMemSize); + } + + /** {@inheritDoc} */ + @Override protected Factory> createNearPolicyFactory(int nearMax) { + SortedEvictionPolicyFactory plc = new SortedEvictionPolicyFactory<>(); + + plc.setMaxSize(nearMax); + plc.setBatchSize(plcBatchSize); + + return plc; + } + + /** {@inheritDoc} */ + @Override protected void doTestPolicy() throws Exception { + policyFactory = createPolicyFactory(); + + try { + startGrid(); + + MockEntry e1 = new MockEntry("1", "1"); + MockEntry e2 = new MockEntry("2", "2"); + MockEntry e3 = new MockEntry("3", "3"); + MockEntry e4 = new MockEntry("4", "4"); + MockEntry e5 = new MockEntry("5", "5"); + + SortedEvictionPolicy p = policy(); + + p.onEntryAccessed(false, e1); + + check(MockEntry.ENTRY_SIZE, p.queue(), e1); + + p.onEntryAccessed(false, e2); + + check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2); + + p.onEntryAccessed(false, e3); + + check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2, e3); + + assertFalse(e1.isEvicted()); + assertFalse(e2.isEvicted()); + assertFalse(e3.isEvicted()); + + check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2, e3); + + p.onEntryAccessed(false, e4); + + check(MockEntry.ENTRY_SIZE, p.queue(), e2, e3, e4); + + assertTrue(e1.isEvicted()); + assertFalse(e2.isEvicted()); + assertFalse(e3.isEvicted()); + assertFalse(e4.isEvicted()); + + p.onEntryAccessed(false, e5); + + check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5); + + assertTrue(e2.isEvicted()); + assertFalse(e3.isEvicted()); + assertFalse(e4.isEvicted()); + assertFalse(e5.isEvicted()); + + p.onEntryAccessed(false, e1 = new MockEntry("1", "1")); + + check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5); + + assertTrue(e1.isEvicted()); + assertFalse(e3.isEvicted()); + assertFalse(e4.isEvicted()); + assertFalse(e5.isEvicted()); + + p.onEntryAccessed(false, e5); + + check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5); + + assertFalse(e3.isEvicted()); + assertFalse(e4.isEvicted()); + assertFalse(e5.isEvicted()); + + p.onEntryAccessed(false, e1); + + check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5); + + assertTrue(e1.isEvicted()); + assertFalse(e3.isEvicted()); + assertFalse(e4.isEvicted()); + assertFalse(e5.isEvicted()); + + p.onEntryAccessed(false, e5); + + check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5); + + assertFalse(e3.isEvicted()); + assertFalse(e4.isEvicted()); + assertFalse(e5.isEvicted()); + + p.onEntryAccessed(true, e3); + + check(MockEntry.ENTRY_SIZE, p.queue(), e4, e5); + + assertFalse(e3.isEvicted()); + assertFalse(e4.isEvicted()); + assertFalse(e5.isEvicted()); + + p.onEntryAccessed(true, e4); + + check(MockEntry.ENTRY_SIZE, p.queue(), e5); + + assertFalse(e4.isEvicted()); + assertFalse(e5.isEvicted()); + + p.onEntryAccessed(true, e5); + + check(MockEntry.ENTRY_SIZE, p.queue()); + + assertFalse(e5.isEvicted()); + + info(p); + } + finally { + stopAllGrids(); + } + } + + /** {@inheritDoc} */ + @Override protected void doTestPolicyWithBatch() throws Exception { + policyFactory = createPolicyFactory(); + + try { + startGrid(); + + MockEntry e1 = new MockEntry("1", "1"); + MockEntry e2 = new MockEntry("2", "2"); + MockEntry e3 = new MockEntry("3", "3"); + MockEntry e4 = new MockEntry("4", "4"); + MockEntry e5 = new MockEntry("5", "5"); + + SortedEvictionPolicy p = policy(); + + p.onEntryAccessed(false, e1); + + check(MockEntry.ENTRY_SIZE, p.queue(), e1); + + p.onEntryAccessed(false, e2); + + check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2); + + p.onEntryAccessed(false, e3); + + check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2, e3); + + p.onEntryAccessed(false, e4); + + check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2, e3, e4); + + assertFalse(e1.isEvicted()); + assertFalse(e2.isEvicted()); + assertFalse(e3.isEvicted()); + assertFalse(e4.isEvicted()); + + p.onEntryAccessed(false, e5); + + // Batch evicted. + check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5); + + assertTrue(e1.isEvicted()); + assertTrue(e2.isEvicted()); + assertFalse(e3.isEvicted()); + assertFalse(e4.isEvicted()); + assertFalse(e5.isEvicted()); + + p.onEntryAccessed(false, e1 = new MockEntry("1", "1")); + + check(MockEntry.ENTRY_SIZE, p.queue(), e1, e3, e4, e5); + + assertFalse(e1.isEvicted()); + assertFalse(e3.isEvicted()); + assertFalse(e4.isEvicted()); + assertFalse(e5.isEvicted()); + + p.onEntryAccessed(false, e5); + + check(MockEntry.ENTRY_SIZE, p.queue(), e1, e3, e4, e5); + + assertFalse(e1.isEvicted()); + assertFalse(e3.isEvicted()); + assertFalse(e4.isEvicted()); + assertFalse(e5.isEvicted()); + + p.onEntryAccessed(false, e1); + + check(MockEntry.ENTRY_SIZE, p.queue(), e1, e3, e4, e5); + + assertFalse(e1.isEvicted()); + assertFalse(e3.isEvicted()); + assertFalse(e4.isEvicted()); + assertFalse(e5.isEvicted()); + + p.onEntryAccessed(true, e1); + + check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5); + + assertFalse(e3.isEvicted()); + assertFalse(e4.isEvicted()); + assertFalse(e5.isEvicted()); + + p.onEntryAccessed(true, e4); + + check(MockEntry.ENTRY_SIZE, p.queue(), e3, e5); + + assertFalse(e3.isEvicted()); + assertFalse(e5.isEvicted()); + + p.onEntryAccessed(true, e5); + + check(MockEntry.ENTRY_SIZE, p.queue(), e3); + + assertFalse(e3.isEvicted()); + + p.onEntryAccessed(true, e3); + + check(MockEntry.ENTRY_SIZE, p.queue()); + + assertFalse(e3.isEvicted()); + + info(p); + } + finally { + stopAllGrids(); + } + } +} \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheEvictionSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheEvictionSelfTestSuite.java index 1ad63ee13ff56..fa1eac12fcfc5 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheEvictionSelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheEvictionSelfTestSuite.java @@ -33,10 +33,13 @@ import org.apache.ignite.internal.processors.cache.eviction.GridCacheEvictionLockUnlockSelfTest; import org.apache.ignite.internal.processors.cache.eviction.GridCacheEvictionTouchSelfTest; import org.apache.ignite.internal.processors.cache.eviction.GridCacheSynchronousEvictionsFailoverSelfTest; +import org.apache.ignite.internal.processors.cache.eviction.fifo.FifoEvictionPolicyFactorySelfTest; import org.apache.ignite.internal.processors.cache.eviction.fifo.FifoEvictionPolicySelfTest; +import org.apache.ignite.internal.processors.cache.eviction.lru.LruEvictionPolicyFactorySelfTest; import org.apache.ignite.internal.processors.cache.eviction.lru.LruEvictionPolicySelfTest; import org.apache.ignite.internal.processors.cache.eviction.lru.LruNearEvictionPolicySelfTest; import org.apache.ignite.internal.processors.cache.eviction.lru.LruNearOnlyNearEvictionPolicySelfTest; +import org.apache.ignite.internal.processors.cache.eviction.sorted.SortedEvictionPolicyFactorySelfTest; import org.apache.ignite.internal.processors.cache.eviction.sorted.SortedEvictionPolicySelfTest; /** @@ -53,6 +56,9 @@ public static TestSuite suite() throws Exception { suite.addTest(new TestSuite(FifoEvictionPolicySelfTest.class)); suite.addTest(new TestSuite(SortedEvictionPolicySelfTest.class)); suite.addTest(new TestSuite(LruEvictionPolicySelfTest.class)); + suite.addTest(new TestSuite(FifoEvictionPolicyFactorySelfTest.class)); + suite.addTest(new TestSuite(SortedEvictionPolicyFactorySelfTest.class)); + suite.addTest(new TestSuite(LruEvictionPolicyFactorySelfTest.class)); suite.addTest(new TestSuite(LruNearEvictionPolicySelfTest.class)); suite.addTest(new TestSuite(LruNearOnlyNearEvictionPolicySelfTest.class)); suite.addTest(new TestSuite(GridCacheNearEvictionSelfTest.class)); diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java index a261b98514b78..c8d2b49f25283 100644 --- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java +++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java @@ -101,8 +101,10 @@ public IgniteNode(boolean clientMode, Ignite ignite) { if (args.isNearCache()) { NearCacheConfiguration nearCfg = new NearCacheConfiguration(); - if (args.getNearCacheSize() != 0) - nearCfg.setNearEvictionPolicy(new LruEvictionPolicy(args.getNearCacheSize())); + int nearCacheSize = args.getNearCacheSize(); + + if (nearCacheSize != 0) + nearCfg.setNearEvictionPolicy(new LruEvictionPolicy(nearCacheSize)); cc.setNearConfiguration(nearCfg); } From c2feb91cf801fee28bc77019b46c514975a28d8d Mon Sep 17 00:00:00 2001 From: mcherkasov Date: Fri, 10 Nov 2017 23:05:15 +0300 Subject: [PATCH 340/357] GG-12915 DataStreamer.addData(single entry) produces too much garbage --- .../util/future/GridFutureAdapter.java | 29 ++++++++++--------- 1 file changed, 16 insertions(+), 13 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java index 257f1996ddd08..925d77c8ef849 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java @@ -18,6 +18,8 @@ package org.apache.ignite.internal.util.future; import java.util.Arrays; +import java.util.LinkedList; +import java.util.List; import java.util.concurrent.Executor; import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.AbstractQueuedSynchronizer; @@ -213,10 +215,10 @@ private R get0(boolean ignoreInterrupts) throws IgniteCheckedException { if (!done) { if (lsnr == null) lsnr = lsnr0; - else if (lsnr instanceof ArrayListener) - ((ArrayListener)lsnr).add(lsnr0); + else if (lsnr instanceof GridFutureAdapter.LinkedListListener) + ((LinkedListListener)lsnr).add(lsnr0); else - lsnr = (IgniteInClosure)new ArrayListener(lsnr, lsnr0); + lsnr = (IgniteInClosure)new LinkedListListener(lsnr, lsnr0); return; } @@ -436,38 +438,39 @@ private String state() { /** * */ - private static class ArrayListener implements IgniteInClosure> { + private static class LinkedListListener implements IgniteInClosure> { /** */ private static final long serialVersionUID = 0L; /** */ - private IgniteInClosure>[] arr; + private List>> list; /** * @param lsnrs Listeners. */ - private ArrayListener(IgniteInClosure... lsnrs) { - this.arr = lsnrs; + private LinkedListListener(IgniteInClosure... lsnrs) { + list = new LinkedList<>(); + for (IgniteInClosure lsnr : lsnrs) { + list.add(lsnr); + } } /** {@inheritDoc} */ @Override public void apply(IgniteInternalFuture fut) { - for (int i = 0; i < arr.length; i++) - arr[i].apply(fut); + for (IgniteInClosure> closure : list) + closure.apply(fut); } /** * @param lsnr Listener. */ void add(IgniteInClosure> lsnr) { - arr = Arrays.copyOf(arr, arr.length + 1); - - arr[arr.length - 1] = lsnr; + list.add(lsnr); } /** {@inheritDoc} */ @Override public String toString() { - return S.toString(ArrayListener.class, this, "arrSize", arr.length); + return S.toString(LinkedListListener.class, this, "listSize", list.size()); } } From ebf512c02c05dd1644207a1b7359c1c8319bf297 Mon Sep 17 00:00:00 2001 From: amashenkov Date: Mon, 13 Nov 2017 10:26:53 +0300 Subject: [PATCH 341/357] GG-13021: Fixed NPE on node stop when SSL is used. (cherry picked from commit 132ec3f) --- .../internal/util/nio/GridNioServer.java | 8 ++++-- .../IgniteCommunicationBalanceTest.java | 13 +++++++++ .../IgniteCommunicationSslBalanceTest.java | 28 +++++++++++++++++++ .../testsuites/IgniteCacheTestSuite.java | 2 ++ 4 files changed, 49 insertions(+), 2 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationSslBalanceTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java index a59adba9d16e7..c3208f9d9ec0b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java @@ -3024,8 +3024,12 @@ protected HeadFilter() { GridSelectorNioSessionImpl ses0 = (GridSelectorNioSessionImpl)ses; - if (!ses0.procWrite.get() && ses0.procWrite.compareAndSet(false, true)) - ses0.worker().registerWrite(ses0); + if (!ses0.procWrite.get() && ses0.procWrite.compareAndSet(false, true)) { + GridNioWorker worker = ses0.worker(); + + if (worker != null) + worker.registerWrite(ses0); + } return null; } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalanceTest.java index 4271417554fa7..a913cb96bb26c 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalanceTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalanceTest.java @@ -72,9 +72,19 @@ public class IgniteCommunicationBalanceTest extends GridCommonAbstractTest { cfg.setClientMode(client); + if (sslEnabled()) + cfg.setSslContextFactory(GridTestUtils.sslFactory()); + return cfg; } + /** + * @return {@code True} to enable SSL. + */ + protected boolean sslEnabled() { + return false; + } + /** * @return Value for {@link TcpCommunicationSpi#setUsePairedConnections(boolean)}. */ @@ -100,6 +110,9 @@ protected int connectionsPerNode() { * @throws Exception If failed. */ public void testBalance1() throws Exception { + if (sslEnabled()) + return; + System.setProperty(IgniteSystemProperties.IGNITE_IO_BALANCE_PERIOD, "5000"); try { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationSslBalanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationSslBalanceTest.java new file mode 100644 index 0000000000000..68094e265577a --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationSslBalanceTest.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.managers.communication; + +/** + * + */ +public class IgniteCommunicationSslBalanceTest extends IgniteCommunicationBalanceTest { + /** {@inheritDoc} */ + @Override protected boolean sslEnabled() { + return true; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java index 159bc78c38ec3..fd8d43fa46d7a 100755 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java @@ -47,6 +47,7 @@ import org.apache.ignite.internal.managers.communication.IgniteCommunicationBalanceMultipleConnectionsTest; import org.apache.ignite.internal.managers.communication.IgniteCommunicationBalancePairedConnectionsTest; import org.apache.ignite.internal.managers.communication.IgniteCommunicationBalanceTest; +import org.apache.ignite.internal.managers.communication.IgniteCommunicationSslBalanceTest; import org.apache.ignite.internal.managers.communication.IgniteIoTestMessagesTest; import org.apache.ignite.internal.managers.communication.IgniteVariousConnectionNumberTest; import org.apache.ignite.internal.processors.cache.CacheAffinityCallSelfTest; @@ -351,6 +352,7 @@ public static TestSuite suite(Set ignoredTests) throws Exception { suite.addTestSuite(IgniteCommunicationBalanceTest.class); suite.addTestSuite(IgniteCommunicationBalancePairedConnectionsTest.class); suite.addTestSuite(IgniteCommunicationBalanceMultipleConnectionsTest.class); + suite.addTestSuite(IgniteCommunicationSslBalanceTest.class); suite.addTestSuite(IgniteIoTestMessagesTest.class); suite.addTestSuite(GridStoreLoadCacheTest.class); From 111fd8ba8c2a9459a731c3c4fde8ccfcff39dcc8 Mon Sep 17 00:00:00 2001 From: Tikhonov Nikolay Date: Sat, 11 Nov 2017 12:44:33 +0300 Subject: [PATCH 342/357] Fixed .NET test compilation. Signed-off-by: nikolay_tikhonov --- .../Cache/Store/CacheStoreSessionTest.cs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreSessionTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreSessionTest.cs index 8ab3b25c65b94..8118305af3591 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreSessionTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreSessionTest.cs @@ -110,7 +110,7 @@ public void TestSession() } Assert.AreEqual(1, _dumps.Count); - ops = _dumps.First(); + var ops = _dumps.First(); Assert.AreEqual(3, ops.Count); Assert.AreEqual(1, ops.Count(op => op.Type == OperationType.Write && Cache1.Equals(op.CacheName) && 1.Equals(op.Key) && 1.Equals(op.Value))); From 40a51bb83cb400a1e42e57ad955693e41faa175d Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Mon, 13 Nov 2017 10:35:21 +0300 Subject: [PATCH 343/357] IGNITE-6818 Handle half open connection in communication. (cherry picked from commit 191295d) --- .../tcp/TcpCommunicationSpi.java | 39 +++-- ...municationSpiHalfOpenedConnectionTest.java | 142 ++++++++++++++++++ .../IgniteSpiCommunicationSelfTestSuite.java | 2 + 3 files changed, 169 insertions(+), 14 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiHalfOpenedConnectionTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java index dffa181058c56..55e2b6b95af98 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 @@ -65,7 +65,6 @@ import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener; -import org.apache.ignite.internal.util.typedef.CI1; import org.apache.ignite.internal.util.GridConcurrentFactory; import org.apache.ignite.internal.util.GridSpinReadWriteLock; import org.apache.ignite.internal.util.future.GridFutureAdapter; @@ -94,6 +93,7 @@ import org.apache.ignite.internal.util.nio.ssl.BlockingSslHandler; import org.apache.ignite.internal.util.nio.ssl.GridNioSslFilter; import org.apache.ignite.internal.util.nio.ssl.GridSslMeta; +import org.apache.ignite.internal.util.typedef.CI1; import org.apache.ignite.internal.util.typedef.CI2; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.X; @@ -529,15 +529,7 @@ private void onFirstMessage(final GridNioSession ses, Message msg) { if (c.failed) { ses.send(new RecoveryLastReceivedMessage(-1)); - for (GridNioSession ses0 : nioSrvr.sessions()) { - ConnectionKey key0 = ses0.meta(CONN_IDX_META); - - if (ses0.accepted() && key0 != null && - key0.nodeId().equals(connKey.nodeId()) && - key0.connectionIndex() == connKey.connectionIndex() && - key0.connectCount() < connKey.connectCount()) - ses0.close(); - } + closeStaleConnections(connKey); } } } @@ -557,11 +549,13 @@ private void onFirstMessage(final GridNioSession ses, Message msg) { if (oldClient instanceof GridTcpNioCommunicationClient) { if (log.isInfoEnabled()) log.info("Received incoming connection when already connected " + - "to this node, rejecting [locNode=" + locNode.id() + - ", rmtNode=" + sndId + ']'); + "to this node, rejecting [locNode=" + locNode.id() + + ", rmtNode=" + sndId + ']'); ses.send(new RecoveryLastReceivedMessage(-1)); + closeStaleConnections(connKey); + return; } else { @@ -589,11 +583,13 @@ private void onFirstMessage(final GridNioSession ses, Message msg) { if (log.isInfoEnabled()) log.info("Received incoming connection when already connected " + - "to this node, rejecting [locNode=" + locNode.id() + - ", rmtNode=" + sndId + ']'); + "to this node, rejecting [locNode=" + locNode.id() + + ", rmtNode=" + sndId + ']'); ses.send(new RecoveryLastReceivedMessage(-1)); + closeStaleConnections(connKey); + fut.onDone(oldClient); return; @@ -647,6 +643,21 @@ private void onFirstMessage(final GridNioSession ses, Message msg) { } } + /** + * @param connKey Connection key. + */ + private void closeStaleConnections(ConnectionKey connKey) { + for (GridNioSession ses0 : nioSrvr.sessions()) { + ConnectionKey key0 = ses0.meta(CONN_IDX_META); + + if (ses0.accepted() && key0 != null && + key0.nodeId().equals(connKey.nodeId()) && + key0.connectionIndex() == connKey.connectionIndex() && + key0.connectCount() < connKey.connectCount()) + ses0.close(); + } + } + @Override public void onMessage(GridNioSession ses, Message msg) { ConnectionKey connKey = ses.meta(CONN_IDX_META); diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiHalfOpenedConnectionTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiHalfOpenedConnectionTest.java new file mode 100644 index 0000000000000..3e10f942c4459 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiHalfOpenedConnectionTest.java @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.communication.tcp; + +import java.io.IOException; +import java.util.Iterator; +import java.util.UUID; +import java.util.concurrent.ConcurrentMap; +import org.apache.ignite.Ignite; +import org.apache.ignite.cluster.ClusterGroup; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.util.nio.GridCommunicationClient; +import org.apache.ignite.internal.util.nio.GridNioRecoveryDescriptor; +import org.apache.ignite.internal.util.nio.GridNioServerListener; +import org.apache.ignite.internal.util.nio.GridTcpNioCommunicationClient; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Tests case when connection is closed only for one side, when other is not notified. + */ +public class TcpCommunicationSpiHalfOpenedConnectionTest extends GridCommonAbstractTest { + /** Client spi. */ + private TcpCommunicationSpi clientSpi; + + /** Paired connections. */ + private boolean pairedConnections; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + if (igniteInstanceName.contains("client")) { + cfg.setClientMode(true); + + clientSpi = (TcpCommunicationSpi)cfg.getCommunicationSpi(); + } + + ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setUsePairedConnections(pairedConnections); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(true); + } + + /** + * @throws Exception If failed. + */ + public void testReconnect() throws Exception { + pairedConnections = false; + + checkReconnect(); + } + + /** + * @throws Exception If failed. + */ + public void testReconnectPaired() throws Exception { + pairedConnections = true; + + checkReconnect(); + } + + /** + * @throws Exception If failed. + */ + private void checkReconnect() throws Exception { + Ignite srv = startGrid("server"); + Ignite client = startGrid("client"); + + UUID nodeId = srv.cluster().localNode().id(); + + System.out.println(">> Server ID: " + nodeId); + + ClusterGroup srvGrp = client.cluster().forNodeId(nodeId); + + System.out.println(">> Send job"); + + // Establish connection + client.compute(srvGrp).run(F.noop()); + + ConcurrentMap clients = U.field(clientSpi, "clients"); + ConcurrentMap recoveryDescs = U.field(clientSpi, "recoveryDescs"); + ConcurrentMap outRecDescs = U.field(clientSpi, "outRecDescs"); + ConcurrentMap inRecDescs = U.field(clientSpi, "inRecDescs"); + GridNioServerListener lsnr = U.field(clientSpi, "srvLsnr"); + + Iterator it = F.concat( + recoveryDescs.values().iterator(), + outRecDescs.values().iterator(), + inRecDescs.values().iterator() + ); + + while (it.hasNext()) { + GridNioRecoveryDescriptor desc = it.next(); + + // Need to simulate connection close in GridNioServer as it + // releases descriptors on disconnect. + desc.release(); + } + + // Remove client to avoid calling close(), in that case server + // will close connection too, but we want to keep the server + // uninformed and force ping old connection. + GridCommunicationClient[] clients0 = clients.remove(nodeId); + + for (GridCommunicationClient commClient : clients0) + lsnr.onDisconnected(((GridTcpNioCommunicationClient)commClient).session(), new IOException("Test exception")); + + info(">> Removed client"); + + // Reestablish connection + client.compute(srvGrp).run(F.noop()); + + info(">> Sent second job"); + } + + /** {@inheritDoc} */ + @Override protected long getTestTimeout() { + return 30_000; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java index 77de3fcc54999..8e96a3f223604 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java @@ -38,6 +38,7 @@ import org.apache.ignite.spi.communication.tcp.IgniteTcpCommunicationRecoveryAckClosureSelfTest; import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpiDropNodesTest; import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpiFaultyClientTest; +import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpiHalfOpenedConnectionTest; /** * Test suite for all communication SPIs. @@ -78,6 +79,7 @@ public static TestSuite suite() throws Exception { suite.addTest(new TestSuite(TcpCommunicationSpiFaultyClientTest.class)); suite.addTest(new TestSuite(TcpCommunicationSpiDropNodesTest.class)); + suite.addTest(new TestSuite(TcpCommunicationSpiHalfOpenedConnectionTest.class)); return suite; } From 86e3349c168bf11b45ef219e93c86adf42cb3a55 Mon Sep 17 00:00:00 2001 From: Ilya Kasnacheev Date: Tue, 14 Nov 2017 15:40:41 +0300 Subject: [PATCH 344/357] GG-12821 IGNITE-4642: Added "enforceJoinOrder" flag to thick JDBC driver. Also add detection of UPDATE/DELETE in executeQuery(), tangentially related to IGNITE-6326 --- .../jdbc2/JdbcConnectionSelfTest.java | 25 ++++++ .../jdbc2/JdbcDeleteStatementSelfTest.java | 30 +++++++ .../org/apache/ignite/IgniteJdbcDriver.java | 14 +++- .../ignite/internal/jdbc2/JdbcConnection.java | 12 +++ .../ignite/internal/jdbc2/JdbcQueryTask.java | 2 +- .../internal/jdbc2/JdbcQueryTaskV2.java | 8 ++ .../internal/jdbc2/JdbcQueryTaskV3.java | 49 +++++++++++ .../ignite/internal/jdbc2/JdbcResultSet.java | 19 ++++- .../ignite/internal/jdbc2/JdbcStatement.java | 82 ++++++++++++++----- 9 files changed, 216 insertions(+), 25 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTaskV3.java diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcConnectionSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcConnectionSelfTest.java index 0bbdca86c67f9..e080027059ec8 100644 --- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcConnectionSelfTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcConnectionSelfTest.java @@ -268,6 +268,31 @@ public void testClose() throws Exception { } } + /** + * @throws Exception If failed. + */ + public void testSqlHints() throws Exception { + try (final Connection conn = DriverManager.getConnection(CFG_URL_PREFIX + "enforceJoinOrder=true@" + + CFG_URL)) { + assertTrue(((JdbcConnection)conn).isEnforceJoinOrder()); + assertFalse(((JdbcConnection)conn).isDistributedJoins()); + assertFalse(((JdbcConnection)conn).isCollocatedQuery()); + } + + try (final Connection conn = DriverManager.getConnection(CFG_URL_PREFIX + "distributedJoins=true@" + + CFG_URL)) { + assertFalse(((JdbcConnection)conn).isEnforceJoinOrder()); + assertTrue(((JdbcConnection)conn).isDistributedJoins()); + assertFalse(((JdbcConnection)conn).isCollocatedQuery()); + } + + try (final Connection conn = DriverManager.getConnection(CFG_URL_PREFIX + "collocated=true@" + CFG_URL)) { + assertFalse(((JdbcConnection)conn).isEnforceJoinOrder()); + assertFalse(((JdbcConnection)conn).isDistributedJoins()); + assertTrue(((JdbcConnection)conn).isCollocatedQuery()); + } + } + /** * @throws Exception If failed. */ diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcDeleteStatementSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcDeleteStatementSelfTest.java index d55c979f7e52b..55cc8b949dbce 100644 --- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcDeleteStatementSelfTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcDeleteStatementSelfTest.java @@ -17,9 +17,13 @@ package org.apache.ignite.internal.jdbc2; +import java.sql.ResultSet; import java.sql.SQLException; import java.util.Arrays; import java.util.HashSet; +import java.util.concurrent.Callable; +import javax.cache.CacheException; +import org.apache.ignite.testframework.GridTestUtils; /** * @@ -46,4 +50,30 @@ public void testExecuteUpdate() throws SQLException { assertFalse(jcache(0).containsKey("p2")); assertTrue(jcache(0).containsKeys(new HashSet(Arrays.asList("p1", "p3")))); } + + /** + * @throws Exception If failed. + */ + public void testStatementTypeMismatchUpdate() throws Exception { + GridTestUtils.assertThrowsAnyCause(log, + new Callable() { + @Override public Object call() throws Exception { + conn.createStatement().executeQuery("delete from person where id=1"); + + return null; + } + }, + CacheException.class, + "Given statement type does not match that declared by JDBC driver"); + + ResultSet rs = conn.createStatement().executeQuery("select age from person where id=1"); + + boolean next = rs.next(); + + assert next; + + assert rs.getInt(1) == 25 : "The data must not be updated. " + + "Because update statement is executed via 'executeQuery' method." + + " Data [val=" + rs.getInt(1) + ']'; + } } diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteJdbcDriver.java b/modules/core/src/main/java/org/apache/ignite/IgniteJdbcDriver.java index 0ee41c0fc5d10..9d1489a01f988 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteJdbcDriver.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteJdbcDriver.java @@ -109,6 +109,10 @@ * combination with {@code local} and/or {@code collocated} flags with {@code true} value or in case of querying * of local cache. Default value is {@code false}. * + *
  • + * {@code enforceJoinOrder} - Sets flag to enforce join order of tables in the query. If set to {@code true} + * query optimizer will not reorder tables in join. By default is {@code false}. + *
  • * * *

    Configuration of Ignite Java client based connection

    @@ -289,6 +293,9 @@ public class IgniteJdbcDriver implements Driver { /** Collocated parameter name. */ private static final String PARAM_COLLOCATED = "collocated"; + /** Parameter: enforce join order flag. */ + public static final String PARAM_ENFORCE_JOIN_ORDER = "enforceJoinOrder"; + /** Distributed joins parameter name. */ private static final String PARAM_DISTRIBUTED_JOINS = "distributedJoins"; @@ -349,6 +356,10 @@ public class IgniteJdbcDriver implements Driver { /** Whether DML streaming will overwrite existing cache entries. */ public static final String PROP_STREAMING_ALLOW_OVERWRITE = PROP_PREFIX + PARAM_STREAMING_ALLOW_OVERWRITE; + /** Enforce join order property name. */ + public static final String PROP_ENFORCE_JOIN_ORDER = PROP_PREFIX + PARAM_ENFORCE_JOIN_ORDER; + + /** Cache name property name. */ public static final String PROP_CFG = PROP_PREFIX + "cfg"; @@ -415,7 +426,8 @@ public class IgniteJdbcDriver implements Driver { new PropertyInfo("Local", info.getProperty(PROP_LOCAL), ""), new PropertyInfo("Collocated", info.getProperty(PROP_COLLOCATED), ""), new PropertyInfo("Distributed Joins", info.getProperty(PROP_DISTRIBUTED_JOINS), ""), - new PropertyInfo("Transactions Allowed", info.getProperty(PROP_TX_ALLOWED), "") + new PropertyInfo("Transactions Allowed", info.getProperty(PROP_TX_ALLOWED), ""), + new PropertyInfo("Enforce Join Order", info.getProperty(PROP_ENFORCE_JOIN_ORDER), "") ); if (info.getProperty(PROP_CFG) != null) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java index b2b0dcb312031..9008e751ac745 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java @@ -75,6 +75,7 @@ import static org.apache.ignite.IgniteJdbcDriver.PROP_CFG; import static org.apache.ignite.IgniteJdbcDriver.PROP_COLLOCATED; import static org.apache.ignite.IgniteJdbcDriver.PROP_DISTRIBUTED_JOINS; +import static org.apache.ignite.IgniteJdbcDriver.PROP_ENFORCE_JOIN_ORDER; import static org.apache.ignite.IgniteJdbcDriver.PROP_LOCAL; import static org.apache.ignite.IgniteJdbcDriver.PROP_NODE_ID; import static org.apache.ignite.IgniteJdbcDriver.PROP_TX_ALLOWED; @@ -149,6 +150,9 @@ public class JdbcConnection implements Connection { /** Allow overwrites for duplicate keys on streamed {@code INSERT}s. */ private final boolean streamAllowOverwrite; + /** Enforced join order flag. */ + private boolean enforceJoinOrder; + /** Statements. */ final Set statements = new HashSet<>(); @@ -170,6 +174,7 @@ public JdbcConnection(String url, Properties props) throws SQLException { collocatedQry = Boolean.parseBoolean(props.getProperty(PROP_COLLOCATED)); distributedJoins = Boolean.parseBoolean(props.getProperty(PROP_DISTRIBUTED_JOINS)); txAllowed = Boolean.parseBoolean(props.getProperty(PROP_TX_ALLOWED)); + enforceJoinOrder = Boolean.parseBoolean(props.getProperty(PROP_ENFORCE_JOIN_ORDER)); stream = Boolean.parseBoolean(props.getProperty(PROP_STREAMING)); streamAllowOverwrite = Boolean.parseBoolean(props.getProperty(PROP_STREAMING_ALLOW_OVERWRITE)); @@ -789,6 +794,13 @@ boolean isDistributedJoins() { return distributedJoins; } + /** + * @return Enforce join order flag. + */ + boolean isEnforceJoinOrder() { + return enforceJoinOrder; + } + /** * Ensures that connection is not closed. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTask.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTask.java index bd6b0f2c74c98..0d1d602b5856a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTask.java @@ -151,7 +151,7 @@ public JdbcQueryTask(Ignite ignite, String cacheName, String sql, throw new SQLException("Cache not found [cacheName=" + cacheName + ']'); } - SqlFieldsQuery qry = new SqlFieldsQuery(sql).setArgs(args); + SqlFieldsQuery qry = new JdbcSqlFieldsQuery(sql, true).setArgs(args); qry.setPageSize(fetchSize); qry.setLocal(locQry); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTaskV2.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTaskV2.java index 61f152dd66000..d9c91d5d12805 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTaskV2.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTaskV2.java @@ -160,6 +160,7 @@ public JdbcQueryTaskV2(Ignite ignite, String cacheName, String sql, qry.setLocal(locQry); qry.setCollocated(collocatedQry); qry.setDistributedJoins(distributedJoins); + qry.setEnforceJoinOrder(enforceJoinOrder()); QueryCursorImpl> qryCursor = (QueryCursorImpl>)cache.withKeepBinary().query(qry); @@ -211,6 +212,13 @@ else if (!loc && !CURSORS.replace(uuid, cursor, new Cursor(cursor.cursor, cursor return new QueryResult(uuid, finished, isQry, rows, cols, tbls, types); } + /** + * @return Enforce join order flag. + */ + protected boolean enforceJoinOrder() { + return false; + } + /** * Schedules removal of stored cursor in case of remote query execution. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTaskV3.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTaskV3.java new file mode 100644 index 0000000000000..5bb2454332167 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTaskV3.java @@ -0,0 +1,49 @@ +/* + * 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.jdbc2; + +import java.util.UUID; +import org.apache.ignite.Ignite; + +/** + * + */ +class JdbcQueryTaskV3 extends JdbcQueryTaskV2 { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + + /** Enforce join order flag. */ + private final boolean enforceJoinOrder; + + /** + * {@inheritDoc} + * @param enforceJoinOrder Enforce joins order falg. + */ + public JdbcQueryTaskV3(Ignite ignite, String cacheName, String sql, + Boolean isQry, boolean loc, Object[] args, int fetchSize, UUID uuid, + boolean locQry, boolean collocatedQry, boolean distributedJoins, boolean enforceJoinOrder) { + super(ignite, cacheName, sql, isQry, loc, args, fetchSize, uuid, locQry, collocatedQry, distributedJoins); + + this.enforceJoinOrder = enforceJoinOrder; + } + + /** {@inheritDoc} */ + @Override protected boolean enforceJoinOrder() { + return enforceJoinOrder; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcResultSet.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcResultSet.java index 187930ec4cc66..e398397d61441 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcResultSet.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcResultSet.java @@ -92,7 +92,9 @@ public class JdbcResultSet implements ResultSet { private final boolean useNewQryTask; /** - * Creates new result set. + * Creates new result set with predefined fields. + * Result set created with this constructor will + * never execute remote tasks. * * @param uuid Query UUID. * @param stmt Statement. @@ -100,6 +102,7 @@ public class JdbcResultSet implements ResultSet { * @param cols Column names. * @param types Types. * @param fields Fields. + * @param finished Result set finished flag (the last result set). */ JdbcResultSet(@Nullable UUID uuid, JdbcStatement stmt, List tbls, List cols, List types, Collection> fields, boolean finished) { @@ -185,8 +188,17 @@ else if (!finished) { if (useNewQryTask) { // Connections from new clients send queries with new tasks, so we have to continue in the same manner - JdbcQueryTaskV2 qryTask = new JdbcQueryTaskV2(loc ? ignite : null, conn.cacheName(), null, true, loc, null, - fetchSize, uuid, conn.isLocalQuery(), conn.isCollocatedQuery(), conn.isDistributedJoins()); + JdbcQueryTaskV2 qryTask; + if (conn.isEnforceJoinOrder()) { + qryTask = new JdbcQueryTaskV3(loc ? ignite : null, conn.cacheName(), null, true, loc, null, + fetchSize, uuid, conn.isLocalQuery(), conn.isCollocatedQuery(), + conn.isDistributedJoins(), true); + } + else { + qryTask = new JdbcQueryTaskV2(loc ? ignite : null, conn.cacheName(), null, true, loc, null, + fetchSize, uuid, conn.isLocalQuery(), conn.isCollocatedQuery(), + conn.isDistributedJoins()); + } try { JdbcQueryTaskV2.QueryResult res = @@ -243,6 +255,7 @@ else if (!finished) { /** * Marks result set as closed. * If this result set is associated with locally executed query then query cursor will also closed. + * @throws SQLException On error. */ void closeInternal() throws SQLException { if (((JdbcConnection)stmt.getConnection()).nodeId() == null && uuid != null) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStatement.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStatement.java index 44db3757eee40..9752a536641c8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStatement.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStatement.java @@ -108,28 +108,53 @@ public class JdbcStatement implements Statement { boolean loc = nodeId == null; - JdbcQueryTask qryTask = new JdbcQueryTask(loc ? ignite : null, conn.cacheName(), sql, loc, getArgs(), - fetchSize, uuid, conn.isLocalQuery(), conn.isCollocatedQuery(), conn.isDistributedJoins()); + JdbcResultSet rs; + if (conn.isEnforceJoinOrder()) { + JdbcQueryTaskV3 qryTask = new JdbcQueryTaskV3(loc ? ignite : null, conn.cacheName(), sql, true, + loc, getArgs(), fetchSize, uuid, conn.isLocalQuery(), conn.isCollocatedQuery(), + conn.isDistributedJoins(), true); - try { - JdbcQueryTask.QueryResult res = - loc ? qryTask.call() : ignite.compute(ignite.cluster().forNodeId(nodeId)).call(qryTask); + try { + JdbcQueryTaskV3.QueryResult res = + loc ? qryTask.call() : ignite.compute(ignite.cluster().forNodeId(nodeId)).call(qryTask); - JdbcResultSet rs = new JdbcResultSet(uuid, this, res.getTbls(), res.getCols(), res.getTypes(), - res.getRows(), res.isFinished()); + rs = new JdbcResultSet(uuid, this, res.getTbls(), res.getCols(), res.getTypes(), + res.getRows(), res.isFinished()); - rs.setFetchSize(fetchSize); + rs.setFetchSize(fetchSize); + } + catch (IgniteSQLException e) { + throw e.toJdbcException(); + } + catch (Exception e) { + throw new SQLException("Failed to query Ignite.", e); + } + } + else { + JdbcQueryTask qryTask = new JdbcQueryTask(loc ? ignite : null, conn.cacheName(), sql, loc, getArgs(), + fetchSize, uuid, conn.isLocalQuery(), conn.isCollocatedQuery(), + conn.isDistributedJoins()); - resSets.add(rs); + try { + JdbcQueryTask.QueryResult res = + loc ? qryTask.call() : ignite.compute(ignite.cluster().forNodeId(nodeId)).call(qryTask); - return rs; - } - catch (IgniteSQLException e) { - throw e.toJdbcException(); - } - catch (Exception e) { - throw new SQLException("Failed to query Ignite.", e); + rs = new JdbcResultSet(uuid, this, res.getTbls(), res.getCols(), res.getTypes(), + res.getRows(), res.isFinished()); + + rs.setFetchSize(fetchSize); + } + catch (IgniteSQLException e) { + throw e.toJdbcException(); + } + catch (Exception e) { + throw new SQLException("Failed to query Ignite.", e); + } } + + resSets.add(rs); + + return rs; } /** {@inheritDoc} */ @@ -165,8 +190,15 @@ long doUpdate(String sql, Object[] args) throws SQLException { if (!conn.isDmlSupported()) throw new SQLException("Failed to query Ignite: DML operations are supported in versions 1.8.0 and newer"); - JdbcQueryTaskV2 qryTask = new JdbcQueryTaskV2(loc ? ignite : null, conn.cacheName(), sql, false, loc, args, - fetchSize, uuid, conn.isLocalQuery(), conn.isCollocatedQuery(), conn.isDistributedJoins()); + JdbcQueryTaskV2 qryTask; + if (conn.isEnforceJoinOrder()) { + qryTask = new JdbcQueryTaskV3(loc ? ignite : null, conn.cacheName(), sql, false, loc, args, + fetchSize, uuid, conn.isLocalQuery(), conn.isCollocatedQuery(), conn.isDistributedJoins(), true); + } + else { + qryTask = new JdbcQueryTaskV2(loc ? ignite : null, conn.cacheName(), sql, false, loc, args, + fetchSize, uuid, conn.isLocalQuery(), conn.isCollocatedQuery(), conn.isDistributedJoins()); + } try { JdbcQueryTaskV2.QueryResult qryRes = @@ -219,6 +251,7 @@ private static long updateCounterFromQueryResult(List> rows) throws SQLE /** * Marks statement as closed and closes all result sets. + * @throws SQLException On error. */ void closeInternal() throws SQLException { for (Iterator it = resSets.iterator(); it.hasNext(); ) { @@ -332,8 +365,17 @@ void closeInternal() throws SQLException { boolean loc = nodeId == null; - JdbcQueryTaskV2 qryTask = new JdbcQueryTaskV2(loc ? ignite : null, conn.cacheName(), sql, null, loc, getArgs(), - fetchSize, uuid, conn.isLocalQuery(), conn.isCollocatedQuery(), conn.isDistributedJoins()); + JdbcQueryTaskV2 qryTask; + if (conn.isEnforceJoinOrder()) { + qryTask = new JdbcQueryTaskV3(loc ? ignite : null, conn.cacheName(), sql, null, loc, getArgs(), + fetchSize, uuid, conn.isLocalQuery(), conn.isCollocatedQuery(), + conn.isDistributedJoins(), true); + } + else { + qryTask = new JdbcQueryTaskV2(loc ? ignite : null, conn.cacheName(), sql, null, loc, getArgs(), + fetchSize, uuid, conn.isLocalQuery(), conn.isCollocatedQuery(), + conn.isDistributedJoins()); + } try { JdbcQueryTaskV2.QueryResult res = From 5356817709fcc46a7ff8fd5e0abec184139b19b9 Mon Sep 17 00:00:00 2001 From: Alexey Goncharuk Date: Thu, 29 Jun 2017 17:11:39 +0300 Subject: [PATCH 345/357] Backport IGNITE-5613 into 1.8.x IGNITE-5613 - Fixed deadlock on sequence update inside transaction (cherry picked from commit 7db925c) (cherry picked from commit 1488391f1e7b2687d212804da5a509896d452c87) --- .../GridCacheAtomicSequenceImpl.java | 249 +++++------------- 1 file changed, 64 insertions(+), 185 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java index 754d8f50a1ca9..8d636b44fe47b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java @@ -24,19 +24,16 @@ import java.io.ObjectOutput; import java.io.ObjectStreamException; import java.util.concurrent.Callable; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.internal.GridKernalContext; -import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.IgniteInternalCache; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; -import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.CU; @@ -45,8 +42,7 @@ import org.apache.ignite.lang.IgniteBiTuple; import org.jetbrains.annotations.Nullable; -import static java.util.concurrent.TimeUnit.MILLISECONDS; -import static org.apache.ignite.internal.util.typedef.internal.CU.retryTopologySafe; +import static org.apache.ignite.internal.processors.cache.GridCacheUtils.retryTopologySafe; import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ; @@ -88,7 +84,7 @@ public final class GridCacheAtomicSequenceImpl implements GridCacheAtomicSequenc /** Local value of sequence. */ @GridToStringInclude(sensitive = true) - private long locVal; + private volatile long locVal; /** Upper bound of local counter. */ private long upBound; @@ -96,11 +92,14 @@ public final class GridCacheAtomicSequenceImpl implements GridCacheAtomicSequenc /** Sequence batch size */ private volatile int batchSize; - /** Synchronization lock. */ - private final Lock lock = new ReentrantLock(); + /** Synchronization lock for local value updates. */ + private final Lock localUpdate = new ReentrantLock(); - /** Await condition. */ - private Condition cond = lock.newCondition(); + /** Synchronization for distributed sequence update. Acquired by threads with free topology (not in TX). */ + private final ReentrantLock distUpdateFreeTop = new ReentrantLock(); + + /** Synchronization for distributed sequence update. Acquired by threads with locked topology (inside TX). */ + private final ReentrantLock distUpdateLockedTop = new ReentrantLock(); /** Callable for execution {@link #incrementAndGet} operation in async and sync mode. */ private final Callable incAndGetCall = internalUpdate(1, true); @@ -108,9 +107,6 @@ public final class GridCacheAtomicSequenceImpl implements GridCacheAtomicSequenc /** Callable for execution {@link #getAndIncrement} operation in async and sync mode. */ private final Callable getAndIncCall = internalUpdate(1, false); - /** Add and get cache call guard. */ - private final AtomicBoolean updateGuard = new AtomicBoolean(); - /** * Empty constructor required by {@link Externalizable}. */ @@ -162,14 +158,7 @@ public GridCacheAtomicSequenceImpl(String name, @Override public long get() { checkRemoved(); - lock.lock(); - - try { - return locVal; - } - finally { - lock.unlock(); - } + return locVal; } /** {@inheritDoc} */ @@ -232,154 +221,51 @@ private long internalUpdate(long l, @Nullable Callable updateCall, boolean assert l > 0; - lock.lock(); + localUpdate.lock(); try { // If reserved range isn't exhausted. - if (locVal + l <= upBound) { - long curVal = locVal; + long locVal0 = locVal; - locVal += l; + if (locVal0 + l <= upBound) { + locVal = locVal0 + l; - return updated ? locVal : curVal; + return updated ? locVal0 + l : locVal0; } } finally { - lock.unlock(); + localUpdate.unlock(); } - if (updateCall == null) - updateCall = internalUpdate(l, updated); - - while (true) { - if (updateGuard.compareAndSet(false, true)) { - try { - try { - return updateCall.call(); - } - catch (IgniteCheckedException | IgniteException | IllegalStateException e) { - throw e; - } - catch (Exception e) { - throw new IgniteCheckedException(e); - } - } - finally { - lock.lock(); - - try { - updateGuard.set(false); - - cond.signalAll(); - } - finally { - lock.unlock(); - } - } - } - else { - lock.lock(); - - try { - while (locVal >= upBound && updateGuard.get()) - U.await(cond, 500, MILLISECONDS); + AffinityTopologyVersion lockedVer = ctx.shared().lockedTopologyVersion(null); - checkRemoved(); - - // If reserved range isn't exhausted. - if (locVal + l <= upBound) { - long curVal = locVal; - - locVal += l; + // We need two separate locks here because two independent thread may attempt to update the sequence + // simultaneously, one thread with locked topology and other with unlocked. + // We cannot use the same lock for both cases because it leads to a deadlock when free-topology thread + // waits for topology change, and locked topology thread waits to acquire the lock. + // If a thread has locked topology, it must bypass sync with non-locked threads, but at the same time + // we do not want multiple threads to attempt to run identical cache updates. + ReentrantLock distLock = lockedVer == null ? distUpdateFreeTop : distUpdateLockedTop; - return updated ? locVal : curVal; - } - } - finally { - lock.unlock(); - } - } - } - } - - /** - * Asynchronous sequence update operation. Will add given amount to the sequence value. - * - * @param l Increment amount. - * @param updateCall Cache call that will update sequence reservation count in accordance with l. - * @param updated If {@code true}, will return sequence value after update, otherwise will return sequence value - * prior to update. - * @return Future indicating sequence value. - * @throws IgniteCheckedException If update failed. - */ - @SuppressWarnings("SignalWithoutCorrespondingAwait") - private IgniteInternalFuture internalUpdateAsync(long l, @Nullable Callable updateCall, boolean updated) - throws IgniteCheckedException { - checkRemoved(); - - A.ensure(l > 0, " Parameter mustn't be less then 1: " + l); - - lock.lock(); + distLock.lock(); try { - // If reserved range isn't exhausted. - if (locVal + l <= upBound) { - long curVal = locVal; + if (updateCall == null) + updateCall = internalUpdate(l, updated); - locVal += l; - - return new GridFinishedFuture<>(updated ? locVal : curVal); + try { + return updateCall.call(); } - } - finally { - lock.unlock(); - } - - if (updateCall == null) - updateCall = internalUpdate(l, updated); - - while (true) { - if (updateGuard.compareAndSet(false, true)) { - try { - // This call must be outside lock. - return ctx.closures().callLocalSafe(updateCall, true); - } - finally { - lock.lock(); - - try { - updateGuard.set(false); - - cond.signalAll(); - } - finally { - lock.unlock(); - } - } + catch (IgniteCheckedException | IgniteException | IllegalStateException e) { + throw e; } - else { - lock.lock(); - - try { - while (locVal >= upBound && updateGuard.get()) - U.await(cond, 500, MILLISECONDS); - - checkRemoved(); - - // If reserved range isn't exhausted. - if (locVal + l <= upBound) { - long curVal = locVal; - - locVal += l; - - return new GridFinishedFuture<>(updated ? locVal : curVal); - } - } - finally { - lock.unlock(); - } + catch (Exception e) { + throw new IgniteCheckedException(e); } } + finally { + distLock.unlock(); + } } /** Get local batch size for this sequences. @@ -398,13 +284,13 @@ private IgniteInternalFuture internalUpdateAsync(long l, @Nullable Callabl @Override public void batchSize(int size) { A.ensure(size > 0, " Batch size can't be less then 0: " + size); - lock.lock(); + localUpdate.lock(); try { batchSize = size; } finally { - lock.unlock(); + localUpdate.unlock(); } } @@ -486,6 +372,8 @@ private IllegalStateException removedError() { private Callable internalUpdate(final long l, final boolean updated) { return retryTopologySafe(new Callable() { @Override public Long call() throws Exception { + assert distUpdateFreeTop.isHeldByCurrentThread() || distUpdateLockedTop.isHeldByCurrentThread(); + try (IgniteInternalTx tx = CU.txStartInternal(ctx, seqView, PESSIMISTIC, REPEATABLE_READ)) { GridCacheAtomicSequenceValue seq = seqView.get(key); @@ -497,48 +385,39 @@ private Callable internalUpdate(final long l, final boolean updated) { long newUpBound; - lock.lock(); - - try { - curLocVal = locVal; + curLocVal = locVal; - // If local range was already reserved in another thread. - if (locVal + l <= upBound) { - long retVal = locVal; + // If local range was already reserved in another thread. + if (curLocVal + l <= upBound) { + locVal = curLocVal + l; - locVal += l; - - return updated ? locVal : retVal; - } + return updated ? curLocVal + l : curLocVal; + } - long curGlobalVal = seq.get(); + long curGlobalVal = seq.get(); - long newLocVal; + long newLocVal; - /* We should use offset because we already reserved left side of range.*/ - long off = batchSize > 1 ? batchSize - 1 : 1; + /* We should use offset because we already reserved left side of range.*/ + long off = batchSize > 1 ? batchSize - 1 : 1; - // Calculate new values for local counter, global counter and upper bound. - if (curLocVal + l >= curGlobalVal) { - newLocVal = curLocVal + l; + // Calculate new values for local counter, global counter and upper bound. + if (curLocVal + l >= curGlobalVal) { + newLocVal = curLocVal + l; - newUpBound = newLocVal + off; - } - else { - newLocVal = curGlobalVal; + newUpBound = newLocVal + off; + } + else { + newLocVal = curGlobalVal; - newUpBound = newLocVal + off; - } + newUpBound = newLocVal + off; + } - locVal = newLocVal; - upBound = newUpBound; + locVal = newLocVal; + upBound = newUpBound; - if (updated) - curLocVal = newLocVal; - } - finally { - lock.unlock(); - } + if (updated) + curLocVal = newLocVal; // Global counter must be more than reserved upper bound. seq.set(newUpBound + 1); From b38128dcea7e665ca81ce1dc8566f8b8dcda2e7e Mon Sep 17 00:00:00 2001 From: Alexey Goncharuk Date: Mon, 3 Jul 2017 17:05:48 +0300 Subject: [PATCH 346/357] Backport IGNITE-5613 into 1.8.x IGNITE-5613 - Fixed race on local sequence increment and distributed update (cherry picked from commit 7d42dea) (cherry picked from commit 9f977ec1602e17c01207e464f81cd747539ab6dc) --- .../GridCacheAtomicSequenceImpl.java | 55 +++++++++++-------- ...tionedAtomicSequenceMultiThreadedTest.java | 32 +++++++++++ 2 files changed, 64 insertions(+), 23 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java index 8d636b44fe47b..95e2f2fa44113 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java @@ -385,39 +385,48 @@ private Callable internalUpdate(final long l, final boolean updated) { long newUpBound; - curLocVal = locVal; + // Even though we hold a transaction lock here, we must hold the local update lock here as well + // because we mutate multipe variables (locVal and upBound). + localUpdate.lock(); - // If local range was already reserved in another thread. - if (curLocVal + l <= upBound) { - locVal = curLocVal + l; + try { + curLocVal = locVal; - return updated ? curLocVal + l : curLocVal; - } + // If local range was already reserved in another thread. + if (curLocVal + l <= upBound) { + locVal = curLocVal + l; - long curGlobalVal = seq.get(); + return updated ? curLocVal + l : curLocVal; + } - long newLocVal; + long curGlobalVal = seq.get(); - /* We should use offset because we already reserved left side of range.*/ - long off = batchSize > 1 ? batchSize - 1 : 1; + long newLocVal; - // Calculate new values for local counter, global counter and upper bound. - if (curLocVal + l >= curGlobalVal) { - newLocVal = curLocVal + l; + /* We should use offset because we already reserved left side of range.*/ + long off = batchSize > 1 ? batchSize - 1 : 1; - newUpBound = newLocVal + off; - } - else { - newLocVal = curGlobalVal; + // Calculate new values for local counter, global counter and upper bound. + if (curLocVal + l >= curGlobalVal) { + newLocVal = curLocVal + l; - newUpBound = newLocVal + off; - } + newUpBound = newLocVal + off; + } + else { + newLocVal = curGlobalVal; - locVal = newLocVal; - upBound = newUpBound; + newUpBound = newLocVal + off; + } - if (updated) - curLocVal = newLocVal; + locVal = newLocVal; + upBound = newUpBound; + + if (updated) + curLocVal = newLocVal; + } + finally { + localUpdate.unlock(); + } // Global counter must be more than reserved upper bound. seq.set(newUpBound + 1); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedAtomicSequenceMultiThreadedTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedAtomicSequenceMultiThreadedTest.java index 945650da6b8f6..4db9bd3339904 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedAtomicSequenceMultiThreadedTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedAtomicSequenceMultiThreadedTest.java @@ -19,6 +19,7 @@ import java.util.Random; import java.util.UUID; +import java.util.concurrent.Callable; import org.apache.ignite.IgniteAtomicSequence; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cache.CacheMode; @@ -26,6 +27,7 @@ import org.apache.ignite.internal.processors.cache.datastructures.IgniteAtomicsAbstractTest; import org.apache.ignite.internal.processors.datastructures.GridCacheAtomicSequenceImpl; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.testframework.GridTestUtils; import static org.apache.ignite.cache.CacheMode.PARTITIONED; @@ -280,6 +282,36 @@ public void testMixed2() throws Exception { assertEquals(17 * ITERATION_NUM * THREAD_NUM, seq.get()); } + /** + * @throws Exception if failed. + */ + public void testMultipleSequences() throws Exception { + final int seqCnt = 5; + final int threadCnt = 5; + final int incCnt = 1_000; + + final IgniteAtomicSequence[] seqs = new IgniteAtomicSequence[seqCnt]; + + String seqName = UUID.randomUUID().toString(); + + for (int i = 0; i < seqs.length; i++) + seqs[i] = grid(0).atomicSequence(seqName, 0, true); + + GridTestUtils.runMultiThreaded(new Callable() { + @Override public Object call() throws Exception { + for (int i = 0; i < incCnt; i++) { + for (IgniteAtomicSequence seq : seqs) + seq.incrementAndGet(); + } + + return null; + } + }, threadCnt, "load"); + + for (IgniteAtomicSequence seq : seqs) + assertEquals(seqCnt * threadCnt * incCnt, seq.get()); + } + /** * Executes given closure in a given number of threads given number of times. * From 30b364adda12ee9a7344edf8a2dddf4083b2c037 Mon Sep 17 00:00:00 2001 From: nikolay_tikhonov Date: Wed, 15 Nov 2017 17:24:45 +0300 Subject: [PATCH 347/357] Fixed javadoc build. Signed-off-by: nikolay_tikhonov --- .../internal/processors/cache/query/GridCacheQueryRequest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java index 4a7e14f9f8789..e00db0a4eb85f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java @@ -483,7 +483,7 @@ public int taskHash() { } /** - * This method needed for scan query. {@link #GridCacheQueryRequest#partition()} method related with IO policy and cannot be used + * This method needed for scan query. {@link #partition()} method related with IO policy and cannot be used * for this role. Otherwise Query will be processed in striped pool. * * @return partition. From 5043121c5ae75f899d86368725ba5bc513e304b4 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Wed, 15 Nov 2017 16:04:50 +0300 Subject: [PATCH 348/357] GG-13060 - Fix NPE on started node on GridDhtPartitionsSingleRequest. --- .../GridCachePartitionExchangeManager.java | 8 +++++++- .../GridDhtPartitionsExchangeFuture.java | 19 +++++++++++++++++++ 2 files changed, 26 insertions(+), 1 deletion(-) 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 93fe6a835053b..441201dfc4849 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 @@ -1357,7 +1357,13 @@ private void processSinglePartitionRequest(ClusterNode node, GridDhtPartitionsSi return; try { - sendLocalPartitions(node, msg.exchangeId()); + final GridDhtPartitionsExchangeFuture exchFut = exchangeFuture(msg.exchangeId(), + null, + null, + null, + null); + + exchFut.onReceivePartitionRequest(node); } finally { leaveBusy(); 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 59084f542566c..70311bd2914f1 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 @@ -1615,6 +1615,25 @@ private void sendAllPartitions(final UUID nodeId, final int retryCnt) { } } + /** + * @param node Sender node. + */ + public void onReceivePartitionRequest(final ClusterNode node) { + assert !cctx.kernalContext().clientNode(); + assert !node.isDaemon() && !CU.clientNode(node) : node; + + initFut.listen(new CI1>() { + @Override public void apply(IgniteInternalFuture fut) { + try { + sendLocalPartitions(node); + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to send message to coordinator: " + e); + } + } + }); + } + /** * @param node Sender node. * @param msg Full partition info. From 039b44041741b862f584f160419d9612a7ee7366 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Thu, 16 Nov 2017 11:31:56 +0300 Subject: [PATCH 349/357] GG-13067 - Fix GridClockSyncProcessor may produce NPE on node stop. --- .../internal/processors/clock/GridClockSyncProcessor.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) 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 3586956da99dd..c5cbbec4d6b24 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 @@ -153,6 +153,9 @@ public GridClockSyncProcessor(GridKernalContext ctx) { try { stopping = false; + if (srv != null) + srv.beforeStop(); + if (timeCoord != null) { timeCoord.cancel(); @@ -160,9 +163,6 @@ public GridClockSyncProcessor(GridKernalContext ctx) { timeCoord = null; } - - if (srv != null) - srv.beforeStop(); } finally { rw.writeUnlock(); From 5bfb748d54155f3a008c9f760487d8f967798bbc Mon Sep 17 00:00:00 2001 From: Ilya Kasnacheev Date: Thu, 16 Nov 2017 15:12:20 +0300 Subject: [PATCH 350/357] GG-12706 Unmute IgniteCacheMessageWriteTimeoutTest fixed by IGNITE-6818. --- .../cache/distributed/IgniteCacheMessageWriteTimeoutTest.java | 2 -- 1 file changed, 2 deletions(-) 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 8f3f7723efb60..da9f11f6a7dd8 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 @@ -75,8 +75,6 @@ public class IgniteCacheMessageWriteTimeoutTest extends GridCommonAbstractTest { * @throws Exception If failed. */ public void testMessageQueueLimit() throws Exception { - fail("https://ggsystems.atlassian.net/browse/GG-12398"); - for (int i = 0; i < 15; i++) { log.info("Iteration: " + i); From 0d6adf7c0dc1b40177e9cf630036e6b7979f4024 Mon Sep 17 00:00:00 2001 From: Dmitriy Govorukhin Date: Thu, 25 May 2017 20:19:22 +0300 Subject: [PATCH 351/357] Backported ignite-gg-12138 stop nodes after test + small test refactoring (cherry picked from commit 54beab9) --- .../tcp/ipfinder/TcpDiscoveryIpFinderAbstractSelfTest.java | 7 +++++++ .../tcp/ipfinder/vm/TcpDiscoveryVmIpFinderSelfTest.java | 7 +++++-- 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/TcpDiscoveryIpFinderAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/TcpDiscoveryIpFinderAbstractSelfTest.java index 6f2201f8d34e4..465b38dbe7859 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/TcpDiscoveryIpFinderAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/TcpDiscoveryIpFinderAbstractSelfTest.java @@ -54,6 +54,13 @@ protected TcpDiscoveryIpFinderAbstractSelfTest() throws Exception { injectLogger(finder); } + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + super.afterTest(); + + stopAllGrids(); + } + /** * @throws Exception If any error occurs. */ diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/vm/TcpDiscoveryVmIpFinderSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/vm/TcpDiscoveryVmIpFinderSelfTest.java index aa000071647d3..85f1a9ddf29d0 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/vm/TcpDiscoveryVmIpFinderSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/vm/TcpDiscoveryVmIpFinderSelfTest.java @@ -223,6 +223,7 @@ public void testUnregistration() throws Exception { Ignition.stop("client1", true); Ignition.stop("client2", true); + Ignition.stop("client3", true); assertEquals(3 * srvSize, IP_FINDER.getRegisteredAddresses().size()); @@ -237,12 +238,14 @@ public void testUnregistration() throws Exception { Ignition.stop("server1", true); Ignition.stop("server2", true); - GridTestUtils.waitForCondition(new GridAbsPredicate() { + boolean res = GridTestUtils.waitForCondition(new GridAbsPredicate() { @Override public boolean apply() { - return 0 == G.allGrids().size(); + return G.allGrids().isEmpty(); } }, 10000); + assertTrue(res); + assertTrue(3 * srvSize >= IP_FINDER.getRegisteredAddresses().size()); } From 2af9c2615e7156e246bb0d662f14dbc9c4e13490 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Fri, 9 Jun 2017 00:34:37 +0300 Subject: [PATCH 352/357] Fixed missed node stop after tests. (cherry picked from commit 2252510) --- .../internal/ClusterGroupHostsSelfTest.java | 7 ++ .../internal/GridStartStopSelfTest.java | 65 ++++++++++--------- ...dCachePartitionedUnloadEventsSelfTest.java | 48 ++++++++------ .../near/GridCacheNearTxForceKeyTest.java | 21 +++--- .../near/NearCacheSyncUpdateTest.java | 7 ++ 5 files changed, 90 insertions(+), 58 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/ClusterGroupHostsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/ClusterGroupHostsSelfTest.java index d0251928e3f36..9b0881a6788c9 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/ClusterGroupHostsSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/ClusterGroupHostsSelfTest.java @@ -44,6 +44,13 @@ public class ClusterGroupHostsSelfTest extends GridCommonAbstractTest { startGrid(); } + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + stopAllGrids(); + } + /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { Collection hostNames = Arrays.asList("h_1", "h_2", "h_3"); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridStartStopSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridStartStopSelfTest.java index a08db921bf76e..c0540d3a89508 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/GridStartStopSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/GridStartStopSelfTest.java @@ -85,54 +85,61 @@ public void testStopWhileInUse() throws Exception { cfg.setCacheConfiguration(cc); - final Ignite g0 = G.start(cfg); + try { + final Ignite g0 = G.start(cfg); - cfg = new IgniteConfiguration(); + cfg = new IgniteConfiguration(); - cfg.setGridName(getTestGridName(1)); + cfg.setGridName(getTestGridName(1)); - cc = new CacheConfiguration(); + cc = new CacheConfiguration(); - cc.setAtomicityMode(TRANSACTIONAL); + cc.setAtomicityMode(TRANSACTIONAL); - cfg.setCacheConfiguration(cc); + cfg.setCacheConfiguration(cc); - final CountDownLatch latch = new CountDownLatch(1); + final CountDownLatch latch = new CountDownLatch(1); - Ignite g1 = G.start(cfg); + Ignite g1 = G.start(cfg); - Thread stopper = new Thread(new Runnable() { - @Override public void run() { - try { - try (Transaction ignored = g0.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { - g0.cache(null).get(1); + Thread stopper = new Thread(new Runnable() { + @Override public void run() { + try { + try (Transaction ignored = g0.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + g0.cache(null).get(1); - latch.countDown(); + latch.countDown(); - Thread.sleep(500); + Thread.sleep(500); - info("Before stop."); + info("Before stop."); - G.stop(getTestGridName(1), true); + G.stop(getTestGridName(1), true); + } + } + catch (Exception e) { + error("Error.", e); } } - catch (Exception e) { - error("Error.", e); - } - } - }); + }); - stopper.start(); + stopper.start(); - assert latch.await(1, SECONDS); + assert latch.await(1, SECONDS); - info("Before remove."); + info("Before remove."); - try { - g1.cache(null).remove(1); + try { + g1.cache(null).remove(1); + } + catch (CacheException ignore) { + // No-op. + } + + stopper.join(); } - catch (CacheException ignore) { - // No-op. + finally { + stopAllGrids(); } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedUnloadEventsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedUnloadEventsSelfTest.java index 98f3cd71f9f78..9912bfba06c3c 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedUnloadEventsSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedUnloadEventsSelfTest.java @@ -47,6 +47,9 @@ public class GridCachePartitionedUnloadEventsSelfTest extends GridCommonAbstract /** */ private TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); + /** */ + private static final int EVENTS_COUNT = 40; + /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(gridName); @@ -76,38 +79,41 @@ protected CacheConfiguration cacheConfiguration() { * @throws Exception if failed. */ public void testUnloadEvents() throws Exception { - final Ignite g1 = startGrid("g1"); - - Collection allKeys = new ArrayList<>(100); + try { + final Ignite g1 = startGrid("g1"); - IgniteCache cache = g1.cache(null); + Collection allKeys = new ArrayList<>(EVENTS_COUNT); - for (int i = 0; i < 100; i++) { - cache.put(i, "val"); - allKeys.add(i); - } + IgniteCache cache = g1.cache(null); - Ignite g2 = startGrid("g2"); + for (int i = 0; i < EVENTS_COUNT; i++) { + cache.put(i, "val"); + allKeys.add(i); + } - awaitPartitionMapExchange(); + Ignite g2 = startGrid("g2"); - Map> keysMap = g1.affinity(null).mapKeysToNodes(allKeys); - Collection g2Keys = keysMap.get(g2.cluster().localNode()); + awaitPartitionMapExchange(); - assertNotNull(g2Keys); - assertFalse("There are no keys assigned to g2", g2Keys.isEmpty()); + Map> keysMap = g1.affinity(null).mapKeysToNodes(allKeys); + Collection g2Keys = keysMap.get(g2.cluster().localNode()); - Thread.sleep(5000); + assertNotNull(g2Keys); + assertFalse("There are no keys assigned to g2", g2Keys.isEmpty()); - Collection objEvts = - g1.events().localQuery(F.alwaysTrue(), EVT_CACHE_REBALANCE_OBJECT_UNLOADED); + Collection objEvts = + g1.events().localQuery(F.alwaysTrue(), EVT_CACHE_REBALANCE_OBJECT_UNLOADED); - checkObjectUnloadEvents(objEvts, g1, g2Keys); + checkObjectUnloadEvents(objEvts, g1, g2Keys); - Collection partEvts = - g1.events().localQuery(F.alwaysTrue(), EVT_CACHE_REBALANCE_PART_UNLOADED); + Collection partEvts = + g1.events().localQuery(F.alwaysTrue(), EVT_CACHE_REBALANCE_PART_UNLOADED); - checkPartitionUnloadEvents(partEvts, g1, dht(g2.cache(null)).topology().localPartitions()); + checkPartitionUnloadEvents(partEvts, g1, dht(g2.cache(null)).topology().localPartitions()); + } + finally { + stopAllGrids(); + } } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearTxForceKeyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearTxForceKeyTest.java index 80b447cab86be..f1b17262dcccb 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearTxForceKeyTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearTxForceKeyTest.java @@ -65,19 +65,24 @@ public class GridCacheNearTxForceKeyTest extends GridCommonAbstractTest { * @throws Exception If failed. */ public void testNearTx() throws Exception { - Ignite ignite0 = startGrid(0); + try { + Ignite ignite0 = startGrid(0); - IgniteCache cache = ignite0.cache(null); + IgniteCache cache = ignite0.cache(null); - Ignite ignite1 = startGrid(1); + Ignite ignite1 = startGrid(1); - // This key should become primary for ignite1. - final Integer key = ignite0.configuration().getMarshaller() instanceof OptimizedMarshaller ? 2 : 7; + // This key should become primary for ignite1. + final Integer key = ignite0.configuration().getMarshaller() instanceof OptimizedMarshaller ? 2 : 7; - assertNull(cache.getAndPut(key, key)); + assertNull(cache.getAndPut(key, key)); - awaitPartitionMapExchange(); + awaitPartitionMapExchange(); - assertTrue(ignite0.affinity(null).isPrimary(ignite1.cluster().localNode(), key)); + assertTrue(ignite0.affinity(null).isPrimary(ignite1.cluster().localNode(), key)); + } + finally { + stopAllGrids(); + } } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/NearCacheSyncUpdateTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/NearCacheSyncUpdateTest.java index bbdf50ec7e45f..b29bbbc6ab8f5 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/NearCacheSyncUpdateTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/NearCacheSyncUpdateTest.java @@ -63,6 +63,13 @@ public class NearCacheSyncUpdateTest extends GridCommonAbstractTest { startGridsMultiThreaded(3); } + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + stopAllGrids(); + } + /** * @throws Exception If failed. */ From 67e2bdf04c84145ea6113cf441042a3dfdc9957b Mon Sep 17 00:00:00 2001 From: Vitaliy Biryukov Date: Tue, 19 Sep 2017 13:23:45 +0300 Subject: [PATCH 353/357] Fixed missed node stop after tests. --- .../cache/query/IgniteCacheQueryCacheDestroySelfTest.java | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IgniteCacheQueryCacheDestroySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IgniteCacheQueryCacheDestroySelfTest.java index dc104ff229fe5..be52fc262e257 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IgniteCacheQueryCacheDestroySelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IgniteCacheQueryCacheDestroySelfTest.java @@ -49,6 +49,13 @@ public class IgniteCacheQueryCacheDestroySelfTest extends GridCommonAbstractTest /** */ public static final int GRID_CNT = 3; + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + stopAllGrids(); + } + /** * The main test code. */ From 7e6ba3310dddfd45c50d3aff44ebf8bdd7edeb7c Mon Sep 17 00:00:00 2001 From: Slava Koptilin Date: Fri, 17 Nov 2017 16:43:35 +0300 Subject: [PATCH 354/357] ignite-6924: Fixed missed CacheStoreSessionListener#onSessionStart() call --- .../cache/store/CacheStoreManager.java | 12 +- .../store/GridCacheStoreManagerAdapter.java | 7 + .../store/GridCacheWriteBehindStore.java | 10 +- ...SessionListenerWriteBehindEnabledTest.java | 124 ++++++++++++++++-- 4 files changed, 140 insertions(+), 13 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java index b096edf6a8b95..9938a1cd87479 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java @@ -172,12 +172,22 @@ public boolean removeAll(@Nullable IgniteInternalTx tx, Collection keys) public void sessionEnd(IgniteInternalTx tx, boolean commit, boolean last, boolean storeSessionEnded) throws IgniteCheckedException; /** - * End session initiated by write-behind store. + * Start session initiated by write-behind store. * * @throws IgniteCheckedException If failed. */ public void writeBehindSessionInit() throws IgniteCheckedException; + /** + * Notifies cache store session listeners. + * + * This method is called by write-behind store in case of back-pressure mechanism is initiated. + * It is assumed that cache store session was started by CacheStoreManager before. + * + * @throws IgniteCheckedException If failed. + */ + public void writeBehindCacheStoreSessionListenerStart() throws IgniteCheckedException; + /** * End session initiated by write-behind store. * 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 3dbec4d21a56a..937b85630264b 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 @@ -839,6 +839,13 @@ private void handleClassCastException(ClassCastException e) throws IgniteChecked sessionInit0(null, null, true); } + /** {@inheritDoc} */ + @Override public void writeBehindCacheStoreSessionListenerStart() throws IgniteCheckedException { + assert sesHolder.get() != null; + + notifyCacheStoreSessionListeners(sesHolder.get(), null, true); + } + /** {@inheritDoc} */ @Override public void writeBehindSessionEnd(boolean threwEx) throws IgniteCheckedException { sessionEnd0(null, threwEx); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java index 8537aaba69282..82f88af3ef1b1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java @@ -795,8 +795,14 @@ private boolean updateStore( Flusher flusher ) { try { - if (initSes && storeMgr != null) - storeMgr.writeBehindSessionInit(); + if (storeMgr != null) { + if (initSes) + storeMgr.writeBehindSessionInit(); + else + // Back-pressure mechanism is running. + // Cache store session must be initialized by storeMgr. + storeMgr.writeBehindCacheStoreSessionListenerStart(); + } boolean threwEx = true; diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerWriteBehindEnabledTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerWriteBehindEnabledTest.java index 2b04309a40422..143a16b99a3f3 100644 --- a/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerWriteBehindEnabledTest.java +++ b/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerWriteBehindEnabledTest.java @@ -26,6 +26,8 @@ import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.CyclicBarrier; import java.util.concurrent.atomic.AtomicInteger; import java.util.logging.Logger; import javax.cache.Cache; @@ -34,14 +36,15 @@ import javax.cache.integration.CacheLoaderException; import javax.cache.integration.CacheWriterException; import javax.sql.DataSource; -import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteException; import org.apache.ignite.cache.store.jdbc.CacheJdbcStoreSessionListener; import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.processors.cache.GridCacheAbstractSelfTest; import org.apache.ignite.internal.processors.cache.store.GridCacheWriteBehindStore; -import org.apache.ignite.resources.IgniteInstanceResource; +import org.apache.ignite.resources.CacheStoreSessionResource; +import org.apache.ignite.testframework.GridTestUtils; /** * This class tests that calls of {@link CacheStoreSessionListener#onSessionStart(CacheStoreSession)} @@ -64,14 +67,17 @@ public class CacheStoreSessionListenerWriteBehindEnabledTest extends GridCacheAb /** */ private static final AtomicInteger entryCnt = new AtomicInteger(); + /** */ + private static final AtomicInteger uninitializedListenerCnt = new AtomicInteger(); + /** {@inheritDoc} */ @Override protected int gridCount() { return 1; } /** {@inheritDoc} */ - @Override protected CacheConfiguration cacheConfiguration(String igniteInstanceName) throws Exception { - CacheConfiguration cacheCfg = super.cacheConfiguration(igniteInstanceName); + @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception { + CacheConfiguration cacheCfg = super.cacheConfiguration(gridName); cacheCfg.setName(DEFAULT_CACHE_NAME); @@ -98,6 +104,8 @@ public class CacheStoreSessionListenerWriteBehindEnabledTest extends GridCacheAb operations.clear(); entryCnt.set(0); + + uninitializedListenerCnt.set(0); } /** @@ -140,6 +148,85 @@ public void testRemove() { checkSessionCounters(1); } + /** + * Tests that cache store session listeners are notified by write-behind store. + */ + public void testFlushSingleValue() throws Exception { + CacheConfiguration cfg = cacheConfiguration(getTestGridName()); + + cfg.setName("back-pressure-control"); + cfg.setWriteBehindBatchSize(2); + cfg.setWriteBehindFlushSize(2); + cfg.setWriteBehindFlushFrequency(1_000); + cfg.setWriteBehindCoalescing(true); + + IgniteCache cache = grid(0).getOrCreateCache(cfg); + + try { + int nUploaders = 5; + + final CyclicBarrier barrier = new CyclicBarrier(nUploaders); + + IgniteInternalFuture[] uploaders = new IgniteInternalFuture[nUploaders]; + + for (int i = 0; i < nUploaders; ++i) { + uploaders[i] = GridTestUtils.runAsync( + new Uploader(cache, barrier, i * CNT), + "uploader-" + i); + } + + for (int i = 0; i < nUploaders; ++i) + uploaders[i].get(); + + assertEquals("Uninitialized cache store session listener.", 0, uninitializedListenerCnt.get()); + } + finally { + cache.destroy(); + } + } + + /** + * + */ + public static class Uploader implements Callable { + /** */ + private final int start; + + /** */ + private final CyclicBarrier barrier; + + /** */ + private final IgniteCache cache; + + /** + * @param cache Ignite cache. + * @param barrier Cyclic barrier. + * @param start Key index. + */ + public Uploader(IgniteCache cache, CyclicBarrier barrier, int start) { + this.cache = cache; + + this.barrier = barrier; + + this.start = start; + } + + /** {@inheritDoc} */ + @Override public Void call() { + try { + barrier.await(); + + for (int i = start; i < start + CNT; ++i) + cache.put(i, i); + } + catch (Exception e) { + fail("Unexpected exception [" + e + "]"); + } + + return null; + } + } + /** * @param startedSessions Number of expected sessions. */ @@ -150,6 +237,8 @@ private void checkSessionCounters(int startedSessions) { assertEquals(CNT, entryCnt.get()); + assertEquals("Uninitialized cache store session listener.", 0, uninitializedListenerCnt.get()); + checkOpCount(operations, OperationType.SESSION_START, startedSessions); checkOpCount(operations, OperationType.SESSION_END, startedSessions); @@ -206,18 +295,19 @@ public static class CacheStoreSessionFactory implements Factory { /** */ - @IgniteInstanceResource - private Ignite ignite; + @CacheStoreSessionResource + private CacheStoreSession ses; /** {@inheritDoc} */ @Override public Object load(Object key) throws CacheLoaderException { entryCnt.getAndIncrement(); + + if (ses.attachment() == null) + uninitializedListenerCnt.incrementAndGet(); + return null; } /** {@inheritDoc} */ @Override public void writeAll(Collection> entries) { entryCnt.addAndGet(entries.size()); + + if (ses.attachment() == null) + uninitializedListenerCnt.incrementAndGet(); } /** {@inheritDoc} */ @Override public void write(Cache.Entry entry) throws CacheWriterException { + if (ses.attachment() == null) + uninitializedListenerCnt.incrementAndGet(); } /** {@inheritDoc} */ @Override public void deleteAll(Collection keys) { entryCnt.addAndGet(keys.size()); + + if (ses.attachment() == null) + uninitializedListenerCnt.incrementAndGet(); } /** {@inheritDoc} */ @Override public void delete(Object key) throws CacheWriterException { + if (ses.attachment() == null) + uninitializedListenerCnt.incrementAndGet(); } } From 53e93afd936e8a257c99dddfff5222c40464738a Mon Sep 17 00:00:00 2001 From: Slava Koptilin Date: Fri, 17 Nov 2017 15:53:15 +0300 Subject: [PATCH 355/357] IGNITE-6496 Client node should release queue semaphore on disconnect - Fixes #2981. Signed-off-by: Alexey Goncharuk --- .../CacheDataStructuresManager.java | 12 ++ .../datastructures/GridCacheQueueAdapter.java | 5 + .../GridCacheQueueClientDisconnectTest.java | 119 ++++++++++++++++++ ...gniteCacheDataStructuresSelfTestSuite.java | 2 + 4 files changed, 138 insertions(+) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueClientDisconnectTest.java 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 2b3080981ec36..92e5d51e78e72 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 @@ -62,6 +62,7 @@ import org.apache.ignite.internal.util.GridSpinBusyLock; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.resources.IgniteInstanceResource; import org.jetbrains.annotations.NotNull; @@ -135,6 +136,17 @@ public CacheDataStructuresManager() { q.delegate().onKernalStop(); } + /** {@inheritDoc} */ + @Override public void onDisconnected(IgniteFuture reconnectFut) { + super.onDisconnected(reconnectFut); + + for (Map.Entry e : queuesMap.entrySet()) { + GridCacheQueueProxy queue = e.getValue(); + + queue.delegate().onClientDisconnected(); + } + } + /** * @param set Set. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheQueueAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheQueueAdapter.java index 6e087e69c3017..41b6161a1a618 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheQueueAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheQueueAdapter.java @@ -486,6 +486,11 @@ protected final void checkRemoved(@Nullable GridCacheQueueHeader hdr) { onRemoved(true); } + /** Release all semaphores used in blocking operations in case of client disconnect. */ + public void onClientDisconnected() { + releaseSemaphores(); + } + /** * Marks queue as removed. * diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueClientDisconnectTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueClientDisconnectTest.java new file mode 100644 index 0000000000000..98f3544bdeeb4 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueClientDisconnectTest.java @@ -0,0 +1,119 @@ +/* + * 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.datastructures; + +import java.util.concurrent.Callable; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteClientDisconnectedException; +import org.apache.ignite.IgniteQueue; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.configuration.CollectionConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +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; + +public class GridCacheQueueClientDisconnectTest extends GridCommonAbstractTest { + /** */ + private static final String IGNITE_QUEUE_NAME = "ignite-queue-client-reconnect-test"; + + /** */ + private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); + + /** */ + private static final int FAILURE_DETECTION_TIMEOUT = 10_000; + + /** */ + private boolean clientMode; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + TcpDiscoverySpi spi = new TcpDiscoverySpi(); + + spi.setIpFinder(ipFinder); + + spi.setClientReconnectDisabled(false); + + cfg.setDiscoverySpi(spi); + + cfg.setFailureDetectionTimeout(FAILURE_DETECTION_TIMEOUT); + + if (clientMode) + cfg.setClientMode(true); + + return cfg; + } + + private static CollectionConfiguration collectionConfiguration(CacheAtomicityMode cacheAtomicityMode) { + CollectionConfiguration colCfg = new CollectionConfiguration(); + + colCfg.setAtomicityMode(cacheAtomicityMode); + + return colCfg; + } + + public void testClientDisconnect() throws Exception { + try { + Ignite server = startGrid(0); + + clientMode = true; + + Ignite client = startGrid(1); + + awaitPartitionMapExchange(); + + final IgniteQueue queue = client.queue( + IGNITE_QUEUE_NAME, 0, collectionConfiguration(CacheAtomicityMode.ATOMIC)); + + final CountDownLatch latch = new CountDownLatch(1); + + GridTestUtils.runAsync(new Callable() { + @Override public Void call() { + try { + Object value = queue.take(); + } + catch (IgniteClientDisconnectedException icd) { + latch.countDown(); + } + catch (Exception e) { + } + + return null; + } + }); + + U.sleep(5000); + + server.close(); + + boolean countReachedZero = latch.await(FAILURE_DETECTION_TIMEOUT * 2, TimeUnit.MILLISECONDS); + + assertTrue("IgniteClientDisconnectedException was not thrown", countReachedZero); + } + finally { + stopAllGrids(); + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java index 980d550ecd608..bb0e1a3f16413 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java @@ -19,6 +19,7 @@ import junit.framework.TestSuite; import org.apache.ignite.internal.processors.cache.datastructures.GridCacheQueueCleanupSelfTest; +import org.apache.ignite.internal.processors.cache.datastructures.GridCacheQueueClientDisconnectTest; import org.apache.ignite.internal.processors.cache.datastructures.GridCacheQueueMultiNodeConsistencySelfTest; import org.apache.ignite.internal.processors.cache.datastructures.IgniteClientDataStructuresTest; import org.apache.ignite.internal.processors.cache.datastructures.IgniteClientDiscoveryDataStructuresTest; @@ -133,6 +134,7 @@ public static TestSuite suite() throws Exception { suite.addTest(new TestSuite(GridCachePartitionedQueueMultiNodeSelfTest.class)); suite.addTest(new TestSuite(GridCachePartitionedAtomicQueueMultiNodeSelfTest.class)); suite.addTest(new TestSuite(GridCachePartitionedAtomicOffheapQueueMultiNodeSelfTest.class)); + suite.addTest(new TestSuite(GridCacheQueueClientDisconnectTest.class)); suite.addTest(new TestSuite(GridCachePartitionedQueueCreateMultiNodeSelfTest.class)); suite.addTest(new TestSuite(GridCachePartitionedAtomicQueueCreateMultiNodeSelfTest.class)); From 75eef0dbc53710fa9d572a60e71af308634b5f63 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Fri, 17 Nov 2017 12:54:26 +0300 Subject: [PATCH 356/357] GG-13072 - Fix half open connection when server initiated connection. --- .../spi/communication/tcp/TcpCommunicationSpi.java | 11 ++++++----- 1 file changed, 6 insertions(+), 5 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 55e2b6b95af98..1432b51e8454b 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 @@ -529,7 +529,7 @@ private void onFirstMessage(final GridNioSession ses, Message msg) { if (c.failed) { ses.send(new RecoveryLastReceivedMessage(-1)); - closeStaleConnections(connKey); + closeStaleConnections(connKey, true); } } } @@ -554,7 +554,7 @@ private void onFirstMessage(final GridNioSession ses, Message msg) { ses.send(new RecoveryLastReceivedMessage(-1)); - closeStaleConnections(connKey); + closeStaleConnections(connKey, false); return; } @@ -588,7 +588,7 @@ private void onFirstMessage(final GridNioSession ses, Message msg) { ses.send(new RecoveryLastReceivedMessage(-1)); - closeStaleConnections(connKey); + closeStaleConnections(connKey, false); fut.onDone(oldClient); @@ -645,12 +645,13 @@ private void onFirstMessage(final GridNioSession ses, Message msg) { /** * @param connKey Connection key. + * @param paired Paired connection. */ - private void closeStaleConnections(ConnectionKey connKey) { + private void closeStaleConnections(ConnectionKey connKey, boolean paired) { for (GridNioSession ses0 : nioSrvr.sessions()) { ConnectionKey key0 = ses0.meta(CONN_IDX_META); - if (ses0.accepted() && key0 != null && + if ((!paired || ses0.accepted()) && key0 != null && key0.nodeId().equals(connKey.nodeId()) && key0.connectionIndex() == connKey.connectionIndex() && key0.connectCount() < connKey.connectCount()) From db0c0a024caf972f7b16e3d35bfca028e9ef3fc7 Mon Sep 17 00:00:00 2001 From: Slava Koptilin Date: Fri, 17 Nov 2017 18:55:05 +0300 Subject: [PATCH 357/357] fixed JdbcStreamingSelfTest, JdbcAbstractDmlStatementSelfTest --- .../internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java | 4 ++++ .../apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java | 1 + 2 files changed, 5 insertions(+) diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java index 4965acff26b7d..f37e60e898cf1 100644 --- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java @@ -207,7 +207,11 @@ protected String getCfgUrl() { grid(0).cache(null).clear(); + conn.close(); + assertEquals(0, grid(0).cache(null).size(CachePeekMode.ALL)); + + assertTrue(conn.isClosed()); } /** diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java index ead2a6e6d4563..8a0dc6153bcff 100644 --- a/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java +++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java @@ -72,6 +72,7 @@ public static TestSuite suite() throws Exception { suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcInsertStatementSelfTest.class)); suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcBinaryMarshallerInsertStatementSelfTest.class)); suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcDeleteStatementSelfTest.class)); +// suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcUpdateStatementSelfTest.class)); suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcStreamingSelfTest.class)); suite.addTest(new TestSuite(JdbcBlobTest.class));